Merge branch 'gerrit/7.1.x' into 'gerrit/neo'
Ext-ref: MB-64327
Change-Id: Id8dc7cfa24e3458ad2f287283a89a4d2fe24637e
diff --git a/asterixdb/asterix-active/pom.xml b/asterixdb/asterix-active/pom.xml
index 971534f..55c5b66 100644
--- a/asterixdb/asterix-active/pom.xml
+++ b/asterixdb/asterix-active/pom.xml
@@ -21,7 +21,7 @@
<parent>
<groupId>org.apache.asterix</groupId>
<artifactId>apache-asterixdb</artifactId>
- <version>0.9.8-SNAPSHOT</version>
+ <version>0.9.8.2-SNAPSHOT</version>
</parent>
<artifactId>asterix-active</artifactId>
<properties>
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveManager.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveManager.java
index b99e4f2..a75ee33 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveManager.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveManager.java
@@ -104,9 +104,9 @@
}
public void submit(ActiveManagerMessage message) throws HyracksDataException {
- LOGGER.debug("Message of type {} received in {}", message.getKind(), nodeId);
switch (message.getKind()) {
case STOP_ACTIVITY:
+ LOGGER.debug("Message of type {} received in {}", message.getKind(), nodeId);
stopRuntime(message);
break;
case REQUEST_STATS:
@@ -140,8 +140,6 @@
IActiveRuntime runtime = runtimes.get(runtimeId);
long reqId = message.getReqId();
if (runtime == null) {
- LOGGER.warn("Request stats of a runtime that is not registered {}; sending failure response",
- runtimeId);
// Send a failure message
((NodeControllerService) serviceCtx.getControllerService()).sendApplicationMessageToCC(
message.getCcId(),
@@ -151,7 +149,6 @@
return;
}
String stats = runtime.getStats();
- LOGGER.debug("Sending stats response for {} ", runtimeId);
ActiveStatsResponse response = new ActiveStatsResponse(reqId, stats, null);
((NodeControllerService) serviceCtx.getControllerService()).sendRealTimeApplicationMessageToCC(
message.getCcId(), JavaSerializationUtils.serialize(response), null);
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveNotificationHandler.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveNotificationHandler.java
index 60bfa69..9a05148 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveNotificationHandler.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveNotificationHandler.java
@@ -19,6 +19,8 @@
*/
package org.apache.asterix.active;
+import java.util.Collection;
+
import org.apache.asterix.active.message.ActivePartitionMessage;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -53,6 +55,11 @@
void unregisterListener(IActiveEntityEventsListener listener) throws HyracksDataException;
/**
+ * @return List of all the registered event listeners
+ */
+ Collection<IActiveEntityEventsListener> getEventListenersAsList();
+
+ /**
* @return all the registered event listeners
*/
IActiveEntityEventsListener[] getEventListeners();
diff --git a/asterixdb/asterix-algebra/pom.xml b/asterixdb/asterix-algebra/pom.xml
index 7a32f42..af1eb92 100644
--- a/asterixdb/asterix-algebra/pom.xml
+++ b/asterixdb/asterix-algebra/pom.xml
@@ -21,7 +21,7 @@
<parent>
<artifactId>apache-asterixdb</artifactId>
<groupId>org.apache.asterix</groupId>
- <version>0.9.8-SNAPSHOT</version>
+ <version>0.9.8.2-SNAPSHOT</version>
</parent>
<artifactId>asterix-algebra</artifactId>
@@ -202,6 +202,10 @@
<artifactId>hyracks-api</artifactId>
</dependency>
<dependency>
+ <groupId>org.apache.hyracks</groupId>
+ <artifactId>hyracks-client</artifactId>
+ </dependency>
+ <dependency>
<groupId>org.apache.logging.log4j</groupId>
<artifactId>log4j-api</artifactId>
</dependency>
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/DefaultRuleSetFactory.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/DefaultRuleSetFactory.java
index ba42036..67b7910 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/DefaultRuleSetFactory.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/DefaultRuleSetFactory.java
@@ -23,23 +23,33 @@
import org.apache.asterix.common.dataflow.ICcApplicationContext;
import org.apache.asterix.optimizer.base.RuleCollections;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.common.utils.Pair;
import org.apache.hyracks.algebricks.compiler.rewriter.rulecontrollers.SequentialFirstRuleCheckFixpointRuleController;
import org.apache.hyracks.algebricks.compiler.rewriter.rulecontrollers.SequentialFixpointRuleController;
import org.apache.hyracks.algebricks.compiler.rewriter.rulecontrollers.SequentialOnceRuleController;
import org.apache.hyracks.algebricks.core.rewriter.base.AbstractRuleController;
import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.algebricks.core.rewriter.base.IRuleSetKind;
public class DefaultRuleSetFactory implements IRuleSetFactory {
@Override
public List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> getLogicalRewrites(
- ICcApplicationContext appCtx) throws AlgebricksException {
+ ICcApplicationContext appCtx) {
return buildLogical(appCtx);
}
@Override
+ public List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> getLogicalRewrites(IRuleSetKind ruleSetKind,
+ ICcApplicationContext appCtx) {
+ if (ruleSetKind == RuleSetKind.SAMPLING) {
+ return buildLogicalSampling();
+ } else {
+ throw new IllegalArgumentException(String.valueOf(ruleSetKind));
+ }
+ }
+
+ @Override
public List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> getPhysicalRewrites(
ICcApplicationContext appCtx) {
return buildPhysical(appCtx);
@@ -66,9 +76,10 @@
defaultLogicalRewrites
.add(new Pair<>(seqCtrlNoDfs, RuleCollections.buildCondPushDownAndJoinInferenceRuleCollection()));
defaultLogicalRewrites.add(new Pair<>(seqCtrlFullDfs, RuleCollections.buildLoadFieldsRuleCollection(appCtx)));
- defaultLogicalRewrites.add(new Pair<>(seqOnceCtrl, RuleCollections.buildDataExchangeRuleCollection()));
- defaultLogicalRewrites.add(new Pair<>(seqCtrlNoDfs, RuleCollections.buildConsolidationRuleCollection()));
defaultLogicalRewrites.add(new Pair<>(seqOnceCtrl, RuleCollections.buildFulltextContainsRuleCollection()));
+ defaultLogicalRewrites.add(new Pair<>(seqOnceCtrl, RuleCollections.buildDataExchangeRuleCollection()));
+ defaultLogicalRewrites.add(new Pair<>(seqOnceCtrl, RuleCollections.buildCBORuleCollection()));
+ defaultLogicalRewrites.add(new Pair<>(seqCtrlNoDfs, RuleCollections.buildConsolidationRuleCollection()));
defaultLogicalRewrites.add(new Pair<>(seqCtrlNoDfs, RuleCollections.buildAccessMethodRuleCollection()));
defaultLogicalRewrites.add(new Pair<>(seqCtrlNoDfs, RuleCollections.buildPlanCleanupRuleCollection()));
@@ -76,6 +87,14 @@
return defaultLogicalRewrites;
}
+ public static List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> buildLogicalSampling() {
+ List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> logicalRewrites = new ArrayList<>();
+ SequentialFixpointRuleController seqCtrlNoDfs = new SequentialFixpointRuleController(false);
+ logicalRewrites.add(new Pair<>(seqCtrlNoDfs, RuleCollections.buildConsolidationRuleCollection()));
+ logicalRewrites.add(new Pair<>(seqCtrlNoDfs, RuleCollections.buildPlanCleanupRuleCollection()));
+ return logicalRewrites;
+ }
+
public static List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> buildPhysical(
ICcApplicationContext appCtx) {
List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> defaultPhysicalRewrites = new ArrayList<>();
@@ -88,5 +107,4 @@
defaultPhysicalRewrites.add(new Pair<>(seqOnceCtrl, RuleCollections.prepareForJobGenRuleCollection()));
return defaultPhysicalRewrites;
}
-
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/ILangCompilationProvider.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/ILangCompilationProvider.java
index f625343..30c6623 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/ILangCompilationProvider.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/ILangCompilationProvider.java
@@ -18,6 +18,8 @@
*/
package org.apache.asterix.compiler.provider;
+import java.util.Set;
+
import org.apache.asterix.algebra.base.ILangExpressionToPlanTranslatorFactory;
import org.apache.asterix.algebra.base.ILangExtension;
import org.apache.asterix.lang.common.base.IAstPrintVisitorFactory;
@@ -54,4 +56,9 @@
* @return the rule set factory of a language implementation
*/
IRuleSetFactory getRuleSetFactory();
+
+ /**
+ * @return all configurable parameters of a language implementation.
+ */
+ Set<String> getCompilerOptions();
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/IRuleSetFactory.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/IRuleSetFactory.java
index 2300e4a..c643d21 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/IRuleSetFactory.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/IRuleSetFactory.java
@@ -21,24 +21,30 @@
import java.util.List;
import org.apache.asterix.common.dataflow.ICcApplicationContext;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.common.utils.Pair;
import org.apache.hyracks.algebricks.core.rewriter.base.AbstractRuleController;
import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.algebricks.core.rewriter.base.IRuleSetKind;
public interface IRuleSetFactory {
+ enum RuleSetKind implements IRuleSetKind {
+ SAMPLING
+ }
+
/**
* @return the logical rewrites
- * @throws AlgebricksException
*/
- public List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> getLogicalRewrites(
- ICcApplicationContext appCtx) throws AlgebricksException;
+ List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> getLogicalRewrites(ICcApplicationContext appCtx);
+
+ /**
+ * @return the logical rewrites of the specified kind
+ */
+ List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> getLogicalRewrites(IRuleSetKind ruleSetKind,
+ ICcApplicationContext appCtx);
/**
* @return the physical rewrites
*/
- public List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> getPhysicalRewrites(
- ICcApplicationContext appCtx) throws AlgebricksException;
-
+ List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> getPhysicalRewrites(ICcApplicationContext appCtx);
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/SqlppCompilationProvider.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/SqlppCompilationProvider.java
index 2c18d41..4ad888c 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/SqlppCompilationProvider.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/SqlppCompilationProvider.java
@@ -18,14 +18,27 @@
*/
package org.apache.asterix.compiler.provider;
+import java.util.HashSet;
+import java.util.Set;
+
import org.apache.asterix.algebra.base.ILangExpressionToPlanTranslatorFactory;
import org.apache.asterix.algebra.base.ILangExtension;
+import org.apache.asterix.common.config.CompilerProperties;
+import org.apache.asterix.external.feed.watch.FeedActivityDetails;
import org.apache.asterix.lang.common.base.IAstPrintVisitorFactory;
import org.apache.asterix.lang.common.base.IParserFactory;
import org.apache.asterix.lang.common.base.IRewriterFactory;
+import org.apache.asterix.lang.common.statement.StartFeedStatement;
+import org.apache.asterix.lang.common.util.FunctionUtil;
import org.apache.asterix.lang.sqlpp.parser.SqlppParserFactory;
+import org.apache.asterix.lang.sqlpp.rewrites.SqlppQueryRewriter;
import org.apache.asterix.lang.sqlpp.rewrites.SqlppRewriterFactory;
import org.apache.asterix.lang.sqlpp.visitor.SqlppAstPrintVisitorFactory;
+import org.apache.asterix.optimizer.base.FuzzyUtils;
+import org.apache.asterix.optimizer.rules.DisjunctivePredicateToJoinRule;
+import org.apache.asterix.optimizer.rules.SetAsterixPhysicalOperatorsRule;
+import org.apache.asterix.optimizer.rules.util.EquivalenceClassUtils;
+import org.apache.asterix.translator.SqlppExpressionToPlanTranslator;
import org.apache.asterix.translator.SqlppExpressionToPlanTranslatorFactory;
public class SqlppCompilationProvider implements ILangCompilationProvider {
@@ -59,4 +72,26 @@
public IRuleSetFactory getRuleSetFactory() {
return new DefaultRuleSetFactory();
}
+
+ @Override
+ public Set<String> getCompilerOptions() {
+ return new HashSet<>(Set.of(CompilerProperties.COMPILER_JOINMEMORY_KEY,
+ CompilerProperties.COMPILER_GROUPMEMORY_KEY, CompilerProperties.COMPILER_SORTMEMORY_KEY,
+ CompilerProperties.COMPILER_WINDOWMEMORY_KEY, CompilerProperties.COMPILER_TEXTSEARCHMEMORY_KEY,
+ CompilerProperties.COMPILER_PARALLELISM_KEY, CompilerProperties.COMPILER_SORT_PARALLEL_KEY,
+ CompilerProperties.COMPILER_SORT_SAMPLES_KEY, CompilerProperties.COMPILER_EXTERNALSCANMEMORY_KEY,
+ CompilerProperties.COMPILER_INDEXONLY_KEY, CompilerProperties.COMPILER_INTERNAL_SANITYCHECK_KEY,
+ CompilerProperties.COMPILER_EXTERNAL_FIELD_PUSHDOWN_KEY, CompilerProperties.COMPILER_SUBPLAN_MERGE_KEY,
+ CompilerProperties.COMPILER_SUBPLAN_NESTEDPUSHDOWN_KEY, CompilerProperties.COMPILER_ARRAYINDEX_KEY,
+ CompilerProperties.COMPILER_CBO_KEY, CompilerProperties.COMPILER_CBO_TEST_KEY,
+ CompilerProperties.COMPILER_FORCE_JOIN_ORDER_KEY, CompilerProperties.COMPILER_QUERY_PLAN_SHAPE_KEY,
+ CompilerProperties.COMPILER_MIN_MEMORY_ALLOCATION_KEY, FunctionUtil.IMPORT_PRIVATE_FUNCTIONS,
+ FuzzyUtils.SIM_FUNCTION_PROP_NAME, FuzzyUtils.SIM_THRESHOLD_PROP_NAME,
+ StartFeedStatement.WAIT_FOR_COMPLETION, FeedActivityDetails.FEED_POLICY_NAME,
+ FeedActivityDetails.COLLECT_LOCATIONS, SqlppQueryRewriter.INLINE_WITH_OPTION,
+ SqlppExpressionToPlanTranslator.REWRITE_IN_AS_OR_OPTION, "hash_merge", "output-record-type",
+ DisjunctivePredicateToJoinRule.REWRITE_OR_AS_JOIN_OPTION,
+ SetAsterixPhysicalOperatorsRule.REWRITE_ATTEMPT_BATCH_ASSIGN,
+ EquivalenceClassUtils.REWRITE_INTERNAL_QUERYUID_PK, SqlppQueryRewriter.SQL_COMPAT_OPTION));
+ }
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/AnalysisUtil.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/AnalysisUtil.java
index 8b5c8bc..b6f79d9 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/AnalysisUtil.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/AnalysisUtil.java
@@ -18,32 +18,70 @@
*/
package org.apache.asterix.optimizer.base;
+import java.io.DataInputStream;
+import java.nio.ByteBuffer;
import java.util.ArrayList;
+import java.util.Arrays;
import java.util.List;
+import org.apache.asterix.common.dataflow.ICcApplicationContext;
import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.common.transactions.TxnId;
+import org.apache.asterix.common.utils.JobUtils;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
import org.apache.asterix.metadata.declared.DataSourceId;
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.metadata.declared.ResultSetDataSink;
+import org.apache.asterix.metadata.declared.ResultSetSinkId;
+import org.apache.asterix.om.base.IAObject;
import org.apache.asterix.om.functions.BuiltinFunctions;
import org.apache.asterix.optimizer.rules.am.AccessMethodUtils;
+import org.apache.asterix.runtime.job.listener.JobEventListenerFactory;
+import org.apache.asterix.translator.ResultMetadata;
+import org.apache.asterix.translator.SessionConfig;
import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.compiler.api.ICompiler;
+import org.apache.hyracks.algebricks.compiler.api.ICompilerFactory;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractDataSourceOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.WindowOperator;
+import org.apache.hyracks.algebricks.core.algebra.plan.ALogicalPlanImpl;
+import org.apache.hyracks.algebricks.core.rewriter.base.IRuleSetKind;
+import org.apache.hyracks.api.comm.IFrame;
+import org.apache.hyracks.api.comm.VSizeFrame;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.api.result.IResultSetReader;
+import org.apache.hyracks.api.result.ResultSetId;
+import org.apache.hyracks.control.nc.resources.memory.FrameManager;
+import org.apache.hyracks.data.std.util.ByteArrayAccessibleInputStream;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
public class AnalysisUtil {
+
+ private static final List<FunctionIdentifier> fieldAccessFunctions =
+ Arrays.asList(BuiltinFunctions.GET_DATA, BuiltinFunctions.GET_HANDLE, BuiltinFunctions.TYPE_OF);
+
/*
* If the first child of op is of type opType, then it returns that child,
* o/w returns null.
@@ -204,11 +242,86 @@
return true;
}
- private static List<FunctionIdentifier> fieldAccessFunctions = new ArrayList<>();
+ public static List<List<IAObject>> runQuery(Mutable<ILogicalOperator> topOp, List<LogicalVariable> resultVars,
+ IOptimizationContext queryOptCtx, IRuleSetKind ruleSetKind) throws AlgebricksException {
- static {
- fieldAccessFunctions.add(BuiltinFunctions.GET_DATA);
- fieldAccessFunctions.add(BuiltinFunctions.GET_HANDLE);
- fieldAccessFunctions.add(BuiltinFunctions.TYPE_OF);
+ MetadataProvider metadataProvider = (MetadataProvider) queryOptCtx.getMetadataProvider();
+ ICcApplicationContext appCtx = metadataProvider.getApplicationContext();
+ TxnId mainTxnId = metadataProvider.getTxnId();
+ try {
+ TxnId newTxnId = metadataProvider.getTxnIdFactory().create();
+ metadataProvider.setTxnId(newTxnId);
+
+ IVariableTypeEnvironment topOpTypeEnv = queryOptCtx.getOutputTypeEnvironment(topOp.getValue());
+ SerializerDeserializerProvider serdeProvider = SerializerDeserializerProvider.INSTANCE;
+
+ int nFields = resultVars.size();
+ List<Mutable<ILogicalExpression>> resultExprList = new ArrayList<>(nFields);
+ List<ISerializerDeserializer<?>> resultSerdeList = new ArrayList<>(nFields);
+
+ for (LogicalVariable var : resultVars) {
+ Object varType = topOpTypeEnv.getVarType(var);
+ if (varType == null) {
+ throw new IllegalArgumentException("Cannot determine type of " + var);
+ }
+ resultSerdeList.add(serdeProvider.getSerializerDeserializer(varType));
+ resultExprList.add(new MutableObject<>(new VariableReferenceExpression(var)));
+ }
+
+ ResultMetadata resultMetadata = new ResultMetadata(SessionConfig.OutputFormat.ADM);
+ ResultSetId resultSetId = new ResultSetId(metadataProvider.getResultSetIdCounter().getAndInc());
+ ResultSetSinkId rssId = new ResultSetSinkId(resultSetId);
+ ResultSetDataSink sink = new ResultSetDataSink(rssId, null);
+
+ DistributeResultOperator resultOp = new DistributeResultOperator(resultExprList, sink, resultMetadata);
+ resultOp.getInputs().add(topOp);
+ queryOptCtx.computeAndSetTypeEnvironmentForOperator(resultOp);
+
+ MutableObject<ILogicalOperator> newResultOpRef = new MutableObject<>(resultOp);
+
+ ICompilerFactory compilerFactory = (ICompilerFactory) queryOptCtx.getCompilerFactory();
+ ICompiler compiler =
+ compilerFactory.createCompiler(new ALogicalPlanImpl(newResultOpRef), queryOptCtx, ruleSetKind);
+ compiler.optimize();
+
+ JobSpecification jobSpec = compiler.createJob(appCtx, new JobEventListenerFactory(newTxnId, false));
+
+ JobId jobId = JobUtils.runJob(appCtx.getHcc(), jobSpec, true);
+
+ IResultSetReader resultSetReader = appCtx.getResultSet().createReader(jobId, resultSetId);
+ FrameManager frameManager = new FrameManager(queryOptCtx.getPhysicalOptimizationConfig().getFrameSize());
+ IFrame frame = new VSizeFrame(frameManager);
+
+ FrameTupleAccessor fta = new FrameTupleAccessor(null);
+ ByteArrayAccessibleInputStream bais = new ByteArrayAccessibleInputStream(frame.getBuffer().array(), 0, 0);
+ DataInputStream dis = new DataInputStream(bais);
+ List<List<IAObject>> result = new ArrayList<>();
+
+ while (resultSetReader.read(frame) > 0) {
+ ByteBuffer buffer = frame.getBuffer();
+ fta.reset(buffer);
+ int nTuples = fta.getTupleCount();
+ for (int tupleIdx = 0; tupleIdx < nTuples; tupleIdx++) {
+ int tupleStart = fta.getTupleStartOffset(tupleIdx);
+ int tupleEnd = fta.getTupleEndOffset(tupleIdx);
+ bais.setContent(buffer.array(), tupleStart, tupleEnd - tupleStart);
+
+ List<IAObject> values = new ArrayList<>(nFields);
+ for (int fieldIdx = 0; fieldIdx < nFields; fieldIdx++) {
+ IAObject value = (IAObject) resultSerdeList.get(fieldIdx).deserialize(dis);
+ values.add(value);
+ }
+ result.add(values);
+ }
+ }
+
+ return result;
+ } catch (AlgebricksException e) {
+ throw e;
+ } catch (Exception e) {
+ throw new AlgebricksException(e);
+ } finally {
+ metadataProvider.setTxnId(mainTxnId);
+ }
}
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/AsterixOptimizationContext.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/AsterixOptimizationContext.java
index 51c03eb..733da2d 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/AsterixOptimizationContext.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/AsterixOptimizationContext.java
@@ -21,8 +21,11 @@
import java.util.HashSet;
import java.util.Set;
+import java.util.function.Predicate;
import org.apache.asterix.metadata.declared.DataSource;
+import org.apache.asterix.metadata.declared.FunctionDataSource;
+import org.apache.asterix.metadata.utils.MetadataConstants;
import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
import org.apache.hyracks.algebricks.core.algebra.expressions.IConflictingTypeResolver;
import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionEvalSizeComputer;
@@ -31,6 +34,7 @@
import org.apache.hyracks.algebricks.core.algebra.expressions.IMissableTypeComputer;
import org.apache.hyracks.algebricks.core.algebra.prettyprint.IPlanPrettyPrinter;
import org.apache.hyracks.algebricks.core.rewriter.base.AlgebricksOptimizationContext;
+import org.apache.hyracks.algebricks.core.rewriter.base.IOptimizationContextFactory;
import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
import org.apache.hyracks.api.exceptions.IWarningCollector;
@@ -39,17 +43,24 @@
public final class AsterixOptimizationContext extends AlgebricksOptimizationContext {
- private final Int2ObjectMap<Set<DataSource>> dataSourceMap = new Int2ObjectOpenHashMap<>();
+ private final Int2ObjectOpenHashMap<Set<DataSource>> dataSourceMap;
- public AsterixOptimizationContext(int varCounter, IExpressionEvalSizeComputer expressionEvalSizeComputer,
+ public AsterixOptimizationContext(IOptimizationContextFactory optContextFactory, int varCounter,
+ IExpressionEvalSizeComputer expressionEvalSizeComputer,
IMergeAggregationExpressionFactory mergeAggregationExpressionFactory,
IExpressionTypeComputer expressionTypeComputer, IMissableTypeComputer nullableTypeComputer,
IConflictingTypeResolver conflictingTypeResovler, PhysicalOptimizationConfig physicalOptimizationConfig,
AlgebricksPartitionConstraint clusterLocations, IPlanPrettyPrinter prettyPrinter,
IWarningCollector warningCollector) {
- super(varCounter, expressionEvalSizeComputer, mergeAggregationExpressionFactory, expressionTypeComputer,
- nullableTypeComputer, conflictingTypeResovler, physicalOptimizationConfig, clusterLocations,
- prettyPrinter, warningCollector);
+ super(optContextFactory, varCounter, expressionEvalSizeComputer, mergeAggregationExpressionFactory,
+ expressionTypeComputer, nullableTypeComputer, conflictingTypeResovler, physicalOptimizationConfig,
+ clusterLocations, prettyPrinter, warningCollector);
+ dataSourceMap = new Int2ObjectOpenHashMap<>();
+ }
+
+ public AsterixOptimizationContext(AsterixOptimizationContext from) {
+ super(from);
+ dataSourceMap = from.dataSourceMap.clone();
}
public void addDataSource(DataSource dataSource) {
@@ -65,4 +76,30 @@
public Int2ObjectMap<Set<DataSource>> getDataSourceMap() {
return dataSourceMap;
}
+
+ @Override
+ public boolean skipJobCapacityAssignment() {
+ if (dataSourceMap.isEmpty()) {
+ return false;
+ }
+ for (Int2ObjectMap.Entry<Set<DataSource>> me : dataSourceMap.int2ObjectEntrySet()) {
+ int dataSourceType = me.getIntKey();
+ if (dataSourceType != DataSource.Type.INTERNAL_DATASET && dataSourceType != DataSource.Type.FUNCTION) {
+ return false;
+ }
+ Predicate<DataSource> dataSourceTest = AsterixOptimizationContext::skipJobCapacityAssignment;
+ if (!me.getValue().stream().allMatch(dataSourceTest)) {
+ return false;
+ }
+ }
+ return true;
+ }
+
+ private static boolean skipJobCapacityAssignment(DataSource ds) {
+ return MetadataConstants.METADATA_DATAVERSE_NAME.equals(ds.getId().getDataverseName()) || isSkipping(ds);
+ }
+
+ private static boolean isSkipping(DataSource ds) {
+ return (ds instanceof FunctionDataSource) && ((FunctionDataSource) ds).skipJobCapacityAssignment();
+ }
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
index 5493dd1..1c3ef03 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
@@ -26,6 +26,7 @@
import org.apache.asterix.common.dataflow.ICcApplicationContext;
import org.apache.asterix.om.functions.BuiltinFunctions;
import org.apache.asterix.optimizer.rules.AddEquivalenceClassForRecordConstructorRule;
+import org.apache.asterix.optimizer.rules.AnnotateOperatorCostCardinalityRule;
import org.apache.asterix.optimizer.rules.AsterixConsolidateWindowOperatorsRule;
import org.apache.asterix.optimizer.rules.AsterixExtractFunctionsFromJoinConditionRule;
import org.apache.asterix.optimizer.rules.AsterixInlineVariablesRule;
@@ -43,6 +44,7 @@
import org.apache.asterix.optimizer.rules.ExtractBatchableExternalFunctionCallsRule;
import org.apache.asterix.optimizer.rules.ExtractDistinctByExpressionsRule;
import org.apache.asterix.optimizer.rules.ExtractOrderExpressionsRule;
+import org.apache.asterix.optimizer.rules.ExtractRedundantVariablesInJoinRule;
import org.apache.asterix.optimizer.rules.ExtractWindowExpressionsRule;
import org.apache.asterix.optimizer.rules.FeedScanCollectionToUnnest;
import org.apache.asterix.optimizer.rules.FilterRefineSpatialJoinRuleForSTDistanceFunction;
@@ -80,9 +82,12 @@
import org.apache.asterix.optimizer.rules.PushValueAccessToExternalDataScanRule;
import org.apache.asterix.optimizer.rules.RemoveDuplicateFieldsRule;
import org.apache.asterix.optimizer.rules.RemoveLeftOuterUnnestForLeftOuterJoinRule;
+import org.apache.asterix.optimizer.rules.RemoveOrReplaceDefaultNullCastRule;
+import org.apache.asterix.optimizer.rules.RemoveRedundantBooleanExpressionsInJoinRule;
import org.apache.asterix.optimizer.rules.RemoveRedundantListifyRule;
import org.apache.asterix.optimizer.rules.RemoveRedundantSelectRule;
import org.apache.asterix.optimizer.rules.RemoveSortInFeedIngestionRule;
+import org.apache.asterix.optimizer.rules.RemoveUnknownCheckForKnownTypeExpressionRule;
import org.apache.asterix.optimizer.rules.RemoveUnusedOneToOneEquiJoinRule;
import org.apache.asterix.optimizer.rules.RewriteDistinctAggregateRule;
import org.apache.asterix.optimizer.rules.SetAsterixMemoryRequirementsRule;
@@ -96,6 +101,8 @@
import org.apache.asterix.optimizer.rules.am.IntroduceLSMComponentFilterRule;
import org.apache.asterix.optimizer.rules.am.IntroducePrimaryIndexForAggregationRule;
import org.apache.asterix.optimizer.rules.am.IntroduceSelectAccessMethodRule;
+import org.apache.asterix.optimizer.rules.cbo.EnumerateJoinsRule;
+import org.apache.asterix.optimizer.rules.cbo.JoinEnum;
import org.apache.asterix.optimizer.rules.subplan.AsterixMoveFreeVariableOperatorOutOfSubplanRule;
import org.apache.asterix.optimizer.rules.subplan.InlineSubplanInputForNestedTupleSourceRule;
import org.apache.asterix.optimizer.rules.temporal.TranslateIntervalExpressionRule;
@@ -339,6 +346,16 @@
planCleanupRules.add(new RemoveCartesianProductWithEmptyBranchRule());
planCleanupRules.add(new InjectTypeCastForFunctionArgumentsRule());
planCleanupRules.add(new InjectTypeCastForUnionRule());
+ // (1) RemoveOrReplaceDefaultNullCastRule and (2) RemoveUnknownCheckForKnownTypesRule has to run in this order
+ // to ensure removing unknown checks, which requires the removal of null producers by (1)
+ planCleanupRules.add(new RemoveOrReplaceDefaultNullCastRule());
+ planCleanupRules.add(new RemoveUnknownCheckForKnownTypeExpressionRule());
+ // relies on RemoveOrReplaceDefaultNullCastRule AND RemoveUnknownCheckForKnownTypeExpressionRule
+ planCleanupRules.add(new RemoveRedundantSelectRule());
+ planCleanupRules.add(new RemoveRedundantBooleanExpressionsInJoinRule());
+ // RemoveRedundantBooleanExpressionsInJoinRule has to run first to probably eliminate the need for
+ // introducing an assign operator in ExtractSimilarVariablesInJoinRule
+ planCleanupRules.add(new ExtractRedundantVariablesInJoinRule());
// Needs to invoke ByNameToByIndexFieldAccessRule as the last logical optimization rule because
// some rules can push a FieldAccessByName to a place where the name it tries to access is in the closed part.
@@ -354,6 +371,15 @@
return dataExchange;
}
+ public static final List<IAlgebraicRewriteRule> buildCBORuleCollection() {
+ List<IAlgebraicRewriteRule> cbo = new LinkedList<>();
+ cbo.add(new ConsolidateSelectsRule());
+ cbo.add(new EnumerateJoinsRule(new JoinEnum()));
+ cbo.add(new ReinferAllTypesRule());
+ cbo.add(new AsterixExtractFunctionsFromJoinConditionRule());
+ return cbo;
+ }
+
public static final List<IAlgebraicRewriteRule> buildPhysicalRewritesAllLevelsRuleCollection() {
List<IAlgebraicRewriteRule> physicalRewritesAllLevels = new LinkedList<>();
physicalRewritesAllLevels.add(new PullSelectOutOfEqJoin());
@@ -375,7 +401,7 @@
physicalRewritesAllLevels.add(new InlineSingleReferenceVariablesRule());
physicalRewritesAllLevels.add(new RemoveUnusedAssignAndAggregateRule());
physicalRewritesAllLevels.add(new ConsolidateAssignsRule(true));
- // After adding projects, we may need need to set physical operators again.
+ // After adding projects, we may need to set physical operators again.
physicalRewritesAllLevels.add(new SetAsterixPhysicalOperatorsRule());
// Optimized spatial join's query plan produces more join conditions, so we need to pull out these conditions
physicalRewritesAllLevels.add(new PullSelectOutOfSpatialJoin());
@@ -420,6 +446,7 @@
prepareForJobGenRewrites.add(new SweepIllegalNonfunctionalFunctions());
prepareForJobGenRewrites.add(new FixReplicateOperatorOutputsRule());
prepareForJobGenRewrites.add(new PopulateResultMetadataRule());
+ prepareForJobGenRewrites.add(new AnnotateOperatorCostCardinalityRule());
return prepareForJobGenRewrites;
}
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/cost/Cost.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/cost/Cost.java
new file mode 100644
index 0000000..5dda277
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/cost/Cost.java
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.optimizer.cost;
+
+public class Cost implements ICost {
+
+ public static final double MAX_CARD = 1.0e200;
+ protected static final int COST_EQ = 0;
+
+ private final double cost;
+
+ public Cost() {
+ this.cost = 0.0;
+ }
+
+ public Cost(double cost) {
+ this.cost = cost;
+ }
+
+ @Override
+ public ICost zeroCost() {
+ return new Cost();
+ }
+
+ @Override
+ public ICost maxCost() {
+ return new Cost(MAX_CARD);
+ }
+
+ @Override
+ public ICost costAdd(ICost cost) {
+ return new Cost(computeTotalCost() + cost.computeTotalCost());
+ }
+
+ @Override
+ public boolean costEQ(ICost cost) {
+ return compareTo(cost) == COST_EQ;
+ }
+
+ @Override
+ public boolean costLT(ICost cost) {
+ return compareTo(cost) < COST_EQ;
+ }
+
+ @Override
+ public boolean costGT(ICost cost) {
+ return compareTo(cost) > COST_EQ;
+ }
+
+ @Override
+ public boolean costLE(ICost cost) {
+ return compareTo(cost) <= COST_EQ;
+ }
+
+ @Override
+ public boolean costGE(ICost cost) {
+ return compareTo(cost) >= COST_EQ;
+ }
+
+ @Override
+ public double computeTotalCost() {
+ return cost;
+ }
+
+ @Override
+ public int compareTo(ICost cost) {
+ return Double.compare(computeTotalCost(), cost.computeTotalCost());
+ }
+
+ @Override
+ public String toString() {
+ return Double.toString(cost);
+ }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/cost/CostMethods.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/cost/CostMethods.java
new file mode 100644
index 0000000..5dafa54
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/cost/CostMethods.java
@@ -0,0 +1,118 @@
+/*
+ * 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.optimizer.cost;
+
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.optimizer.rules.cbo.JoinNode;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
+
+public class CostMethods implements ICostMethods {
+
+ protected IOptimizationContext optCtx;
+ protected PhysicalOptimizationConfig physOptConfig;
+ protected long blockSize;
+ protected long DOP;
+ protected double maxMemorySize;
+
+ public CostMethods(IOptimizationContext context) {
+ optCtx = context;
+ physOptConfig = context.getPhysicalOptimizationConfig();
+ blockSize = getBufferCachePageSize();
+ DOP = getDOP();
+ maxMemorySize = getMaxMemorySize();
+ }
+
+ public long getBufferCacheSize() {
+ MetadataProvider metadataProvider = (MetadataProvider) optCtx.getMetadataProvider();
+ return metadataProvider.getStorageProperties().getBufferCacheSize();
+ }
+
+ public long getBufferCachePageSize() {
+ MetadataProvider metadataProvider = (MetadataProvider) optCtx.getMetadataProvider();
+ return metadataProvider.getStorageProperties().getBufferCachePageSize();
+ }
+
+ public long getDOP() {
+ return optCtx.getComputationNodeDomain().cardinality();
+ }
+
+ public double getMaxMemorySize() {
+ return physOptConfig.getMaxFramesForJoin() * physOptConfig.getFrameSize();
+ }
+
+ // These cost methods are very simple and rudimentary for now. These can be improved by asterixdb developers as needed.
+ public Cost costFullScan(JoinNode jn) {
+ return new Cost(jn.computeJoinCardinality());
+ }
+
+ public Cost costIndexScan(JoinNode jn) {
+ return new Cost(jn.computeJoinCardinality());
+ }
+
+ public Cost costHashJoin(JoinNode jn) {
+ JoinNode leftJn = jn.getLeftJn();
+ JoinNode rightJn = jn.getRightJn();
+ return new Cost(leftJn.computeJoinCardinality() + rightJn.computeJoinCardinality());
+ }
+
+ public Cost computeHJProbeExchangeCost(JoinNode jn) {
+ JoinNode leftJn = jn.getLeftJn();
+ return new Cost(leftJn.computeJoinCardinality());
+ }
+
+ public Cost computeHJBuildExchangeCost(JoinNode jn) {
+ JoinNode rightJn = jn.getRightJn();
+ return new Cost(rightJn.computeJoinCardinality());
+ }
+
+ public Cost costBroadcastHashJoin(JoinNode jn) {
+ JoinNode leftJn = jn.getLeftJn();
+ JoinNode rightJn = jn.getRightJn();
+ return new Cost(leftJn.computeJoinCardinality() + DOP * rightJn.computeJoinCardinality());
+ }
+
+ public Cost computeBHJBuildExchangeCost(JoinNode jn) {
+ JoinNode rightJn = jn.getRightJn();
+ return new Cost(DOP * rightJn.computeJoinCardinality());
+ }
+
+ public Cost costIndexNLJoin(JoinNode jn) {
+ JoinNode leftJn = jn.getLeftJn();
+ JoinNode rightJn = jn.getRightJn();
+ return new Cost(leftJn.computeJoinCardinality());
+ }
+
+ public Cost computeNLJOuterExchangeCost(JoinNode jn) {
+ JoinNode leftJn = jn.getLeftJn();
+ return new Cost(DOP * leftJn.computeJoinCardinality());
+ }
+
+ public Cost costCartesianProductJoin(JoinNode jn) {
+ JoinNode leftJn = jn.getLeftJn();
+ JoinNode rightJn = jn.getRightJn();
+ return new Cost(leftJn.computeJoinCardinality() * rightJn.computeJoinCardinality());
+ }
+
+ public Cost computeCPRightExchangeCost(JoinNode jn) {
+ JoinNode rightJn = jn.getRightJn();
+ return new Cost(DOP * rightJn.computeJoinCardinality());
+ }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/cost/ICost.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/cost/ICost.java
new file mode 100644
index 0000000..99c3362
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/cost/ICost.java
@@ -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.
+ */
+
+package org.apache.asterix.optimizer.cost;
+
+public interface ICost extends Comparable<ICost> {
+
+ ICost zeroCost();
+
+ ICost maxCost();
+
+ ICost costAdd(ICost cost);
+
+ boolean costEQ(ICost cost);
+
+ boolean costLT(ICost cost);
+
+ boolean costGT(ICost cost);
+
+ boolean costLE(ICost cost);
+
+ boolean costGE(ICost cost);
+
+ double computeTotalCost();
+
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/cost/ICostMethods.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/cost/ICostMethods.java
new file mode 100644
index 0000000..47ef617
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/cost/ICostMethods.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.optimizer.cost;
+
+import org.apache.asterix.optimizer.rules.cbo.JoinNode;
+
+public interface ICostMethods {
+ Cost costFullScan(JoinNode jn);
+
+ Cost costIndexScan(JoinNode jn);
+
+ Cost costHashJoin(JoinNode currentJn);
+
+ Cost costBroadcastHashJoin(JoinNode currentJn);
+
+ Cost costIndexNLJoin(JoinNode currentJn);
+
+ Cost costCartesianProductJoin(JoinNode currentJn);
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/AbstractConditionExpressionRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/AbstractConditionExpressionRule.java
new file mode 100644
index 0000000..2dded4a
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/AbstractConditionExpressionRule.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.optimizer.rules;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public abstract class AbstractConditionExpressionRule implements IAlgebraicRewriteRule {
+ private IOptimizationContext context;
+
+ @Override
+ public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+ throws AlgebricksException {
+ final ILogicalOperator op = opRef.getValue();
+ final Mutable<ILogicalExpression> condRef;
+ switch (op.getOperatorTag()) {
+ case SELECT:
+ final SelectOperator select = (SelectOperator) op;
+ condRef = select.getCondition();
+ break;
+ case INNERJOIN:
+ case LEFTOUTERJOIN:
+ final AbstractBinaryJoinOperator join = (AbstractBinaryJoinOperator) op;
+ condRef = join.getCondition();
+ break;
+ default:
+ return false;
+ }
+
+ this.context = context;
+
+ boolean changed = transform(condRef);
+ if (changed) {
+ context.computeAndSetTypeEnvironmentForOperator(op);
+ }
+
+ return changed;
+ }
+
+ protected final AbstractFunctionCallExpression getFunctionExpression(ILogicalExpression expression) {
+ if (expression.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+ return null;
+ }
+
+ return (AbstractFunctionCallExpression) expression;
+ }
+
+ protected final IFunctionInfo getFunctionInfo(FunctionIdentifier fid) {
+ return context.getMetadataProvider().lookupFunction(fid);
+ }
+
+ /**
+ * Transform condition expression
+ *
+ * @param condRef SELECT or join condition reference
+ * @return {@code <code>true</code>} condition has been modified
+ * {@code <code>false</code>} otherwise.
+ */
+ protected abstract boolean transform(Mutable<ILogicalExpression> condRef);
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/AnnotateOperatorCostCardinalityRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/AnnotateOperatorCostCardinalityRule.java
new file mode 100644
index 0000000..b253a1d
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/AnnotateOperatorCostCardinalityRule.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.optimizer.rules;
+
+import org.apache.asterix.optimizer.rules.cbo.EstimatedCostComputationVisitor;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/**
+ * This rule annotates the operators with their estimated cardinality and estimated cost.
+ */
+public class AnnotateOperatorCostCardinalityRule implements IAlgebraicRewriteRule {
+
+ private boolean hasApplied = false;
+
+ public AnnotateOperatorCostCardinalityRule() {
+ }
+
+ @Override
+ public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+ throws AlgebricksException {
+ if (hasApplied) {
+ return false;
+ }
+ hasApplied = true;
+ EstimatedCostComputationVisitor estCostCompVisitor = new EstimatedCostComputationVisitor();
+ opRef.getValue().accept(estCostCompVisitor, null);
+ return true;
+ }
+
+ @Override
+ public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+ throws AlgebricksException {
+ return false;
+ }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CancelUnnestWithNestedListifyRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CancelUnnestWithNestedListifyRule.java
index 069ba49..93c4ea2 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CancelUnnestWithNestedListifyRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CancelUnnestWithNestedListifyRule.java
@@ -28,6 +28,7 @@
import org.apache.commons.lang3.mutable.Mutable;
import org.apache.commons.lang3.mutable.MutableObject;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.ListSet;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalPlan;
@@ -119,6 +120,10 @@
AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
VariableUtilities.getUsedVariables(op, varSet);
if (op.hasNestedPlans()) {
+ // Variables used by the parent operators should be live at op.
+ Set<LogicalVariable> localLiveVars = new ListSet<>();
+ VariableUtilities.getLiveVariables(op, localLiveVars);
+ varSet.retainAll(localLiveVars);
AbstractOperatorWithNestedPlans aonp = (AbstractOperatorWithNestedPlans) op;
for (ILogicalPlan p : aonp.getNestedPlans()) {
for (Mutable<ILogicalOperator> r : p.getRoots()) {
@@ -194,12 +199,13 @@
AggregateOperator agg = (AggregateOperator) nestedPlanRoot;
Mutable<ILogicalOperator> aggInputOpRef = agg.getInputs().get(0);
- if (agg.getVariables().size() > 1) {
+ if (agg.getVariables().size() != 1) {
return false;
}
- if (OperatorManipulationUtil.ancestorOfOperators(agg, ImmutableSet.of(LogicalOperatorTag.LIMIT,
- LogicalOperatorTag.ORDER, LogicalOperatorTag.GROUP, LogicalOperatorTag.DISTINCT))) {
+ if (OperatorManipulationUtil.ancestorOfOperatorsExcludeCurrent(agg,
+ ImmutableSet.of(LogicalOperatorTag.LIMIT, LogicalOperatorTag.ORDER, LogicalOperatorTag.GROUP,
+ LogicalOperatorTag.DISTINCT, LogicalOperatorTag.AGGREGATE))) {
return false;
}
@@ -219,6 +225,10 @@
if (arg0.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
return false;
}
+ AbstractLogicalOperator aggChild = (AbstractLogicalOperator) agg.getInputs().get(0).getValue();
+ if (aggChild.getOperatorTag() == LogicalOperatorTag.AGGREGATE) {
+ return false;
+ }
LogicalVariable paramVar = ((VariableReferenceExpression) arg0).getVariableReference();
ArrayList<LogicalVariable> assgnVars = new ArrayList<LogicalVariable>(1);
@@ -246,6 +256,9 @@
Mutable<ILogicalOperator> bottomOpRef = aggInputOpRef;
AbstractLogicalOperator bottomOp = (AbstractLogicalOperator) bottomOpRef.getValue();
while (bottomOp.getOperatorTag() != LogicalOperatorTag.NESTEDTUPLESOURCE) {
+ if (bottomOp.getOperatorTag() == LogicalOperatorTag.AGGREGATE) {
+ return false;
+ }
bottomOpRef = bottomOp.getInputs().get(0);
bottomOp = (AbstractLogicalOperator) bottomOpRef.getValue();
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
index 10aedc7..fe000c3 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
@@ -29,7 +29,6 @@
import org.apache.asterix.common.dataflow.ICcApplicationContext;
import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.common.exceptions.ErrorCode;
-import org.apache.asterix.common.exceptions.NoOpWarningCollector;
import org.apache.asterix.common.exceptions.WarningCollector;
import org.apache.asterix.dataflow.data.common.ExpressionTypeComputer;
import org.apache.asterix.dataflow.data.nontagged.MissingWriterFactory;
@@ -90,11 +89,14 @@
import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.serializer.ResultSerializerFactoryProvider;
+import org.apache.hyracks.algebricks.runtime.writers.PrinterBasedWriterFactory;
import org.apache.hyracks.api.application.IServiceContext;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.NoOpWarningCollector;
import org.apache.hyracks.api.exceptions.Warning;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -151,8 +153,9 @@
jobGenCtx = new JobGenContext(null, metadataProvider, appCtx, SerializerDeserializerProvider.INSTANCE,
BinaryHashFunctionFactoryProvider.INSTANCE, BinaryHashFunctionFamilyProvider.INSTANCE,
BinaryComparatorFactoryProvider.INSTANCE, TypeTraitProvider.INSTANCE, BinaryBooleanInspector.FACTORY,
- BinaryIntegerInspector.FACTORY, ADMPrinterFactoryProvider.INSTANCE, MissingWriterFactory.INSTANCE,
- NullWriterFactory.INSTANCE, UnnestingPositionWriterFactory.INSTANCE, null,
+ BinaryIntegerInspector.FACTORY, ADMPrinterFactoryProvider.INSTANCE, PrinterBasedWriterFactory.INSTANCE,
+ ResultSerializerFactoryProvider.INSTANCE, MissingWriterFactory.INSTANCE, NullWriterFactory.INSTANCE,
+ UnnestingPositionWriterFactory.INSTANCE, null,
new ExpressionRuntimeProvider(new QueryLogicalExpressionJobGen(metadataProvider.getFunctionManager())),
ExpressionTypeComputer.INSTANCE, null, null, null, null, GlobalConfig.DEFAULT_FRAME_SIZE, null,
NoOpWarningCollector.INSTANCE, 0, new PhysicalOptimizationConfig());
@@ -218,10 +221,10 @@
boolean changed = constantFoldArgs(expr, arg);
List<Mutable<ILogicalExpression>> argList = expr.getArguments();
int argConstantCount = countConstantArgs(argList);
+ FunctionIdentifier fid = expr.getFunctionIdentifier();
if (argConstantCount != argList.size()) {
- if (argConstantCount > 0 && expr.getFunctionIdentifier().equals(BuiltinFunctions.OR)
- && expr.isFunctional()) {
- if (foldOrArgs(expr)) {
+ if (argConstantCount > 0 && (BuiltinFunctions.OR.equals(fid) || BuiltinFunctions.AND.equals(fid))) {
+ if (foldOrAndArgs(expr)) {
ILogicalExpression changedExpr =
expr.getArguments().size() == 1 ? expr.getArguments().get(0).getValue() : expr;
return new Pair<>(true, changedExpr);
@@ -235,7 +238,7 @@
}
try {
- if (expr.getFunctionIdentifier().equals(BuiltinFunctions.FIELD_ACCESS_BY_NAME)) {
+ if (BuiltinFunctions.FIELD_ACCESS_BY_NAME.equals(fid)) {
IAType argType = (IAType) _emptyTypeEnv.getType(expr.getArguments().get(0).getValue());
if (argType.getTypeTag() == ATypeTag.OBJECT) {
ARecordType rt = (ARecordType) argType;
@@ -247,7 +250,7 @@
}
}
}
- IAObject c = FUNC_ID_TO_CONSTANT.get(expr.getFunctionIdentifier());
+ IAObject c = FUNC_ID_TO_CONSTANT.get(fid);
if (c != null) {
ConstantExpression constantExpression = new ConstantExpression(new AsterixConstantValue(c));
constantExpression.setSourceLocation(expr.getSourceLocation());
@@ -427,7 +430,7 @@
return true;
}
- private boolean foldOrArgs(ScalarFunctionCallExpression expr) {
+ private boolean foldOrAndArgs(ScalarFunctionCallExpression expr) {
// or(true,x,y) -> true; or(false,x,y) -> or(x,y)
boolean changed = false;
List<Mutable<ILogicalExpression>> argList = expr.getArguments();
@@ -439,15 +442,21 @@
if (argExpr.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
continue;
}
+
ConstantExpression cExpr = (ConstantExpression) argExpr;
IAlgebricksConstantValue cValue = cExpr.getValue();
- if (cValue.isTrue()) {
+ FunctionIdentifier fid = expr.getFunctionIdentifier();
+
+ if (replaceAndReturn(cValue, fid)) {
// or(true,x,y) -> true;
+ // and(false, x, y) -> false
argList.clear();
argList.add(argExprRef);
return true;
- } else if (cValue.isFalse()) {
- // remove 'false' from arg list, but save the expression.
+ } else if (removeAndContinue(cValue, fid)) {
+ // or(false, x, y) -> or(x, y)
+ // and(true, x, y) -> and(x, y)
+ // remove 'false' (or 'true') from arg list, but save the expression.
argFalse = argExprRef;
argIter.remove();
changed = true;
@@ -459,6 +468,24 @@
return changed;
}
+ private boolean replaceAndReturn(IAlgebricksConstantValue cValue, FunctionIdentifier fid) {
+ if (BuiltinFunctions.OR.equals(fid)) {
+ return cValue.isTrue();
+ } else {
+ // BuiltinFunctions.AND
+ return cValue.isFalse();
+ }
+ }
+
+ private boolean removeAndContinue(IAlgebricksConstantValue cValue, FunctionIdentifier fid) {
+ if (BuiltinFunctions.OR.equals(fid)) {
+ return cValue.isFalse();
+ } else {
+ // BuiltinFunctions.AND
+ return cValue.isTrue();
+ }
+ }
+
// IEvaluatorContext
@Override
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ExtractRedundantVariablesInJoinRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ExtractRedundantVariablesInJoinRule.java
new file mode 100644
index 0000000..294dd28
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ExtractRedundantVariablesInJoinRule.java
@@ -0,0 +1,187 @@
+/*
+ * 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.optimizer.rules;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+public class ExtractRedundantVariablesInJoinRule implements IAlgebraicRewriteRule {
+ private final Map<LogicalVariable, List<Mutable<ILogicalExpression>>> variableToExpressionsMap = new HashMap<>();
+ private final Set<LogicalVariable> leftLiveVars = new HashSet<>();
+
+ @Override
+ public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+ throws AlgebricksException {
+ ILogicalOperator op = opRef.getValue();
+ if (op.getOperatorTag() != LogicalOperatorTag.INNERJOIN
+ && op.getOperatorTag() != LogicalOperatorTag.LEFTOUTERJOIN) {
+ return false;
+ }
+
+ AbstractBinaryJoinOperator joinOp = (AbstractBinaryJoinOperator) op;
+ if (!ensureAndExtractVarAndExpr(joinOp.getCondition().getValue())) {
+ return false;
+ }
+
+ setLeftLiveVariables(joinOp);
+
+ List<LogicalVariable> leftAssignVars = new ArrayList<>();
+ List<Mutable<ILogicalExpression>> leftAssignExprs = new ArrayList<>();
+
+ List<LogicalVariable> rightAssignVars = new ArrayList<>();
+ List<Mutable<ILogicalExpression>> rightAssignExprs = new ArrayList<>();
+
+ for (Map.Entry<LogicalVariable, List<Mutable<ILogicalExpression>>> kv : variableToExpressionsMap.entrySet()) {
+ LogicalVariable repeatedVariable = kv.getKey();
+ List<Mutable<ILogicalExpression>> repeatedReferences = kv.getValue();
+
+ if (leftLiveVars.contains(repeatedVariable)) {
+ reassignRepeatedVariables(context, repeatedVariable, repeatedReferences, leftAssignVars,
+ leftAssignExprs);
+ } else {
+ reassignRepeatedVariables(context, repeatedVariable, repeatedReferences, rightAssignVars,
+ rightAssignExprs);
+ }
+ }
+
+ SourceLocation sourceLocation = joinOp.getSourceLocation();
+ if (!leftAssignVars.isEmpty()) {
+ createAndSetAssign(context, sourceLocation, joinOp.getInputs().get(0), leftAssignVars, leftAssignExprs);
+ }
+
+ if (!rightAssignVars.isEmpty()) {
+ createAndSetAssign(context, sourceLocation, joinOp.getInputs().get(1), rightAssignVars, rightAssignExprs);
+ }
+
+ context.computeAndSetTypeEnvironmentForOperator(joinOp);
+
+ return true;
+ }
+
+ private void createAndSetAssign(IOptimizationContext context, SourceLocation sourceLocation,
+ Mutable<ILogicalOperator> joinInputRef, List<LogicalVariable> assignVars,
+ List<Mutable<ILogicalExpression>> assignExprs) throws AlgebricksException {
+ AssignOperator assignOp = new AssignOperator(assignVars, assignExprs);
+ assignOp.setSourceLocation(sourceLocation);
+ assignOp.getInputs().add(new MutableObject<>(joinInputRef.getValue()));
+ joinInputRef.setValue(assignOp);
+ context.computeAndSetTypeEnvironmentForOperator(assignOp);
+ }
+
+ private void setLeftLiveVariables(AbstractBinaryJoinOperator op) throws AlgebricksException {
+ ILogicalOperator leftOp = op.getInputs().get(0).getValue();
+ leftLiveVars.clear();
+ VariableUtilities.getLiveVariables(leftOp, leftLiveVars);
+ }
+
+ private void reassignRepeatedVariables(IOptimizationContext context, LogicalVariable repeatedVariable,
+ List<Mutable<ILogicalExpression>> repeatedReferences, List<LogicalVariable> assignVars,
+ List<Mutable<ILogicalExpression>> assignExprs) {
+
+ // keep one of the repeated references and reassign the others
+ for (int i = 1; i < repeatedReferences.size(); i++) {
+ Mutable<ILogicalExpression> exprRef = repeatedReferences.get(i);
+ SourceLocation sourceLocation = exprRef.getValue().getSourceLocation();
+ LogicalVariable newVar = context.newVar();
+
+ exprRef.setValue(new VariableReferenceExpression(newVar, sourceLocation));
+
+ assignVars.add(newVar);
+ assignExprs.add(new MutableObject<>(new VariableReferenceExpression(repeatedVariable, sourceLocation)));
+
+ // Prevent inlining the variable
+ context.addNotToBeInlinedVar(newVar);
+ }
+ }
+
+ private boolean ensureAndExtractVarAndExpr(ILogicalExpression expr) {
+ if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+ return false;
+ }
+
+ AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
+ if (!AlgebricksBuiltinFunctions.AND.equals(funcExpr.getFunctionIdentifier())) {
+ return false;
+ }
+
+ variableToExpressionsMap.clear();
+ boolean containsRepeatedReferences = false;
+ for (Mutable<ILogicalExpression> argRef : funcExpr.getArguments()) {
+ ILogicalExpression arg = argRef.getValue();
+ if (arg.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+ return false;
+ }
+
+ AbstractFunctionCallExpression argFuncExpr = (AbstractFunctionCallExpression) arg;
+ if (!AlgebricksBuiltinFunctions.EQ.equals(argFuncExpr.getFunctionIdentifier())) {
+ return false;
+ }
+
+ List<Mutable<ILogicalExpression>> eqArgs = argFuncExpr.getArguments();
+ Mutable<ILogicalExpression> leftRef = eqArgs.get(0);
+ Mutable<ILogicalExpression> rightRef = eqArgs.get(1);
+
+ ILogicalExpression left = leftRef.getValue();
+ ILogicalExpression right = rightRef.getValue();
+
+ LogicalVariable leftVar = VariableUtilities.getVariable(left);
+ LogicalVariable rightVar = VariableUtilities.getVariable(right);
+
+ // shouldn't be possible. But here for sanity check
+ if (leftVar == null || rightVar == null) {
+ return false;
+ }
+
+ List<Mutable<ILogicalExpression>> leftList =
+ variableToExpressionsMap.computeIfAbsent(leftVar, k -> new ArrayList<>());
+ leftList.add(leftRef);
+
+ List<Mutable<ILogicalExpression>> rightList =
+ variableToExpressionsMap.computeIfAbsent(rightVar, k -> new ArrayList<>());
+ rightList.add(rightRef);
+
+ containsRepeatedReferences |= leftList.size() > 1 || rightList.size() > 1;
+ }
+
+ // return true only if there's a repeated reference to a variable
+ return containsRepeatedReferences;
+ }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/InlineAndRemoveRedundantBooleanExpressionsRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/InlineAndRemoveRedundantBooleanExpressionsRule.java
new file mode 100644
index 0000000..7eebf1c
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/InlineAndRemoveRedundantBooleanExpressionsRule.java
@@ -0,0 +1,107 @@
+/*
+ * 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.optimizer.rules;
+
+import java.util.List;
+
+import org.apache.asterix.lang.common.util.FunctionUtil;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+
+/**
+ * Inline and remove redundant boolean expressions
+ * <p>
+ * Inline Example:
+ * and(x, and(y, and(z, w))) -> and(x, y, z, w)
+ * <p>
+ * Remove redundant example:
+ * or(x, y, y) -> or(x, y)
+ * TODO(wyk) include this rule in {@link org.apache.asterix.optimizer.base.RuleCollections}
+ */
+public class InlineAndRemoveRedundantBooleanExpressionsRule extends AbstractConditionExpressionRule {
+
+ @Override
+ protected boolean transform(Mutable<ILogicalExpression> condRef) {
+ AbstractFunctionCallExpression function = getFunctionExpression(condRef.getValue());
+ if (function == null) {
+ return false;
+ }
+
+ boolean changed = false;
+ for (Mutable<ILogicalExpression> argRef : function.getArguments()) {
+ changed |= transform(argRef);
+ }
+
+ final FunctionIdentifier fid = function.getFunctionIdentifier();
+ if (AlgebricksBuiltinFunctions.AND.equals(fid) || AlgebricksBuiltinFunctions.OR.equals(fid)) {
+ changed |= inlineCondition(function);
+ changed |= removeRedundantExpressions(function.getArguments());
+
+ //Special case: disjuncts/conjuncts have been factored out into a single (non-disjunct/conjunct) expression
+ if (function.getArguments().size() == 1) {
+ final ILogicalExpression newCond = function.getArguments().get(0).getValue();
+ condRef.setValue(newCond);
+ }
+ }
+
+ return changed;
+ }
+
+ private boolean inlineCondition(AbstractFunctionCallExpression function) {
+ final FunctionIdentifier fid = function.getFunctionIdentifier();
+ final List<Mutable<ILogicalExpression>> args = function.getArguments();
+
+ int i = 0;
+ boolean changed = false;
+ while (i < args.size()) {
+ final AbstractFunctionCallExpression argFunction = getFunctionExpression(args.get(i).getValue());
+ if (argFunction != null && fid.equals(argFunction.getFunctionIdentifier())) {
+ args.remove(i);
+ args.addAll(i, argFunction.getArguments());
+ changed = true;
+ } else {
+ i++;
+ }
+ }
+
+ return changed;
+ }
+
+ private boolean removeRedundantExpressions(List<Mutable<ILogicalExpression>> exprs) {
+ final int originalSize = exprs.size();
+ int i = 0;
+ while (i < exprs.size()) {
+ int j = i + 1;
+ while (j < exprs.size()) {
+ if (FunctionUtil.commutativeEquals(exprs.get(i).getValue(), exprs.get(j).getValue())) {
+ exprs.remove(j);
+ } else {
+ j++;
+ }
+ }
+ i++;
+ }
+
+ return exprs.size() != originalSize;
+ }
+
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
index 3fc178a..446c779 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
@@ -27,6 +27,7 @@
import java.util.Objects;
import java.util.Set;
import java.util.stream.Collectors;
+import java.util.stream.Stream;
import org.apache.asterix.algebra.operators.CommitOperator;
import org.apache.asterix.common.config.DatasetConfig.DatasetType;
@@ -190,18 +191,21 @@
metaType = (ARecordType) mp.findType(dataset.getMetaItemTypeDataverseName(), dataset.getMetaItemTypeName());
}
List<Index> indexes = mp.getDatasetIndexes(dataset.getDataverseName(), dataset.getDatasetName());
+ Stream<Index> indexStream = indexes.stream();
+ indexStream = indexStream.filter(index -> index.getIndexType() != IndexType.SAMPLE);
if (primaryIndexModificationOp.getOperation() == Kind.INSERT && !primaryIndexModificationOp.isBulkload()) {
// for insert, primary key index is handled together when primary index
- indexes = indexes.stream().filter(index -> !index.isPrimaryKeyIndex()).collect(Collectors.toList());
+ indexStream = indexStream.filter(index -> !index.isPrimaryKeyIndex());
}
-
- // Set the top operator pointer to the primary IndexInsertDeleteOperator
- ILogicalOperator currentTop = primaryIndexModificationOp;
+ indexes = indexStream.collect(Collectors.toList());
// Put an n-gram or a keyword index in the later stage of index-update,
// since TokenizeOperator needs to be involved.
Collections.sort(indexes, (o1, o2) -> o1.getIndexType().ordinal() - o2.getIndexType().ordinal());
+ // Set the top operator pointer to the primary IndexInsertDeleteOperator
+ ILogicalOperator currentTop = primaryIndexModificationOp;
+
// At this point, we have the data type info, and the indexes info as well
int secondaryIndexTotalCnt = indexes.size() - 1;
if (secondaryIndexTotalCnt > 0) {
@@ -686,7 +690,7 @@
int sourceIndicatorForBaseRecord = arrayIndexDetails.getElementList().get(0).getSourceIndicator();
LogicalVariable sourceVarForBaseRecord = hasMetaPart
? ((sourceIndicatorForBaseRecord == Index.RECORD_INDICATOR) ? recordVar : metaVar) : recordVar;
- UnnestBranchCreator branchCreator = new UnnestBranchCreator(sourceVarForBaseRecord, unnestSourceOp);
+ UnnestBranchCreator branchCreator = new UnnestBranchCreator(index, sourceVarForBaseRecord, unnestSourceOp);
Set<LogicalVariable> secondaryKeyVars = new LinkedHashSet<>();
for (Index.ArrayIndexElement workingElement : arrayIndexDetails.getElementList()) {
@@ -708,7 +712,12 @@
? getFieldAccessFunction(new MutableObject<>(varRef),
recordType.getFieldIndex(atomicFieldName.get(0)), atomicFieldName)
: getFieldAccessFunction(new MutableObject<>(varRef), -1, atomicFieldName);
-
+ IAType fieldType = recordType.getSubFieldType(atomicFieldName);
+ if (fieldType == null) {
+ newVarRef = castFunction(
+ index.isEnforced() ? BuiltinFunctions.CAST_TYPE : BuiltinFunctions.CAST_TYPE_LAX,
+ workingElement.getTypeList().get(0), newVarRef, sourceLoc);
+ }
// Add an assign on top to extract the atomic element.
AssignOperator newAssignOp = new AssignOperator(newVar, new MutableObject<>(newVarRef));
newAssignOp.setSourceLocation(sourceLoc);
@@ -724,16 +733,17 @@
workingElement.getUnnestList(), workingElement.getProjectList().get(0));
List<Boolean> firstUnnestFlags = ArrayIndexUtil.getUnnestFlags(workingElement.getUnnestList(),
workingElement.getProjectList().get(0));
- ArrayIndexUtil.walkArrayPath(index, recordType, flatFirstFieldName, firstUnnestFlags, branchCreator);
+ ArrayIndexUtil.walkArrayPath(index, workingElement, recordType, flatFirstFieldName, firstUnnestFlags,
+ branchCreator);
secondaryKeyVars.add(branchCreator.lastFieldVars.get(0));
// For all other elements in the PROJECT list, add an assign.
for (int j = 1; j < workingElement.getProjectList().size(); j++) {
LogicalVariable newVar = context.newVar();
- AbstractFunctionCallExpression newVarRef =
+ ILogicalExpression newVarRef =
getFieldAccessFunction(new MutableObject<>(branchCreator.createLastRecordVarRef()), -1,
workingElement.getProjectList().get(j));
-
+ newVarRef = createCastExpressionForArrayIndex(newVarRef, recordType, index, workingElement, j);
AssignOperator newAssignOp = new AssignOperator(newVar, new MutableObject<>(newVarRef));
newAssignOp.setSourceLocation(sourceLoc);
branchCreator.currentTop = introduceNewOp(branchCreator.currentTop, newAssignOp, true);
@@ -929,7 +939,7 @@
}
private ScalarFunctionCallExpression castFunction(FunctionIdentifier castFun, IAType requiredType,
- AbstractFunctionCallExpression inputExpr, SourceLocation sourceLoc) throws CompilationException {
+ ILogicalExpression inputExpr, SourceLocation sourceLoc) throws CompilationException {
BuiltinFunctionInfo castInfo = BuiltinFunctions.getBuiltinFunctionInfo(castFun);
ScalarFunctionCallExpression castExpr = new ScalarFunctionCallExpression(castInfo);
castExpr.setSourceLocation(sourceLoc);
@@ -953,6 +963,18 @@
return constructorExpr;
}
+ private ILogicalExpression createCastExpressionForArrayIndex(ILogicalExpression varRef, ARecordType recordType,
+ Index index, Index.ArrayIndexElement workingElement, int fieldPos) throws AlgebricksException {
+ IAType fieldType = ArrayIndexUtil.getSubFieldType(recordType, workingElement.getUnnestList(),
+ workingElement.getProjectList().get(fieldPos));
+ if (fieldType != null) {
+ return varRef;
+ } else {
+ return castFunction(index.isEnforced() ? BuiltinFunctions.CAST_TYPE : BuiltinFunctions.CAST_TYPE_LAX,
+ workingElement.getTypeList().get(fieldPos), varRef, sourceLoc);
+ }
+ }
+
private ILogicalOperator introduceNewOp(ILogicalOperator currentTopOp, ILogicalOperator newOp, boolean afterOp)
throws AlgebricksException {
if (afterOp) {
@@ -1088,8 +1110,10 @@
private final List<LogicalVariable> lastFieldVars;
private LogicalVariable lastRecordVar;
private ILogicalOperator currentTop, currentBottom = null;
+ private final Index index;
- public UnnestBranchCreator(LogicalVariable recordVar, ILogicalOperator sourceOperator) {
+ public UnnestBranchCreator(Index index, LogicalVariable recordVar, ILogicalOperator sourceOperator) {
+ this.index = index;
this.lastRecordVar = recordVar;
this.currentTop = sourceOperator;
this.lastFieldVars = new ArrayList<>();
@@ -1187,19 +1211,23 @@
}
@Override
- public void executeActionOnFinalArrayStep(ARecordType startingStepRecordType, List<String> fieldName,
- boolean isNonArrayStep, boolean requiresOnlyOneUnnest) throws AlgebricksException {
+ public void executeActionOnFinalArrayStep(Index.ArrayIndexElement workingElement, ARecordType baseRecordType,
+ ARecordType startingStepRecordType, List<String> fieldName, boolean isNonArrayStep,
+ boolean requiresOnlyOneUnnest) throws AlgebricksException {
// If the final value is nested inside a record, add an additional ASSIGN.
+ ILogicalExpression accessToFinalVar;
if (!isNonArrayStep) {
- return;
+ accessToFinalVar = createCastExpressionForArrayIndex(createLastRecordVarRef(), baseRecordType, index,
+ workingElement, 0);
+ } else {
+ // Create the function to access our final field.
+ accessToFinalVar = (startingStepRecordType != null)
+ ? getFieldAccessFunction(new MutableObject<>(createLastRecordVarRef()),
+ startingStepRecordType.getFieldIndex(fieldName.get(0)), fieldName)
+ : getFieldAccessFunction(new MutableObject<>(createLastRecordVarRef()), -1, fieldName);
+ accessToFinalVar =
+ createCastExpressionForArrayIndex(accessToFinalVar, baseRecordType, index, workingElement, 0);
}
-
- // Create the function to access our final field.
- AbstractFunctionCallExpression accessToFinalVar = (startingStepRecordType != null)
- ? getFieldAccessFunction(new MutableObject<>(createLastRecordVarRef()),
- startingStepRecordType.getFieldIndex(fieldName.get(0)), fieldName)
- : getFieldAccessFunction(new MutableObject<>(createLastRecordVarRef()), -1, fieldName);
-
LogicalVariable finalVar = context.newVar();
this.lastFieldVars.add(finalVar);
AssignOperator assignOperator = new AssignOperator(finalVar, new MutableObject<>(accessToFinalVar));
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/LoadRecordFieldsRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/LoadRecordFieldsRule.java
index b9d512b..42cce52 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/LoadRecordFieldsRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/LoadRecordFieldsRule.java
@@ -27,6 +27,7 @@
import org.apache.asterix.algebra.base.OperatorAnnotation;
import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.lang.common.util.FunctionUtil;
import org.apache.asterix.om.base.AInt32;
import org.apache.asterix.om.base.AString;
import org.apache.asterix.om.constants.AsterixConstantValue;
@@ -64,7 +65,7 @@
public class LoadRecordFieldsRule implements IAlgebraicRewriteRule {
- private ExtractFieldLoadExpressionVisitor exprVisitor = new ExtractFieldLoadExpressionVisitor();
+ private final ExtractFieldLoadExpressionVisitor exprVisitor = new ExtractFieldLoadExpressionVisitor();
@Override
public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
@@ -98,13 +99,13 @@
// checking if we can annotate a Selection as using just one field
// access
SelectOperator sigma = (SelectOperator) op1;
- LinkedList<LogicalVariable> vars = new LinkedList<LogicalVariable>();
+ List<LogicalVariable> vars = new ArrayList<>();
VariableUtilities.getUsedVariables(sigma, vars);
if (vars.size() == 1) {
// we can annotate Selection
AssignOperator assign1 = (AssignOperator) op1.getInputs().get(0).getValue();
- AbstractLogicalExpression expr1 = (AbstractLogicalExpression) getFirstExpr(assign1);
- if (expr1.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+ ILogicalExpression expr1 = getFirstExpr(assign1);
+ if (FunctionUtil.isFieldAccessFunction(expr1)) {
AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) expr1;
// f should be a call to a field/data access kind of
// function
@@ -141,7 +142,7 @@
}
// create an assign
LogicalVariable v = context.newVar();
- AssignOperator a2 = new AssignOperator(v, new MutableObject<ILogicalExpression>(f));
+ AssignOperator a2 = new AssignOperator(v, new MutableObject<>(f));
a2.setSourceLocation(expr.getSourceLocation());
pushFieldAssign(a2, topOp, context);
context.computeAndSetTypeEnvironmentForOperator(a2);
@@ -151,7 +152,7 @@
LogicalVariable var = ref.getVariableReference();
List<LogicalVariable> keys = context.findPrimaryKey(var);
if (keys != null) {
- List<LogicalVariable> tail = new ArrayList<LogicalVariable>();
+ List<LogicalVariable> tail = new ArrayList<>();
tail.add(v);
FunctionalDependency pk = new FunctionalDependency(keys, tail);
context.addPrimaryKey(pk);
@@ -408,5 +409,4 @@
private static ILogicalExpression getFirstExpr(AssignOperator assign) {
return assign.getExpressions().get(0).getValue();
}
-
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/MetaFunctionToMetaVariableRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/MetaFunctionToMetaVariableRule.java
index aa2b8fd..036c456 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/MetaFunctionToMetaVariableRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/MetaFunctionToMetaVariableRule.java
@@ -95,7 +95,8 @@
if (dataSource.getDatasourceType() != DataSource.Type.EXTERNAL_DATASET
&& dataSource.getDatasourceType() != DataSource.Type.INTERNAL_DATASET
&& dataSource.getDatasourceType() != DataSource.Type.LOADABLE
- && dataSource.getDatasourceType() != DataSource.Type.FUNCTION) {
+ && dataSource.getDatasourceType() != DataSource.Type.FUNCTION
+ && dataSource.getDatasourceType() != DataSource.Type.SAMPLE) {
IMutationDataSource mds = (IMutationDataSource) dataSource;
if (mds.isChange()) {
transformers = new ArrayList<>();
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushAggregateIntoNestedSubplanRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushAggregateIntoNestedSubplanRule.java
index d2ac8e5..378c758 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushAggregateIntoNestedSubplanRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushAggregateIntoNestedSubplanRule.java
@@ -398,14 +398,16 @@
if (!pushableNestedSubplan) {
return false;
}
-
+ Set<ILogicalOperator> visited = new HashSet<>();
for (int i = 0; i < nspOp.getNestedPlans().size(); i++) {
Mutable<ILogicalOperator> nspAggRef = nspOp.getNestedPlans().get(i).getRoots().get(0);
AggregateOperator nspAgg = (AggregateOperator) nspAggRef.getValue();
Mutable<ILogicalOperator> nspAggChildRef = nspAgg.getInputs().get(0);
LogicalVariable listifyVar = findListifiedVariable(nspAgg, varFromNestedAgg);
if (listifyVar != null) {
- OperatorManipulationUtil.substituteVarRec(aggInSubplanOp, unnestVar, listifyVar, true, context);
+ OperatorManipulationUtil.substituteVarRec(aggInSubplanOp, unnestVar, listifyVar, true, context,
+ visited);
+ visited.clear();
nspAgg.getVariables().addAll(aggInSubplanOp.getVariables());
nspAgg.getExpressions().addAll(aggInSubplanOp.getExpressions());
for (LogicalVariable v : aggInSubplanOp.getVariables()) {
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 6d92b51..28166a9 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
@@ -120,12 +120,14 @@
AbstractBinaryJoinOperator join = (AbstractBinaryJoinOperator) opRefJoin.getValue();
gby.getDecorList().clear();
gby.getDecorList().addAll(decorToPush);
+ Set<ILogicalOperator> visited = new HashSet<>();
for (Pair<LogicalVariable, Mutable<ILogicalExpression>> p : decorNotToPush) {
LogicalVariable v1 = p.first;
if (v1 != null) {
VariableReferenceExpression varRef = (VariableReferenceExpression) p.second.getValue();
LogicalVariable v2 = varRef.getVariableReference();
- OperatorManipulationUtil.substituteVarRec(join, v2, v1, true, context);
+ OperatorManipulationUtil.substituteVarRec(join, v2, v1, true, context, visited);
+ visited.clear();
}
}
Mutable<ILogicalOperator> branchRef = join.getInputs().get(branch);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveOrReplaceDefaultNullCastRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveOrReplaceDefaultNullCastRule.java
new file mode 100644
index 0000000..5791bd7
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveOrReplaceDefaultNullCastRule.java
@@ -0,0 +1,180 @@
+/*
+ * 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.optimizer.rules;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/**
+ * This rule removes unnecessary default-null casts functions for known types
+ * For example:
+ * <p>
+ * Before:
+ * assign [$$uid] <- [uuid-default-null(uuid())]
+ * After:
+ * assign [$$uid] <- [uuid()]
+ * <p>
+ * Before:
+ * assign [$$uid] <- [string-default-null(uuid())]
+ * After:
+ * assign [$$uid] <- [string(uuid())]
+ * <p>
+ * It is known that uuid() will not produce a null value. Hence, casting it using uuid-default-null() or
+ * string-default-null() are useless
+ */
+public class RemoveOrReplaceDefaultNullCastRule implements IAlgebraicRewriteRule {
+ private static final Map<FunctionIdentifier, FunctionIdentifier> CAST_MAP;
+
+ static {
+ CAST_MAP = new HashMap<>();
+ CAST_MAP.put(BuiltinFunctions.BOOLEAN_DEFAULT_NULL_CONSTRUCTOR, BuiltinFunctions.BOOLEAN_CONSTRUCTOR);
+
+ CAST_MAP.put(BuiltinFunctions.INT8_DEFAULT_NULL_CONSTRUCTOR, BuiltinFunctions.INT8_CONSTRUCTOR);
+ CAST_MAP.put(BuiltinFunctions.INT16_DEFAULT_NULL_CONSTRUCTOR, BuiltinFunctions.INT16_CONSTRUCTOR);
+ CAST_MAP.put(BuiltinFunctions.INT32_DEFAULT_NULL_CONSTRUCTOR, BuiltinFunctions.INT32_CONSTRUCTOR);
+ CAST_MAP.put(BuiltinFunctions.INT64_DEFAULT_NULL_CONSTRUCTOR, BuiltinFunctions.INT64_CONSTRUCTOR);
+
+ CAST_MAP.put(BuiltinFunctions.FLOAT_DEFAULT_NULL_CONSTRUCTOR, BuiltinFunctions.FLOAT_CONSTRUCTOR);
+ CAST_MAP.put(BuiltinFunctions.DOUBLE_DEFAULT_NULL_CONSTRUCTOR, BuiltinFunctions.DOUBLE_CONSTRUCTOR);
+
+ // *_DEFAULT_NULL_WITH_FORMAT_CONSTRUCTOR are not considered here as format may differ from the original value
+ CAST_MAP.put(BuiltinFunctions.DATE_DEFAULT_NULL_CONSTRUCTOR, BuiltinFunctions.DATE_CONSTRUCTOR);
+ CAST_MAP.put(BuiltinFunctions.TIME_DEFAULT_NULL_CONSTRUCTOR, BuiltinFunctions.TIME_CONSTRUCTOR);
+ CAST_MAP.put(BuiltinFunctions.DATETIME_DEFAULT_NULL_CONSTRUCTOR, BuiltinFunctions.DATETIME_CONSTRUCTOR);
+
+ CAST_MAP.put(BuiltinFunctions.DURATION_DEFAULT_NULL_CONSTRUCTOR, BuiltinFunctions.DURATION_CONSTRUCTOR);
+ CAST_MAP.put(BuiltinFunctions.DAY_TIME_DURATION_DEFAULT_NULL_CONSTRUCTOR,
+ BuiltinFunctions.DAY_TIME_DURATION_CONSTRUCTOR);
+ CAST_MAP.put(BuiltinFunctions.YEAR_MONTH_DURATION_DEFAULT_NULL_CONSTRUCTOR,
+ BuiltinFunctions.YEAR_MONTH_DURATION_CONSTRUCTOR);
+
+ CAST_MAP.put(BuiltinFunctions.STRING_DEFAULT_NULL_CONSTRUCTOR, BuiltinFunctions.STRING_CONSTRUCTOR);
+
+ CAST_MAP.put(BuiltinFunctions.BINARY_BASE64_DEFAULT_NULL_CONSTRUCTOR,
+ BuiltinFunctions.BINARY_BASE64_CONSTRUCTOR);
+
+ CAST_MAP.put(BuiltinFunctions.UUID_DEFAULT_NULL_CONSTRUCTOR, BuiltinFunctions.UUID_CONSTRUCTOR);
+ }
+
+ @Override
+ public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+ throws AlgebricksException {
+ ILogicalOperator op = opRef.getValue();
+
+ if (op.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+ //process assign
+ AssignOperator assignOp = (AssignOperator) op;
+ return processExpressions(context, assignOp, assignOp.getExpressions());
+ } else if (op.getOperatorTag() == LogicalOperatorTag.SELECT) {
+ //process select
+ SelectOperator selectOp = (SelectOperator) op;
+ return processExpression(context, selectOp, selectOp.getCondition());
+ }
+ return false;
+ }
+
+ private boolean processExpressions(IOptimizationContext context, ILogicalOperator op,
+ List<Mutable<ILogicalExpression>> expressions) throws AlgebricksException {
+ boolean changed = false;
+ for (Mutable<ILogicalExpression> exprRef : expressions) {
+ changed |= processExpression(context, op, exprRef);
+ }
+ return changed;
+ }
+
+ private boolean processExpression(IOptimizationContext context, ILogicalOperator op,
+ Mutable<ILogicalExpression> exprRef) throws AlgebricksException {
+ ILogicalExpression expr = exprRef.getValue();
+ if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+ return false;
+ }
+
+ AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
+ FunctionIdentifier fid = funcExpr.getFunctionIdentifier();
+
+ // First, process arguments to handle nested casts
+ boolean changed = processExpressions(context, op, funcExpr.getArguments());
+ if (!CAST_MAP.containsKey(fid)) {
+ return changed;
+ }
+
+ ILogicalExpression castArgExpr = funcExpr.getArguments().get(0).getValue();
+ IVariableTypeEnvironment env = context.getOutputTypeEnvironment(op);
+
+ IAType outputType = ((AUnionType) env.getType(funcExpr)).getActualType();
+ IAType argType = (IAType) env.getType(castArgExpr);
+
+ //If arg type is a union type (or unknowable), then the function removed as below
+ if (isDerivedOrAny(argType) || !outputType.equals(argType) && !isConvertableType(fid, outputType, argType)) {
+ // The types of cast and its argument are different
+ // Also, the cast function isn't a string function
+ return changed;
+ }
+
+ if (outputType.equals(argType)) {
+ exprRef.setValue(castArgExpr);
+ } else {
+ MetadataProvider metadataProvider = (MetadataProvider) context.getMetadataProvider();
+ IFunctionInfo functionInfo = metadataProvider.lookupFunction(CAST_MAP.get(fid));
+ funcExpr.setFunctionInfo(functionInfo);
+ context.computeAndSetTypeEnvironmentForOperator(op);
+ }
+ return true;
+ }
+
+ private boolean isDerivedOrAny(IAType argType) {
+ ATypeTag argTypeTag = argType.getTypeTag();
+ return argTypeTag.isDerivedType() || argTypeTag == ATypeTag.ANY;
+ }
+
+ private boolean isConvertableType(FunctionIdentifier fid, IAType outputType, IAType argType) {
+ ATypeTag outputTypeTag = outputType.getTypeTag();
+ ATypeTag argTypeTag = argType.getTypeTag();
+
+ boolean convertableNumeric = ATypeHierarchy.getTypeDomain(outputTypeTag) == ATypeHierarchy.Domain.NUMERIC
+ && ATypeHierarchy.getTypeDomain(argTypeTag) == ATypeHierarchy.Domain.NUMERIC
+ && (ATypeHierarchy.canPromote(argTypeTag, outputTypeTag)
+ || ATypeHierarchy.canDemote(argTypeTag, outputTypeTag));
+
+ // converting to string is suitable for all non-derived types
+ return BuiltinFunctions.STRING_DEFAULT_NULL_CONSTRUCTOR.equals(fid) || convertableNumeric;
+ }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveRedundantBooleanExpressionsInJoinRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveRedundantBooleanExpressionsInJoinRule.java
new file mode 100644
index 0000000..f77fc2c
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveRedundantBooleanExpressionsInJoinRule.java
@@ -0,0 +1,155 @@
+/*
+ * 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.optimizer.rules;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.EquivalenceClass;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.FDsAndEquivClassesVisitor;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+
+public class RemoveRedundantBooleanExpressionsInJoinRule extends InlineAndRemoveRedundantBooleanExpressionsRule {
+ private final FDsAndEquivClassesVisitor visitor = new FDsAndEquivClassesVisitor();
+ private final Map<LogicalVariable, LogicalVariable> normalizedVariables = new HashMap<>();
+
+ @Override
+ public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+ throws AlgebricksException {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+ throws AlgebricksException {
+ ILogicalOperator op = opRef.getValue();
+ LogicalOperatorTag opTag = op.getOperatorTag();
+
+ if (context.checkIfInDontApplySet(this, op)) {
+ return false;
+ }
+
+ if (opTag != LogicalOperatorTag.INNERJOIN && opTag != LogicalOperatorTag.LEFTOUTERJOIN) {
+ // TODO FDsAndEquivClassesVisitor alters the distinct variables? We have seen bugs with distinct
+ // not sure if that related
+ if (op.getOperatorTag() != LogicalOperatorTag.DISTINCT) {
+ // Compute the equivalent classes for op
+ op.accept(visitor, context);
+ }
+ context.addToDontApplySet(this, op);
+ return false;
+ }
+
+ boolean changed = normalize(context, op);
+ // compute equivalent classes for the join op
+ op.accept(visitor, context);
+ context.addToDontApplySet(this, op);
+ return changed;
+ }
+
+ private boolean normalize(IOptimizationContext context, ILogicalOperator op) {
+ AbstractBinaryJoinOperator joinOp = (AbstractBinaryJoinOperator) op;
+ ILogicalOperator leftOp = joinOp.getInputs().get(0).getValue();
+ ILogicalOperator rightOp = joinOp.getInputs().get(1).getValue();
+
+ Map<LogicalVariable, EquivalenceClass> leftEqMap = context.getEquivalenceClassMap(leftOp);
+ Map<LogicalVariable, EquivalenceClass> rightEqMap = context.getEquivalenceClassMap(rightOp);
+
+ normalizedVariables.clear();
+
+ Mutable<ILogicalExpression> joinCondRef = joinOp.getCondition();
+ Mutable<ILogicalExpression> clonedCondition = new MutableObject<>(joinCondRef.getValue().cloneExpression());
+
+ if (normalizeVariables(leftEqMap, rightEqMap, clonedCondition) && transform(clonedCondition)) {
+ // replace the join condition iff the normalization led to a minimized circuit
+ joinCondRef.setValue(clonedCondition.getValue());
+ return true;
+ }
+
+ return false;
+ }
+
+ private boolean normalizeVariables(Map<LogicalVariable, EquivalenceClass> leftEqMap,
+ Map<LogicalVariable, EquivalenceClass> rightEqMap, Mutable<ILogicalExpression> exprRef) {
+ ILogicalExpression expr = exprRef.getValue();
+ if (expr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+ return processFunction(leftEqMap, rightEqMap, (AbstractFunctionCallExpression) expr);
+ } else if (expr.getExpressionTag() == LogicalExpressionTag.CONSTANT) {
+ // TODO is this possible in joins?
+ return false;
+ }
+
+ LogicalVariable toNormalizeVariable = VariableUtilities.getVariable(expr);
+ LogicalVariable normalized =
+ getNormalizedVariableAndSetEquivalentsIfAny(leftEqMap, rightEqMap, toNormalizeVariable);
+
+ if (normalized == toNormalizeVariable) {
+ // both are the same, do nothing
+ return false;
+ }
+
+ // we need to replace the variable expression using the normalized expression
+ exprRef.setValue(new VariableReferenceExpression(normalized));
+ return true;
+ }
+
+ private LogicalVariable getNormalizedVariableAndSetEquivalentsIfAny(
+ Map<LogicalVariable, EquivalenceClass> leftEqMap, Map<LogicalVariable, EquivalenceClass> rightEqMap,
+ LogicalVariable toNormalizeVariable) {
+ if (normalizedVariables.containsKey(toNormalizeVariable)) {
+ // get the normalized variable
+ return normalizedVariables.get(toNormalizeVariable);
+ } else if (leftEqMap != null && leftEqMap.containsKey(toNormalizeVariable)) {
+ setNormalizedVariables(toNormalizeVariable, leftEqMap.get(toNormalizeVariable));
+ } else if (rightEqMap != null && rightEqMap.containsKey(toNormalizeVariable)) {
+ setNormalizedVariables(toNormalizeVariable, rightEqMap.get(toNormalizeVariable));
+ }
+
+ return toNormalizeVariable;
+ }
+
+ private void setNormalizedVariables(LogicalVariable toNormalizeVariable, EquivalenceClass equivalenceClass) {
+ for (LogicalVariable eqVar : equivalenceClass.getMembers()) {
+ normalizedVariables.put(eqVar, toNormalizeVariable);
+ }
+ }
+
+ private boolean processFunction(Map<LogicalVariable, EquivalenceClass> leftEqMap,
+ Map<LogicalVariable, EquivalenceClass> rightEqMap, AbstractFunctionCallExpression funcExpr) {
+
+ boolean changed = false;
+ for (Mutable<ILogicalExpression> argRef : funcExpr.getArguments()) {
+ changed |= normalizeVariables(leftEqMap, rightEqMap, argRef);
+ }
+
+ return changed;
+ }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveRedundantListifyRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveRedundantListifyRule.java
index f968b35..0f490be 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveRedundantListifyRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveRedundantListifyRule.java
@@ -200,7 +200,7 @@
return false;
}
AggregateOperator agg = (AggregateOperator) r;
- if (agg.getVariables().size() > 1) {
+ if (agg.getVariables().size() != 1) {
return false;
}
LogicalVariable aggVar = agg.getVariables().get(0);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveUnknownCheckForKnownTypeExpressionRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveUnknownCheckForKnownTypeExpressionRule.java
new file mode 100644
index 0000000..93cf6c5
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveUnknownCheckForKnownTypeExpressionRule.java
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.optimizer.rules;
+
+import java.util.List;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/**
+ * Removes unnecessary unknown checks (e.g., not(is-unknown(expr))) for known types
+ * For example:
+ * <p>
+ * Before:
+ * select (not(is-unknown(uid))
+ * * assign [$$uid] <- [uuid()]
+ * After:
+ * select (true) <-- will be removed by another rule later
+ * * assign [$$uid] <- [uuid()]
+ */
+public class RemoveUnknownCheckForKnownTypeExpressionRule implements IAlgebraicRewriteRule {
+ @Override
+ public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+ throws AlgebricksException {
+ ILogicalOperator op = opRef.getValue();
+ if (op.getOperatorTag() != LogicalOperatorTag.SELECT) {
+ return false;
+ }
+
+ SelectOperator selectOp = (SelectOperator) op;
+ return processExpression(context, selectOp, selectOp.getCondition());
+ }
+
+ private boolean processExpressions(IOptimizationContext context, ILogicalOperator op,
+ List<Mutable<ILogicalExpression>> expressions) throws AlgebricksException {
+ boolean changed = false;
+ for (Mutable<ILogicalExpression> exprRef : expressions) {
+ changed |= processExpression(context, op, exprRef);
+ }
+ return changed;
+ }
+
+ private boolean processExpression(IOptimizationContext context, ILogicalOperator op,
+ Mutable<ILogicalExpression> exprRef) throws AlgebricksException {
+
+ AbstractFunctionCallExpression notFuncExpr = getFunctionExpression(exprRef);
+ if (notFuncExpr == null) {
+ return false;
+ }
+ FunctionIdentifier fid = notFuncExpr.getFunctionIdentifier();
+ if (!BuiltinFunctions.NOT.equals(fid)) {
+ return processExpressions(context, op, notFuncExpr.getArguments());
+ }
+
+ AbstractFunctionCallExpression unknownCheckFuncExpr = getFunctionExpression(notFuncExpr.getArguments().get(0));
+ if (unknownCheckFuncExpr == null || !isNullOrIsMissingOrIsUnknownCheck(unknownCheckFuncExpr)) {
+ return false;
+ }
+
+ ILogicalExpression unknownCheckArg = unknownCheckFuncExpr.getArguments().get(0).getValue();
+ IVariableTypeEnvironment env = op.computeInputTypeEnvironment(context);
+
+ IAType type = (IAType) env.getType(unknownCheckArg);
+ ATypeTag typeTag = type.getTypeTag();
+ if (typeTag == ATypeTag.ANY || typeTag == ATypeTag.UNION && ((AUnionType) type).isUnknownableType()) {
+ // Stop if it is ANY, or it is actually an unknown-able type
+ return false;
+ }
+
+ // Set the expression to true and allow the constant folding to remove the SELECT if possible
+ exprRef.setValue(ConstantExpression.TRUE);
+ return true;
+ }
+
+ private boolean isNullOrIsMissingOrIsUnknownCheck(AbstractFunctionCallExpression funcExpr) {
+ FunctionIdentifier fid = funcExpr.getFunctionIdentifier();
+ return BuiltinFunctions.IS_NULL.equals(fid) || BuiltinFunctions.IS_MISSING.equals(fid)
+ || BuiltinFunctions.IS_UNKNOWN.equals(fid);
+ }
+
+ private AbstractFunctionCallExpression getFunctionExpression(Mutable<ILogicalExpression> exprRef) {
+ ILogicalExpression expr = exprRef.getValue();
+
+ if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+ return null;
+ }
+
+ return (AbstractFunctionCallExpression) expr;
+ }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java
index 61361f6..265791e 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java
@@ -24,6 +24,7 @@
import java.util.List;
import java.util.Set;
+import org.apache.asterix.common.config.DatasetConfig;
import org.apache.asterix.metadata.declared.DataSource;
import org.apache.asterix.metadata.declared.DatasetDataSource;
import org.apache.asterix.metadata.entities.Dataset;
@@ -165,10 +166,14 @@
// Check that all datascans scan the same dataset, and that the join condition
// only used primary key variables of those datascans.
for (int i = 0; i < dataScans.size(); i++) {
+ DatasetDataSource currentDataSource = (DatasetDataSource) dataScans.get(i).getDataSource();
+ if (currentDataSource.getDataset().getDatasetType() == DatasetConfig.DatasetType.EXTERNAL) {
+ // The PK condition is not satisfied when external datasets are involved (no PKs)
+ return -1;
+ }
if (i > 0) {
- DatasetDataSource prevAqlDataSource = (DatasetDataSource) dataScans.get(i - 1).getDataSource();
- DatasetDataSource currAqlDataSource = (DatasetDataSource) dataScans.get(i).getDataSource();
- if (!prevAqlDataSource.getDataset().equals(currAqlDataSource.getDataset())) {
+ DatasetDataSource previousDataSource = (DatasetDataSource) dataScans.get(i - 1).getDataSource();
+ if (!previousDataSource.getDataset().equals(currentDataSource.getDataset())) {
return -1;
}
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetAsterixMemoryRequirementsRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetAsterixMemoryRequirementsRule.java
index e9de353..9467555 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetAsterixMemoryRequirementsRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetAsterixMemoryRequirementsRule.java
@@ -63,9 +63,6 @@
int dataSourceType = me.getIntKey();
Predicate<DataSource> dataSourceTest;
switch (dataSourceType) {
- case DataSource.Type.INTERNAL_DATASET:
- dataSourceTest = SetAsterixMemoryRequirementsRule::isMinMemoryBudgetDataset;
- break;
case DataSource.Type.FUNCTION:
dataSourceTest = SetAsterixMemoryRequirementsRule::isMinMemoryBudgetFunction;
break;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
index 7f1ff4a..52f0279 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
@@ -120,11 +120,10 @@
* @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.
- * This way, an index-nested-loop-join transformation can be conducted properly since the transformation
- * process skips an optimzable function expression if the field type of one of its variable is unknown.
+ * @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.
+ * This way, an index-nested-loop-join transformation can be conducted properly since the transformation
+ * process skips an optimzable function expression if the field type of one of its variable is unknown.
* @throws AlgebricksException
*/
protected void fillSubTreeIndexExprs(OptimizableOperatorSubTree subTree,
@@ -174,13 +173,14 @@
}
protected void pruneIndexCandidates(Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs,
- IOptimizationContext context, IVariableTypeEnvironment typeEnvironment) throws AlgebricksException {
+ IOptimizationContext context, IVariableTypeEnvironment typeEnvironment, boolean checkApplicableOnly)
+ throws AlgebricksException {
Iterator<Map.Entry<IAccessMethod, AccessMethodAnalysisContext>> amIt = analyzedAMs.entrySet().iterator();
// Check applicability of indexes by access method type.
while (amIt.hasNext()) {
Map.Entry<IAccessMethod, AccessMethodAnalysisContext> entry = amIt.next();
AccessMethodAnalysisContext amCtx = entry.getValue();
- pruneIndexCandidates(entry.getKey(), amCtx, context, typeEnvironment);
+ pruneIndexCandidates(entry.getKey(), amCtx, context, typeEnvironment, checkApplicableOnly);
// Remove access methods for which there are definitely no
// applicable indexes.
if (amCtx.isIndexExprsAndVarsEmpty()) {
@@ -194,7 +194,8 @@
* process by making it more systematic.
*/
protected Pair<IAccessMethod, Index> chooseBestIndex(Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs) {
- List<Pair<IAccessMethod, Index>> list = chooseAllIndexes(analyzedAMs);
+ List<Pair<IAccessMethod, Index>> list = new ArrayList<>();
+ chooseAllIndexes(analyzedAMs, list);
return list.isEmpty() ? null : list.get(0);
}
@@ -206,9 +207,8 @@
* [InvertedIndexAccessMethod, IndexType.SINGLE_PARTITION_WORD_INVIX || SINGLE_PARTITION_NGRAM_INVIX ||
* LENGTH_PARTITIONED_WORD_INVIX || LENGTH_PARTITIONED_NGRAM_INVIX]
*/
- protected List<Pair<IAccessMethod, Index>> chooseAllIndexes(
- Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs) {
- List<Pair<IAccessMethod, Index>> result = new ArrayList<>();
+ protected void chooseAllIndexes(Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs,
+ List<Pair<IAccessMethod, Index>> result) {
// Use variables (fields) to the index types map to check which type of indexes are applied for the vars.
Map<List<Pair<Integer, Integer>>, List<IndexType>> resultVarsToIndexTypesMap = new HashMap<>();
Iterator<Map.Entry<IAccessMethod, AccessMethodAnalysisContext>> amIt = analyzedAMs.entrySet().iterator();
@@ -265,7 +265,6 @@
}
}
}
- return result;
}
private boolean isSameFullTextConfigInIndexAndQuery(AccessMethodAnalysisContext analysisCtx,
@@ -305,7 +304,8 @@
* @throws AlgebricksException
*/
public void pruneIndexCandidates(IAccessMethod accessMethod, AccessMethodAnalysisContext analysisCtx,
- IOptimizationContext context, IVariableTypeEnvironment typeEnvironment) throws AlgebricksException {
+ IOptimizationContext context, IVariableTypeEnvironment typeEnvironment, boolean checkApplicableOnly)
+ throws AlgebricksException {
Iterator<Map.Entry<Index, List<Pair<Integer, Integer>>>> indexExprAndVarIt =
analysisCtx.getIteratorForIndexExprsAndVars();
boolean hasIndexPreferences = false;
@@ -363,7 +363,7 @@
final IOptimizableFuncExpr optFuncExpr = analysisCtx.getMatchedFuncExpr(exprAndVarIdx.first);
// If expr is not optimizable by concrete index then remove
// expr and continue.
- if (!accessMethod.exprIsOptimizable(index, optFuncExpr)) {
+ if (!accessMethod.exprIsOptimizable(index, optFuncExpr, checkApplicableOnly)) {
exprsAndVarIter.remove();
continue;
}
@@ -662,7 +662,7 @@
* optimizable function expression.
*
* @return true if a candidate index was added to foundIndexExprs, false
- * otherwise
+ * otherwise
* @throws AlgebricksException
*/
protected boolean fillIndexExprs(List<Index> datasetIndexes, List<String> fieldName, IAType fieldType,
@@ -821,7 +821,7 @@
// Remember matching subtree.
optFuncExpr.setOptimizableSubTree(funcVarIndex, subTree);
List<String> fieldName = null;
- MutableInt fieldSource = new MutableInt(0);
+ int fieldSource = 0;
if (subTree.getDataSourceType() == DataSourceType.COLLECTION_SCAN) {
ILogicalExpression expr = optFuncExpr.getArgument(funcVarIndex).getValue();
if (expr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
@@ -832,9 +832,11 @@
if (subTree.getDataSourceType() == DataSourceType.DATASOURCE_SCAN) {
subTree.setLastMatchedDataSourceVars(0, funcVarIndex);
}
- fieldName = AccessMethodUtils.getFieldNameSetStepsFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex, 0,
- subTree.getRecordType(), funcVarIndex, optFuncExpr.getArgument(funcVarIndex).getValue(),
- subTree.getMetaRecordType(), datasetMetaVar, fieldSource, false);
+ Pair<List<String>, Integer> fieldNameAndSource =
+ AccessMethodUtils.getFieldNameSetStepsFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex, 0,
+ funcVarIndex, optFuncExpr.getArgument(funcVarIndex).getValue(), context);
+ fieldName = fieldNameAndSource.first;
+ fieldSource = fieldNameAndSource.second;
if (fieldName.isEmpty()) {
return;
}
@@ -843,13 +845,13 @@
(IAType) context.getOutputTypeEnvironment(unnestOp).getType(optFuncExpr.getLogicalExpr(funcVarIndex));
// Set the fieldName in the corresponding matched function
// expression.
- optFuncExpr.setFieldName(funcVarIndex, fieldName, fieldSource.intValue());
+ optFuncExpr.setFieldName(funcVarIndex, fieldName, fieldSource);
optFuncExpr.setFieldType(funcVarIndex, fieldType);
setTypeTag(context, subTree, optFuncExpr, funcVarIndex);
if (subTree.hasDataSource()) {
fillIndexExprs(datasetIndexes, fieldName, fieldType, optFuncExpr, optFuncExprIndex, funcVarIndex, subTree,
- analysisCtx, fieldSource.intValue(), accessMethod);
+ analysisCtx, fieldSource, accessMethod);
}
}
@@ -860,7 +862,6 @@
boolean doesArrayIndexQualify = context.getPhysicalOptimizationConfig().isArrayIndexEnabled()
&& datasetIndexes.stream().anyMatch(i -> i.getIndexType() == IndexType.ARRAY);
List<LogicalVariable> varList = assignOp.getVariables();
- MutableInt fieldSource = new MutableInt(0);
for (int varIndex = 0; varIndex < varList.size(); varIndex++) {
LogicalVariable var = varList.get(varIndex);
int optVarIndex = optFuncExpr.findLogicalVar(var);
@@ -871,8 +872,7 @@
.analyzeVarForArrayIndexes(datasetIndexes, optFuncExpr, subTree, context, var, analysisCtx);
if (fieldTriplet != null && subTree.hasDataSource()) {
fillIndexExprs(datasetIndexes, fieldTriplet.second, fieldTriplet.third, optFuncExpr,
- optFuncExprIndex, fieldTriplet.first, subTree, analysisCtx, fieldSource.intValue(),
- accessMethod);
+ optFuncExprIndex, fieldTriplet.first, subTree, analysisCtx, 0, accessMethod);
}
}
continue;
@@ -885,22 +885,22 @@
subTree.setLastMatchedDataSourceVars(varIndex, optVarIndex);
}
- fieldSource.setValue(0);
- List<String> fieldName = AccessMethodUtils.getFieldNameSetStepsFromSubTree(optFuncExpr, subTree,
- assignOrUnnestIndex, varIndex, subTree.getRecordType(), optVarIndex,
- optFuncExpr.getArgument(optVarIndex).getValue(), subTree.getMetaRecordType(), datasetMetaVar,
- fieldSource, false);
+ Pair<List<String>, Integer> fieldNameAndSource =
+ AccessMethodUtils.getFieldNameSetStepsFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex,
+ varIndex, optVarIndex, optFuncExpr.getArgument(optVarIndex).getValue(), context);
+ List<String> fieldName = fieldNameAndSource.first;
+ int fieldSource = fieldNameAndSource.second;
IAType fieldType = (IAType) context.getOutputTypeEnvironment(assignOp).getVarType(var);
// Set the fieldName in the corresponding matched
// function expression.
- optFuncExpr.setFieldName(optVarIndex, fieldName, fieldSource.intValue());
+ optFuncExpr.setFieldName(optVarIndex, fieldName, fieldSource);
optFuncExpr.setFieldType(optVarIndex, fieldType);
setTypeTag(context, subTree, optFuncExpr, optVarIndex);
if (subTree.hasDataSource()) {
fillIndexExprs(datasetIndexes, fieldName, fieldType, optFuncExpr, optFuncExprIndex, optVarIndex,
- subTree, analysisCtx, fieldSource.intValue(), accessMethod);
+ subTree, analysisCtx, fieldSource, accessMethod);
}
}
}
@@ -989,16 +989,8 @@
/**
* Finds the field name of each variable in the ASSIGN or UNNEST operators of the sub-tree.
*/
- protected void fillFieldNamesInTheSubTree(OptimizableOperatorSubTree subTree) throws AlgebricksException {
- LogicalVariable datasetMetaVar = null;
- if (subTree.getDataSourceType() != DataSourceType.COLLECTION_SCAN
- && subTree.getDataSourceType() != DataSourceType.INDEXONLY_PLAN_SECONDARY_INDEX_LOOKUP) {
- List<LogicalVariable> datasetVars = subTree.getDataSourceVariables();
- if (subTree.getDataset().hasMetaPart()) {
- datasetMetaVar = datasetVars.get(datasetVars.size() - 1);
- }
- }
- MutableInt fieldSource = new MutableInt(0);
+ protected void fillFieldNamesInTheSubTree(OptimizableOperatorSubTree subTree, IOptimizationContext context)
+ throws AlgebricksException {
for (int assignOrUnnestIndex = 0; assignOrUnnestIndex < subTree.getAssignsAndUnnests()
.size(); assignOrUnnestIndex++) {
AbstractLogicalOperator op = subTree.getAssignsAndUnnests().get(assignOrUnnestIndex);
@@ -1009,10 +1001,8 @@
LogicalVariable var = varList.get(varIndex);
// funcVarIndex is not required. Thus, we set it to -1.
// optFuncExpr and parentFuncExpr are not required, too. Thus, we set them to null.
- fieldSource.setValue(0);
List<String> fieldName = AccessMethodUtils.getFieldNameSetStepsFromSubTree(null, subTree,
- assignOrUnnestIndex, varIndex, subTree.getRecordType(), -1, null,
- subTree.getMetaRecordType(), datasetMetaVar, fieldSource, false);
+ assignOrUnnestIndex, varIndex, -1, null, context).first;
if (fieldName != null && !fieldName.isEmpty()) {
subTree.getVarsToFieldNameMap().put(var, fieldName);
}
@@ -1020,14 +1010,11 @@
} else if (op.getOperatorTag() == LogicalOperatorTag.UNNEST) {
UnnestOperator unnestOp = (UnnestOperator) op;
LogicalVariable var = unnestOp.getVariable();
- List<String> fieldName = null;
if (subTree.getDataSourceType() != DataSourceType.COLLECTION_SCAN) {
// funcVarIndex is not required. Thus, we set it to -1.
// optFuncExpr and parentFuncExpr are not required, too. Thus, we set them to null.
- fieldSource.setValue(0);
- fieldName = AccessMethodUtils.getFieldNameSetStepsFromSubTree(null, subTree, assignOrUnnestIndex, 0,
- subTree.getRecordType(), -1, null, subTree.getMetaRecordType(), datasetMetaVar, fieldSource,
- false);
+ List<String> fieldName = AccessMethodUtils.getFieldNameSetStepsFromSubTree(null, subTree,
+ assignOrUnnestIndex, 0, -1, null, context).first;
if (fieldName != null && !fieldName.isEmpty()) {
subTree.getVarsToFieldNameMap().put(var, fieldName);
}
@@ -1052,10 +1039,8 @@
LogicalVariable var = varList.get(varIndex);
// funcVarIndex is not required. Thus, we set it to -1.
// optFuncExpr and parentFuncExpr are not required, too. Thus, we set them to null.
- fieldSource.setValue(0);
List<String> fieldName = AccessMethodUtils.getFieldNameSetStepsFromSubTree(null, subTree,
- assignOrUnnestIndex, varIndex, subTree.getRecordType(), -1, null,
- subTree.getMetaRecordType(), datasetMetaVar, fieldSource, false);
+ assignOrUnnestIndex, varIndex, -1, null, context).first;
if (fieldName != null && !fieldName.isEmpty()) {
subTree.getVarsToFieldNameMap().put(var, fieldName);
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
index 3b39d9d..b66a0fd 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
@@ -24,7 +24,6 @@
import static org.apache.asterix.om.functions.BuiltinFunctions.FIELD_ACCESS_NESTED;
import java.util.ArrayList;
-import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
@@ -32,6 +31,7 @@
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Set;
+import java.util.stream.Collectors;
import org.apache.asterix.algebra.operators.physical.ExternalDataLookupPOperator;
import org.apache.asterix.common.annotations.AbstractExpressionAnnotationWithIndexNames;
@@ -61,6 +61,7 @@
import org.apache.asterix.om.typecomputer.impl.TypeComputeUtils;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AUnionType;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.IAType;
import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
@@ -147,6 +148,8 @@
BuiltinFunctions.YEAR_MONTH_DURATION_DEFAULT_NULL_CONSTRUCTOR,
BuiltinFunctions.UUID_DEFAULT_NULL_CONSTRUCTOR, BuiltinFunctions.BINARY_BASE64_DEFAULT_NULL_CONSTRUCTOR);
+ private final static Pair<List<String>, Integer> NO_FIELD_NAME = new Pair<>(Collections.emptyList(), 0);
+
public static void appendPrimaryIndexTypes(Dataset dataset, IAType itemType, IAType metaItemType,
List<Object> target) throws AlgebricksException {
ARecordType recordType = (ARecordType) itemType;
@@ -432,7 +435,10 @@
dest.addAll(KeyFieldTypeUtil.getArrayBTreeIndexKeyTypes(index, recordType, metaRecordType));
break;
case BTREE:
- dest.addAll(KeyFieldTypeUtil.getBTreeIndexKeyTypes(index, recordType, metaRecordType));
+ //TODO(ali): check if types should be made nullable/missable
+ List<Pair<IAType, Boolean>> bTreeIndexKeyTypes =
+ KeyFieldTypeUtil.getBTreeIndexKeyTypes(index, recordType, metaRecordType);
+ dest.addAll(bTreeIndexKeyTypes.stream().map(Pair::getFirst).collect(Collectors.toList()));
break;
case RTREE:
dest.addAll(KeyFieldTypeUtil.getRTreeIndexKeyTypes(index, recordType, metaRecordType));
@@ -961,6 +967,7 @@
leftOuterMissingValue, context, sourceLoc);
}
+ OperatorManipulationUtil.copyCardCostAnnotations(joinRef.getValue(), finalOp);
joinRef.setValue(finalOp);
return true;
}
@@ -1735,10 +1742,10 @@
* false positive results, we can apply instantTryLock() on PK optimization since a result from these indexes
* doesn't have to be verified by the primary index-lookup and a subsequent SELECT operator.
* (i.e., we can guarantee the correctness of the result.)
- *
+ * <p>
* Case A) non-index-only plan
* sidx-search -> (optional) sort -> (optional) distinct -> pdix-search
- *
+ * <p>
* Case B) index-only plan
* left path (an instantTryLock() on the PK fail path):
* right path(an instantTryLock() on the PK success path):
@@ -1937,8 +1944,9 @@
}
// New < For external datasets indexing>
- private static void appendExternalRecTypes(Dataset dataset, IAType itemType, List<Object> target) {
- target.add(itemType);
+ private static void appendExternalRecTypes(IAType itemType, List<Object> target) {
+ // the output of external-lookup could be missing. Make it unknowable
+ target.add(AUnionType.createUnknownableType(itemType));
}
private static void appendExternalRecPrimaryKeys(Dataset dataset, List<Object> target) throws AlgebricksException {
@@ -1998,7 +2006,7 @@
List<Object> outputTypes = new ArrayList<>();
// Append output variables/types generated by the data scan (not forwarded from input).
externalUnnestVars.addAll(dataSourceOp.getVariables());
- appendExternalRecTypes(dataset, recordType, outputTypes);
+ appendExternalRecTypes(recordType, outputTypes);
IFunctionInfo externalLookup = FunctionUtil.getFunctionInfo(BuiltinFunctions.EXTERNAL_LOOKUP);
AbstractFunctionCallExpression externalLookupFunc =
@@ -2358,10 +2366,9 @@
* unless the variables are produced after the SELECT (JOIN) operator.
*
* @return Pair<Boolean, Boolean>: the first boolean value tells whether the given plan is an index-only plan.
- * The second boolean value tells whether the secondary key field variable(s) are used after the given
- * SELECT (JOIN) operator.
+ * The second boolean value tells whether the secondary key field variable(s) are used after the given
+ * SELECT (JOIN) operator.
* @throws AlgebricksException
- *
*/
private static void checkVarUsageAfterSelectOp(List<Mutable<ILogicalOperator>> afterSelectOpRefs,
List<LogicalVariable> liveVarsAfterSelJoinOp, List<LogicalVariable> dataScanPKVars,
@@ -2848,12 +2855,9 @@
/**
* Checks whether a LogicalVariable exists in a list of Triple<LogicalVariable, LogicalVariable, LogicalVariable>.
*
- * @param varsList
- * list that contains triples of LogicalVariable.
- * @param varToFind
- * a LogicalVariable to find
- * @param checkOnlyFirst
- * specifies whether it is required to check only the first variable in the given triple.
+ * @param varsList list that contains triples of LogicalVariable.
+ * @param varToFind a LogicalVariable to find
+ * @param checkOnlyFirst specifies whether it is required to check only the first variable in the given triple.
* @return
*/
public static boolean findVarInTripleVarList(
@@ -2898,17 +2902,16 @@
return ann == null ? null : ann.getIndexNames();
}
- public static List<String> getFieldNameSetStepsFromSubTree(IOptimizableFuncExpr optFuncExpr,
- OptimizableOperatorSubTree subTree, int opIndex, int assignVarIndex, ARecordType recordType,
- int funcVarIndex, ILogicalExpression parentFuncExpr, ARecordType metaType, LogicalVariable metaVar,
- MutableInt fieldSource, boolean isUnnestOverVarAllowed) throws AlgebricksException {
+ public static Pair<List<String>, Integer> getFieldNameSetStepsFromSubTree(IOptimizableFuncExpr optFuncExpr,
+ OptimizableOperatorSubTree subTree, int opIndex, int assignVarIndex, int funcVarIndex,
+ ILogicalExpression parentFuncExpr, IOptimizationContext context) throws AlgebricksException {
if (optFuncExpr != null) {
if (parentFuncExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
optFuncExpr.addStepExpr(funcVarIndex, ((AbstractFunctionCallExpression) parentFuncExpr));
}
}
- return getFieldNameAndStepsFromSubTree(optFuncExpr, subTree, opIndex, assignVarIndex, recordType, funcVarIndex,
- parentFuncExpr, metaType, metaVar, fieldSource, isUnnestOverVarAllowed);
+ return getFieldNameAndStepsFromSubTree(optFuncExpr, subTree, opIndex, assignVarIndex, funcVarIndex,
+ parentFuncExpr, context);
}
/**
@@ -2918,10 +2921,9 @@
*
* @throws AlgebricksException
*/
- private static List<String> getFieldNameAndStepsFromSubTree(IOptimizableFuncExpr optFuncExpr,
- OptimizableOperatorSubTree subTree, int opIndex, int assignVarIndex, ARecordType recordType,
- int funcVarIndex, ILogicalExpression parentFuncExpr, ARecordType metaType, LogicalVariable metaVar,
- MutableInt fieldSource, boolean isUnnestOverVarAllowed) throws AlgebricksException {
+ private static Pair<List<String>, Integer> getFieldNameAndStepsFromSubTree(IOptimizableFuncExpr optFuncExpr,
+ OptimizableOperatorSubTree subTree, int opIndex, int assignVarIndex, int funcVarIndex,
+ ILogicalExpression parentFuncExpr, IOptimizationContext context) throws AlgebricksException {
// Get expression corresponding to opVar at varIndex.
AbstractLogicalExpression expr = null;
AbstractFunctionCallExpression childFuncExpr = null;
@@ -2931,23 +2933,23 @@
expr = (AbstractLogicalExpression) assignOp.getExpressions().get(assignVarIndex).getValue();
// Can't get a field name from a constant expression. So, return null.
if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
- return Collections.emptyList();
+ return NO_FIELD_NAME;
}
childFuncExpr = (AbstractFunctionCallExpression) expr;
} else {
UnnestOperator unnestOp = (UnnestOperator) op;
expr = (AbstractLogicalExpression) unnestOp.getExpressionRef().getValue();
if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
- return Collections.emptyList();
+ return NO_FIELD_NAME;
}
childFuncExpr = (AbstractFunctionCallExpression) expr;
if (childFuncExpr.getFunctionIdentifier() != BuiltinFunctions.SCAN_COLLECTION) {
- return Collections.emptyList();
+ return NO_FIELD_NAME;
}
expr = (AbstractLogicalExpression) childFuncExpr.getArguments().get(0).getValue();
}
if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
- return Collections.emptyList();
+ return NO_FIELD_NAME;
}
AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
FunctionIdentifier funcIdent = funcExpr.getFunctionIdentifier();
@@ -2960,21 +2962,21 @@
if (funcIdent == BuiltinFunctions.FIELD_ACCESS_BY_NAME) {
fieldName = ConstantExpressionUtil.getStringArgument(funcExpr, 1);
if (fieldName == null) {
- return Collections.emptyList();
+ return NO_FIELD_NAME;
}
isFieldAccess = true;
isByName = true;
} else if (funcIdent == BuiltinFunctions.FIELD_ACCESS_BY_INDEX) {
Integer idx = ConstantExpressionUtil.getIntArgument(funcExpr, 1);
if (idx == null) {
- return Collections.emptyList();
+ return NO_FIELD_NAME;
}
fieldIndex = idx;
isFieldAccess = true;
} else if (funcIdent == BuiltinFunctions.FIELD_ACCESS_NESTED) {
ILogicalExpression nameArg = funcExpr.getArguments().get(1).getValue();
if (nameArg.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
- return Collections.emptyList();
+ return NO_FIELD_NAME;
}
ConstantExpression constExpr = (ConstantExpression) nameArg;
AOrderedList orderedNestedFieldName =
@@ -2989,11 +2991,6 @@
if (isFieldAccess) {
LogicalVariable sourceVar =
((VariableReferenceExpression) funcExpr.getArguments().get(0).getValue()).getVariableReference();
- if (sourceVar.equals(metaVar)) {
- fieldSource.setValue(1);
- } else {
- fieldSource.setValue(0);
- }
if (optFuncExpr != null) {
optFuncExpr.setLogicalExpr(funcVarIndex, parentFuncExpr);
optFuncExpr.addStepExpr(funcVarIndex, funcExpr);
@@ -3024,8 +3021,7 @@
if (parentVars.contains(var)) {
//Found the variable we are looking for.
//return assign and index of expression
- int[] returnValues = { i, varIndex };
- assignAndExpressionIndexes = returnValues;
+ assignAndExpressionIndexes = new int[] { i, varIndex };
}
}
}
@@ -3033,34 +3029,25 @@
//We found the nested assign
//Recursive call on nested assign
- List<String> parentFieldNames = getFieldNameAndStepsFromSubTree(optFuncExpr, subTree,
- assignAndExpressionIndexes[0], assignAndExpressionIndexes[1], recordType, funcVarIndex,
- parentFuncExpr, metaType, metaVar, fieldSource, isUnnestOverVarAllowed);
+ Pair<List<String>, Integer> parentFieldNames =
+ getFieldNameAndStepsFromSubTree(optFuncExpr, subTree, assignAndExpressionIndexes[0],
+ assignAndExpressionIndexes[1], funcVarIndex, parentFuncExpr, context);
- boolean isPreviousOperatorLegalUnnest = isUnnestOverVarAllowed && subTree.getAssignsAndUnnests()
- .get(assignAndExpressionIndexes[0]).getOperatorTag().equals(LogicalOperatorTag.UNNEST);
- if (parentFieldNames.isEmpty() && !isPreviousOperatorLegalUnnest) {
+ if (parentFieldNames.first.isEmpty()) {
//Nested assign was not a field access.
//We will not use index
- return Collections.emptyList();
- } else if (isPreviousOperatorLegalUnnest) {
- parentFieldNames = new ArrayList<>();
+ return NO_FIELD_NAME;
}
if (!isByName) {
- IAType subFieldType;
- if (isUnnestOverVarAllowed && isPreviousOperatorLegalUnnest) {
- // In the case of UNNESTing over a variable, we use the record type given by our caller instead.
- subFieldType = sourceVar.equals(metaVar) ? metaType : recordType;
- } else {
- subFieldType = sourceVar.equals(metaVar) ? metaType.getSubFieldType(parentFieldNames)
- : recordType.getSubFieldType(parentFieldNames);
- // Sub-field type can be AUnionType in case if optional. Thus, needs to get the actual type.
- subFieldType = TypeComputeUtils.getActualType(subFieldType);
- if (subFieldType.getTypeTag() != ATypeTag.OBJECT) {
- throw CompilationException.create(ErrorCode.TYPE_CONVERT, subFieldType,
- ARecordType.class.getName());
- }
+ IVariableTypeEnvironment outputTypeEnvironment = context.getOutputTypeEnvironment(
+ subTree.getAssignsAndUnnests().get(assignAndExpressionIndexes[0]));
+ IAType subFieldType = (IAType) outputTypeEnvironment.getVarType(sourceVar);
+ // Sub-field type can be AUnionType in case if optional. Thus, needs to get the actual type.
+ subFieldType = TypeComputeUtils.getActualType(subFieldType);
+ if (subFieldType.getTypeTag() != ATypeTag.OBJECT) {
+ throw CompilationException.create(ErrorCode.TYPE_CONVERT, subFieldType,
+ ARecordType.class.getName());
}
fieldName = ((ARecordType) subFieldType).getFieldNames()[fieldIndex];
@@ -3068,13 +3055,12 @@
if (optFuncExpr != null) {
optFuncExpr.setSourceVar(funcVarIndex, ((AssignOperator) op).getVariables().get(assignVarIndex));
}
+
//add fieldName to the nested fieldName, return
if (nestedAccessFieldName != null) {
- for (int i = 0; i < nestedAccessFieldName.size(); i++) {
- parentFieldNames.add(nestedAccessFieldName.get(i));
- }
+ parentFieldNames.first.addAll(nestedAccessFieldName);
} else {
- parentFieldNames.add(fieldName);
+ parentFieldNames.first.add(fieldName);
}
return (parentFieldNames);
}
@@ -3083,15 +3069,15 @@
optFuncExpr.setSourceVar(funcVarIndex, ((AssignOperator) op).getVariables().get(assignVarIndex));
}
//no nested assign, we are at the lowest level.
+ OptimizableOperatorSubTree.RecordTypeSource recType = subTree.getRecordTypeFor(sourceVar);
if (isByName) {
if (nestedAccessFieldName != null) {
- return nestedAccessFieldName;
+ return new Pair<>(nestedAccessFieldName, recType.sourceIndicator);
}
- return new ArrayList<>(Arrays.asList(fieldName));
+ return new Pair<>(new ArrayList<>(List.of(fieldName)), recType.sourceIndicator);
}
- return new ArrayList<>(Arrays.asList(sourceVar.equals(metaVar) ? metaType.getFieldNames()[fieldIndex]
- : recordType.getFieldNames()[fieldIndex]));
-
+ return new Pair<>(new ArrayList<>(List.of(recType.recordType.getFieldNames()[fieldIndex])),
+ recType.sourceIndicator);
}
// We use a part of the field in edit distance computation
@@ -3101,17 +3087,17 @@
}
List<Mutable<ILogicalExpression>> funcArgs = funcExpr.getArguments();
if (funcArgs.isEmpty()) {
- return Collections.emptyList();
+ return NO_FIELD_NAME;
}
// We expect the function's argument to be a variable, otherwise we
// cannot apply an index.
ILogicalExpression argExpr = funcArgs.get(0).getValue();
if (argExpr.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
- return Collections.emptyList();
+ return NO_FIELD_NAME;
}
for (int i = 1; i < funcArgs.size(); i++) {
if (funcArgs.get(i).getValue().getExpressionTag() != LogicalExpressionTag.CONSTANT) {
- return Collections.emptyList();
+ return NO_FIELD_NAME;
}
}
if (optFuncExpr != null) {
@@ -3131,20 +3117,19 @@
if (var.equals(curVar) && optFuncExpr != null) {
optFuncExpr.setSourceVar(funcVarIndex, var);
return getFieldNameAndStepsFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex, varIndex,
- recordType, funcVarIndex, childFuncExpr, metaType, metaVar, fieldSource,
- isUnnestOverVarAllowed);
+ funcVarIndex, childFuncExpr, context);
}
}
} else {
UnnestOperator unnestOp = (UnnestOperator) curOp;
LogicalVariable var = unnestOp.getVariable();
if (var.equals(curVar)) {
- getFieldNameAndStepsFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex, 0, recordType,
- funcVarIndex, childFuncExpr, metaType, metaVar, fieldSource, isUnnestOverVarAllowed);
+ getFieldNameAndStepsFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex, 0, funcVarIndex,
+ childFuncExpr, context);
}
}
}
- return Collections.emptyList();
+ return NO_FIELD_NAME;
}
public static Triple<Integer, List<String>, IAType> analyzeVarForArrayIndexes(List<Index> datasetIndexes,
@@ -3177,7 +3162,7 @@
List<String> flatName = ArrayIndexUtil.getFlattenedKeyFieldNames(e.getUnnestList(), project);
List<Boolean> unnestFlags = ArrayIndexUtil.getUnnestFlags(e.getUnnestList(), project);
analysisCtx.getArrayIndexStructureMatcher().reset(assignVar, subTree);
- ArrayIndexUtil.walkArrayPath(index, subTree.getRecordType(), flatName, unnestFlags,
+ ArrayIndexUtil.walkArrayPath(index, e, subTree.getRecordType(), flatName, unnestFlags,
analysisCtx.getArrayIndexStructureMatcher());
LogicalVariable varAfterWalk = analysisCtx.getArrayIndexStructureMatcher().getEndVar();
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
index 2eebe5b..2a94359 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
@@ -796,6 +796,7 @@
metaRecordType, dataset, context);
}
+ OperatorManipulationUtil.copyCardCostAnnotations(dataSourceOp, indexSearchOp);
return indexSearchOp;
}
@@ -1020,7 +1021,8 @@
}
@Override
- public boolean exprIsOptimizable(Index index, IOptimizableFuncExpr optFuncExpr) throws AlgebricksException {
+ public boolean exprIsOptimizable(Index index, IOptimizableFuncExpr optFuncExpr, boolean checkApplicableOnly)
+ throws AlgebricksException {
// If we are optimizing a join, check for the indexed nested-loop join hint.
if (optFuncExpr.getNumLogicalVars() == 2) {
if (optFuncExpr.getOperatorSubTree(0) == optFuncExpr.getOperatorSubTree(1)) {
@@ -1033,7 +1035,8 @@
//And we were unable to determine its type
return false;
}
- } else if (!optFuncExpr.getFuncExpr().hasAnnotation(IndexedNLJoinExpressionAnnotation.class)) {
+ } else if (!checkApplicableOnly
+ && !optFuncExpr.getFuncExpr().hasAnnotation(IndexedNLJoinExpressionAnnotation.class)) {
return false;
}
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IAccessMethod.java
index 5688a70..bac1a0b 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IAccessMethod.java
@@ -47,8 +47,8 @@
/**
* @return A list of function identifiers that are optimizable by this
- * access method. Also, the second boolean tells whether that
- * function can generate a false-positive result.
+ * access method. Also, the second boolean tells whether that
+ * function can generate a false-positive result.
*/
public List<Pair<FunctionIdentifier, Boolean>> getOptimizableFunctions();
@@ -61,7 +61,7 @@
* OptimizableFunction to analysisCtx.matchedFuncExprs for further analysis.
*
* @return true if funcExpr is optimizable by this access method, false
- * otherwise
+ * otherwise
* @throws AlgebricksException
*/
boolean analyzeFuncExprArgsAndUpdateAnalysisCtx(AbstractFunctionCallExpression funcExpr,
@@ -71,8 +71,8 @@
/**
* Indicates whether this access method is applicable for the given index type.
*
- * @return boolean
* @param indexType
+ * @return boolean
*/
public boolean matchIndexType(IndexType indexType);
@@ -80,8 +80,8 @@
* Indicates whether all index expressions must be matched in order for this
* index to be applicable.
*
- * @return boolean
* @param index
+ * @return boolean
*/
public boolean matchAllIndexExprs(Index index);
@@ -89,8 +89,8 @@
* Indicates whether this index is applicable if only a prefix of the index
* expressions are matched.
*
- * @return boolean
* @param index
+ * @return boolean
*/
public boolean matchPrefixIndexExprs(Index index);
@@ -127,7 +127,8 @@
*
* @throws AlgebricksException
*/
- public boolean exprIsOptimizable(Index index, IOptimizableFuncExpr optFuncExpr) throws AlgebricksException;
+ public boolean exprIsOptimizable(Index index, IOptimizableFuncExpr optFuncExpr, boolean checkApplicableOnly)
+ throws AlgebricksException;
public Collection<String> getSecondaryIndexPreferences(IOptimizableFuncExpr optFuncExpr);
@@ -136,12 +137,11 @@
/**
* Checks whether the function applied to an indexed field is acceptable by the access method.
*
- * @param functionExpr applied function
- * @param index the index definition
+ * @param functionExpr applied function
+ * @param index the index definition
* @param indexedFieldType the type of the indexed field in the index definition
- * @param defaultNull true if the candidate index has CAST (DEFAULT NULL) modifier
- * @param finalStep true if the functionExpr is the final function applied
- *
+ * @param defaultNull true if the candidate index has CAST (DEFAULT NULL) modifier
+ * @param finalStep true if the functionExpr is the final function applied
* @return true if the access method accepts the argument function. False, otherwise.
*/
public boolean acceptsFunction(AbstractFunctionCallExpression functionExpr, Index index, IAType indexedFieldType,
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
index 6f53219..b2b5e7c 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
@@ -58,21 +58,21 @@
* The order of the join inputs matters (left branch - outer relation, right branch - inner relation).
* This rule tries to utilize an index on the inner relation.
* If that's not possible, it stops transforming the given join into an index-nested-loop join.
- *
+ * <p>
* This rule replaces the above pattern with the following simplified plan:
* select <-- assign+ <-- unnest-map(pidx) <-- sort <-- unnest-map(sidx) <-- assign+ <-- (datasource scan|unnest-map)
* The sorting PK process is optional, and some access methods may choose not to sort.
* Note that for some index-based optimizations we do not remove the triggering
* condition from the join, since the secondary index may only act as a filter, and the
* final verification must still be done with the original join condition.
- *
+ * <p>
* The basic outline of this rule is:
* 1. Match operator pattern.
* 2. Analyze join condition to see if there are optimizable functions (delegated to IAccessMethods).
* 3. Check metadata to see if there are applicable indexes.
* 4. Choose an index to apply (for now only a single index will be chosen).
* 5. Rewrite plan using index (delegated to IAccessMethods).
- *
+ * <p>
* For left-outer-join, additional patterns are checked and additional treatment is needed as follows:
* 1. First it checks if there is a groupByOp above the join: groupby <-- leftouterjoin
* 2. Inherently, only the right-subtree of the lojOp can be used as indexSubtree.
@@ -81,7 +81,7 @@
* Here, the primary key variable from datasourceScanOp replaces the introduced null placeholder variable.
* If the primary key is a composite key, then the first variable of the primary key variables becomes the
* null place holder variable. This null placeholder variable works for all three types of indexes.
- *
+ * <p>
* If the inner-branch can be transformed as an index-only plan, this rule creates an index-only-plan path
* that is similar to one described in IntroduceSelectAccessMethod Rule.
*/
@@ -140,7 +140,7 @@
afterJoinRefs = new ArrayList<>();
// Recursively checks the given plan whether the desired pattern exists in it.
// If so, try to optimize the plan.
- boolean planTransformed = checkAndApplyJoinTransformation(opRef, context);
+ boolean planTransformed = checkAndApplyJoinTransformation(opRef, context, false);
if (joinOp != null) {
// We found an optimization here. Don't need to optimize this operator again.
@@ -156,6 +156,27 @@
return planTransformed;
}
+ public boolean checkApplicable(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+ throws AlgebricksException {
+ clear();
+ setMetadataDeclarations(context);
+
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+
+ afterJoinRefs = new ArrayList<>();
+ // Recursively checks the given plan whether the desired pattern exists in it.
+ // If so, try to optimize the plan.
+ boolean planTransformed = checkAndApplyJoinTransformation(opRef, context, true);
+
+ if (!planTransformed) {
+ return false;
+ } else {
+ OperatorPropertiesUtil.typeOpRec(opRef, context);
+ }
+
+ return planTransformed;
+ }
+
/**
* Removes indexes from the outer branch from the optimizer's consideration for this rule,
* since we only use indexes from the inner branch.
@@ -234,8 +255,8 @@
* optimize the path from the given join operator to the EMPTY_TUPLE_SOURCE operator
* if it is not already optimized.
*/
- protected boolean checkAndApplyJoinTransformation(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
- throws AlgebricksException {
+ protected boolean checkAndApplyJoinTransformation(Mutable<ILogicalOperator> opRef, IOptimizationContext context,
+ boolean checkApplicableOnly) throws AlgebricksException {
AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
boolean joinFoundAndOptimizationApplied;
@@ -246,7 +267,7 @@
// Recursively check the plan and try to optimize it. We first check the children of the given operator
// to make sure an earlier join in the path is optimized first.
for (Mutable<ILogicalOperator> inputOpRef : op.getInputs()) {
- joinFoundAndOptimizationApplied = checkAndApplyJoinTransformation(inputOpRef, context);
+ joinFoundAndOptimizationApplied = checkAndApplyJoinTransformation(inputOpRef, context, checkApplicableOnly);
if (joinFoundAndOptimizationApplied) {
return true;
}
@@ -313,7 +334,7 @@
// the subplan into the index branch and giving the join a condition for this rule to optimize.
// *No nodes* from this rewrite will be used beyond this point.
joinFromSubplanRewrite.findAfterSubplanSelectOperator(afterJoinRefs);
- if (rewriteLocallyAndTransform(joinRef, context, joinFromSubplanRewrite)) {
+ if (rewriteLocallyAndTransform(joinRef, context, joinFromSubplanRewrite, checkApplicableOnly)) {
// Connect the after-join operators to the index subtree root before this rewrite. This also avoids
// performing the secondary index validation step twice.
ILogicalOperator lastAfterJoinOp = afterJoinRefs.get(afterJoinRefs.size() - 1).getValue();
@@ -364,7 +385,7 @@
fillSubTreeIndexExprs(rightSubTree, analyzedAMs, context, false);
// Prunes the access methods based on the function expression and access methods.
- pruneIndexCandidates(analyzedAMs, context, typeEnvironment);
+ pruneIndexCandidates(analyzedAMs, context, typeEnvironment, checkApplicableOnly);
// If the right subtree (inner branch) has indexes, one of those indexes will be used.
// Removes the indexes from the outer branch in the optimizer's consideration list for this rule.
@@ -382,10 +403,10 @@
// Finds the field name of each variable in the sub-tree such as variables for order by.
// This step is required when checking index-only plan.
if (checkLeftSubTreeMetadata) {
- fillFieldNamesInTheSubTree(leftSubTree);
+ fillFieldNamesInTheSubTree(leftSubTree, context);
}
if (checkRightSubTreeMetadata) {
- fillFieldNamesInTheSubTree(rightSubTree);
+ fillFieldNamesInTheSubTree(rightSubTree, context);
}
// Applies the plan transformation using chosen index.
@@ -426,6 +447,10 @@
return false;
}
+ if (checkApplicableOnly) {
+ return true;
+ }
+
// Finally, tries to apply plan transformation using the chosen index.
boolean res = chosenIndex.first.applyJoinPlanTransformation(afterJoinRefs, joinRef, leftSubTree,
rightSubTree, chosenIndex.second, analysisCtx, context, isLeftOuterJoin,
@@ -487,12 +512,13 @@
}
private boolean rewriteLocallyAndTransform(Mutable<ILogicalOperator> opRef, IOptimizationContext context,
- IIntroduceAccessMethodRuleLocalRewrite<AbstractBinaryJoinOperator> rewriter) throws AlgebricksException {
+ IIntroduceAccessMethodRuleLocalRewrite<AbstractBinaryJoinOperator> rewriter, boolean checkApplicableOnly)
+ throws AlgebricksException {
AbstractBinaryJoinOperator joinRewrite = rewriter.createOperator(joinOp, context);
boolean transformationResult = false;
if (joinRewrite != null) {
Mutable<ILogicalOperator> joinRuleInput = new MutableObject<>(joinRewrite);
- transformationResult = checkAndApplyJoinTransformation(joinRuleInput, context);
+ transformationResult = checkAndApplyJoinTransformation(joinRuleInput, context, checkApplicableOnly);
}
// Restore our state, so we can look for more optimizations if this transformation failed.
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
index d9b5da9..f021845 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
@@ -50,10 +50,12 @@
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator.ExecutionMode;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.DelegateOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.IntersectOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import org.apache.hyracks.algebricks.core.algebra.prettyprint.IPlanPrettyPrinter;
import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
/**
@@ -176,7 +178,44 @@
afterSelectRefs = new ArrayList<>();
// Recursively check the given plan whether the desired pattern exists in it.
// If so, try to optimize the plan.
- boolean planTransformed = checkAndApplyTheSelectTransformation(opRef, context);
+ List<Pair<IAccessMethod, Index>> chosenIndexes = new ArrayList<>();
+ Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs = null;
+ boolean planTransformed =
+ checkAndApplyTheSelectTransformation(opRef, context, false, chosenIndexes, analyzedAMs);
+
+ if (selectOp != null) {
+ // We found an optimization here. Don't need to optimize this operator again.
+ context.addToDontApplySet(this, selectOp);
+ }
+
+ if (!planTransformed) {
+ return false;
+ } else {
+ OperatorPropertiesUtil.typeOpRec(opRef, context);
+ }
+
+ return planTransformed;
+ }
+
+ public boolean checkApplicable(Mutable<ILogicalOperator> opRef, IOptimizationContext context,
+ List<Pair<IAccessMethod, Index>> chosenIndexes, Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs)
+ throws AlgebricksException {
+ clear();
+ setMetadataDeclarations(context);
+
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+
+ // Already checked?
+ if (context.checkIfInDontApplySet(this, op)) {
+ return false;
+ }
+
+ afterSelectRefs = new ArrayList<>();
+ // Recursively check the given plan whether the desired pattern exists in it.
+ // If so, try to optimize the plan.
+
+ boolean planTransformed =
+ checkAndApplyTheSelectTransformation(opRef, context, true, chosenIndexes, analyzedAMs);
if (selectOp != null) {
// We found an optimization here. Don't need to optimize this operator again.
@@ -256,7 +295,7 @@
*
* @param chosenIndexes
* @return Pair<IAccessMethod, Index> for the primary index
- * null otherwise
+ * null otherwise
* @throws AlgebricksException
*/
private Pair<IAccessMethod, Index> fetchPrimaryIndexAmongChosenIndexes(
@@ -286,21 +325,31 @@
"The primary search has multiple inputs.");
}
+ // The operator directly under the subroot may either be an ORDER-BY or a DISTINCT (if an array index).
ILogicalOperator curRoot = subRoots.get(i);
- OrderOperator order = (OrderOperator) curRoot.getInputs().get(0).getValue();
- List<LogicalVariable> orderedColumn = new ArrayList<>(order.getOrderExpressions().size());
- for (Pair<OrderOperator.IOrder, Mutable<ILogicalExpression>> orderExpression : order
- .getOrderExpressions()) {
- if (orderExpression.second.getValue().getExpressionTag() != LogicalExpressionTag.VARIABLE) {
- throw new CompilationException(ErrorCode.COMPILATION_ERROR,
- orderExpression.second.getValue().getSourceLocation(),
- "The order by expression should be variables, but they aren't variables.");
+ ILogicalOperator curRootInput = curRoot.getInputs().get(0).getValue();
+ if (curRootInput.getOperatorTag() == LogicalOperatorTag.ORDER) {
+ OrderOperator order = (OrderOperator) curRootInput;
+ List<LogicalVariable> orderedColumn = new ArrayList<>(order.getOrderExpressions().size());
+ for (Pair<OrderOperator.IOrder, Mutable<ILogicalExpression>> orderExpression : order
+ .getOrderExpressions()) {
+ if (orderExpression.second.getValue().getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+ throw new CompilationException(ErrorCode.COMPILATION_ERROR,
+ orderExpression.second.getValue().getSourceLocation(),
+ "The order by expression should be variables, but they aren't variables.");
+ }
+ VariableReferenceExpression orderedVar =
+ (VariableReferenceExpression) orderExpression.second.getValue();
+ orderedColumn.add(orderedVar.getVariableReference());
}
- VariableReferenceExpression orderedVar =
- (VariableReferenceExpression) orderExpression.second.getValue();
- orderedColumn.add(orderedVar.getVariableReference());
+ inputVars.add(orderedColumn);
+ } else if (curRootInput.getOperatorTag() == LogicalOperatorTag.DISTINCT) {
+ DistinctOperator distinct = (DistinctOperator) curRootInput;
+ inputVars.add(distinct.getDistinctByVarList());
+ } else {
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, curRootInput.getSourceLocation(),
+ "Operator directly below the primary index search should be either a DISTINCT or an ORDER!");
}
- inputVars.add(orderedColumn);
}
List<LogicalVariable> inputVars0 = inputVars.get(0);
@@ -328,11 +377,13 @@
* if it is not already optimized.
*/
protected boolean checkAndApplyTheSelectTransformation(Mutable<ILogicalOperator> opRef,
- IOptimizationContext context) throws AlgebricksException {
+ IOptimizationContext context, boolean checkApplicableOnly, List<Pair<IAccessMethod, Index>> chosenIndexes,
+ Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs) throws AlgebricksException {
AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
boolean selectFoundAndOptimizationApplied;
boolean isSelectOp = false;
+ IPlanPrettyPrinter pp = context.getPrettyPrinter();
Mutable<ILogicalOperator> selectRefFromThisOp = null;
SelectOperator selectOpFromThisOp = null;
@@ -351,7 +402,8 @@
// Recursively check the plan and try to optimize it. We first check the children of the given operator
// to make sure an earlier select in the path is optimized first.
for (Mutable<ILogicalOperator> inputOpRef : op.getInputs()) {
- selectFoundAndOptimizationApplied = checkAndApplyTheSelectTransformation(inputOpRef, context);
+ selectFoundAndOptimizationApplied = checkAndApplyTheSelectTransformation(inputOpRef, context,
+ checkApplicableOnly, chosenIndexes, analyzedAMs);
if (selectFoundAndOptimizationApplied) {
return true;
}
@@ -376,8 +428,7 @@
// For each access method, contains the information about
// whether an available index can be applicable or not.
- Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs = null;
- if (continueCheck) {
+ if (!checkApplicableOnly && continueCheck) {
analyzedAMs = new TreeMap<>();
}
@@ -386,14 +437,17 @@
// If there exists a composite atomic-array index, our conjuncts will be split across multiple
// SELECTs. This rewrite is to be used **solely** for the purpose of changing a DATA-SCAN into a
// non-index-only plan branch. No nodes introduced from this rewrite will be used beyond this point.
- if (rewriteLocallyAndTransform(selectRef, context, mergedSelectRewrite)) {
+
+ if (rewriteLocallyAndTransform(selectRef, context, mergedSelectRewrite, checkApplicableOnly,
+ chosenIndexes, analyzedAMs)) {
return true;
}
// If there exists a SUBPLAN in our plan, and we are conditioning on a variable, attempt to rewrite
// this subplan to allow an array-index AM to be introduced. Again, this rewrite is to be used
// **solely** for the purpose of changing a DATA-SCAN into a non-index-only plan branch.
- if (rewriteLocallyAndTransform(selectRef, context, selectFromSubplanRewrite)) {
+ if (rewriteLocallyAndTransform(selectRef, context, selectFromSubplanRewrite, checkApplicableOnly,
+ chosenIndexes, analyzedAMs)) {
return true;
}
}
@@ -426,20 +480,22 @@
fillSubTreeIndexExprs(subTree, analyzedAMs, context, false);
// Prune the access methods based on the function expression and access methods.
- pruneIndexCandidates(analyzedAMs, context, typeEnvironment);
+ pruneIndexCandidates(analyzedAMs, context, typeEnvironment, false);
// Choose all indexes that will be applied.
- List<Pair<IAccessMethod, Index>> chosenIndexes = chooseAllIndexes(analyzedAMs);
+ chooseAllIndexes(analyzedAMs, chosenIndexes);
if (chosenIndexes == null || chosenIndexes.isEmpty()) {
// We can't apply any index for this SELECT operator
context.addToDontApplySet(this, selectRef.getValue());
return false;
}
+ if (checkApplicableOnly) {
+ return true;
+ }
// Apply plan transformation using chosen index.
boolean res;
-
// Primary index applicable?
Pair<IAccessMethod, Index> chosenPrimaryIndex = fetchPrimaryIndexAmongChosenIndexes(chosenIndexes);
if (chosenPrimaryIndex != null) {
@@ -453,7 +509,7 @@
AccessMethodAnalysisContext analysisCtx = analyzedAMs.get(chosenIndexes.get(0).first);
// Finds the field name of each variable in the sub-tree.
- fillFieldNamesInTheSubTree(subTree);
+ fillFieldNamesInTheSubTree(subTree, context);
// Finally, try to apply plan transformation using chosen index.
res = chosenIndexes.get(0).first.applySelectPlanTransformation(afterSelectRefs, selectRef, subTree,
@@ -492,12 +548,23 @@
}
private boolean rewriteLocallyAndTransform(Mutable<ILogicalOperator> opRef, IOptimizationContext context,
- IIntroduceAccessMethodRuleLocalRewrite<SelectOperator> rewriter) throws AlgebricksException {
+ IIntroduceAccessMethodRuleLocalRewrite<SelectOperator> rewriter, boolean checkApplicableOnly,
+ List<Pair<IAccessMethod, Index>> chosenIndexes, Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs)
+ throws AlgebricksException {
+
SelectOperator selectRewrite = rewriter.createOperator(selectOp, context);
boolean transformationResult = false;
if (selectRewrite != null) {
Mutable<ILogicalOperator> selectRuleInput = new MutableObject<>(selectRewrite);
- transformationResult = checkAndApplyTheSelectTransformation(selectRuleInput, context);
+ if (checkApplicableOnly) {
+ transformationResult = checkAndApplyTheSelectTransformation(selectRuleInput, context,
+ checkApplicableOnly, chosenIndexes, analyzedAMs);
+ } else {
+ List<Pair<IAccessMethod, Index>> chosenIndexes2 = new ArrayList<>();
+ Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs2 = null;
+ transformationResult = checkAndApplyTheSelectTransformation(selectRuleInput, context,
+ checkApplicableOnly, chosenIndexes2, analyzedAMs2);
+ }
}
// Restore our state, so we can look for more optimizations if this transformation failed.
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
index 62812aa..031bc74 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
@@ -949,7 +949,8 @@
}
@Override
- public boolean exprIsOptimizable(Index index, IOptimizableFuncExpr optFuncExpr) throws AlgebricksException {
+ public boolean exprIsOptimizable(Index index, IOptimizableFuncExpr optFuncExpr, boolean checkApplicableOnly)
+ throws AlgebricksException {
if (AccessMethodUtils.skipSecondaryIndexRequestedByAnnotation(index, optFuncExpr)) {
return false;
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
index c55edf9..bbfb365 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
@@ -59,7 +59,7 @@
*/
public class OptimizableOperatorSubTree {
- public static enum DataSourceType {
+ public enum DataSourceType {
DATASOURCE_SCAN,
EXTERNAL_SCAN,
PRIMARY_INDEX_LOOKUP,
@@ -68,6 +68,16 @@
NO_DATASOURCE
}
+ public static class RecordTypeSource {
+ final ARecordType recordType;
+ final int sourceIndicator;
+
+ RecordTypeSource(ARecordType recordType, int sourceIndicator) {
+ this.recordType = recordType;
+ this.sourceIndicator = sourceIndicator;
+ }
+ }
+
private ILogicalOperator root = null;
private Mutable<ILogicalOperator> rootRef = null;
private final List<Mutable<ILogicalOperator>> assignsAndUnnestsRefs = new ArrayList<>();
@@ -91,6 +101,7 @@
private List<DataSourceType> ixJoinOuterAdditionalDataSourceTypes = null;
private List<Dataset> ixJoinOuterAdditionalDatasets = null;
private List<ARecordType> ixJoinOuterAdditionalRecordTypes = null;
+ private final Map<LogicalVariable, RecordTypeSource> varsToRecordType = new HashMap<>();
/**
* Identifies the root of the subtree and initializes the data-source, assign, and unnest information.
@@ -172,10 +183,10 @@
AccessMethodJobGenParams jobGenParams = new AccessMethodJobGenParams();
jobGenParams.readFromFuncArgs(f.getArguments());
if (jobGenParams.isPrimaryIndex()) {
- intializeDataSourceRefAndType(DataSourceType.PRIMARY_INDEX_LOOKUP, subTreeOpRef);
+ initializeDataSourceRefAndType(DataSourceType.PRIMARY_INDEX_LOOKUP, subTreeOpRef);
dataSourceFound = true;
} else if (unnestMapOp.getGenerateCallBackProceedResultVar()) {
- intializeDataSourceRefAndType(DataSourceType.INDEXONLY_PLAN_SECONDARY_INDEX_LOOKUP,
+ initializeDataSourceRefAndType(DataSourceType.INDEXONLY_PLAN_SECONDARY_INDEX_LOOKUP,
subTreeOpRef);
dataSourceFound = true;
}
@@ -221,7 +232,7 @@
return false;
}
- private void intializeDataSourceRefAndType(DataSourceType dsType, Mutable<ILogicalOperator> opRef) {
+ private void initializeDataSourceRefAndType(DataSourceType dsType, Mutable<ILogicalOperator> opRef) {
if (getDataSourceRef() == null) {
setDataSourceRef(opRef);
setDataSourceType(dsType);
@@ -238,12 +249,6 @@
* Also sets recordType to be the type of that dataset.
*/
public boolean setDatasetAndTypeMetadata(MetadataProvider metadataProvider) throws AlgebricksException {
- DataverseName dataverseName = null;
- String datasetName = null;
-
- Dataset ds = null;
- ARecordType rType = null;
-
List<Mutable<ILogicalOperator>> sourceOpRefs = new ArrayList<>();
List<DataSourceType> dsTypes = new ArrayList<>();
@@ -259,6 +264,9 @@
}
for (int i = 0; i < sourceOpRefs.size(); i++) {
+ List<LogicalVariable> vars;
+ DataverseName dataverseName;
+ String datasetName;
switch (dsTypes.get(i)) {
case DATASOURCE_SCAN:
DataSourceScanOperator dataSourceScan = (DataSourceScanOperator) sourceOpRefs.get(i).getValue();
@@ -272,6 +280,7 @@
Pair<DataverseName, String> datasetInfo = AnalysisUtil.getDatasetInfo(dataSourceScan);
dataverseName = datasetInfo.first;
datasetName = datasetInfo.second;
+ vars = dataSourceScan.getScanVariables();
break;
case PRIMARY_INDEX_LOOKUP:
case INDEXONLY_PLAN_SECONDARY_INDEX_LOOKUP:
@@ -282,12 +291,14 @@
jobGenParams.readFromFuncArgs(f.getArguments());
datasetName = jobGenParams.getDatasetName();
dataverseName = jobGenParams.getDataverseName();
+ vars = unnestMapOp.getScanVariables();
break;
case EXTERNAL_SCAN:
UnnestMapOperator externalScan = (UnnestMapOperator) sourceOpRefs.get(i).getValue();
datasetInfo = AnalysisUtil.getExternalDatasetInfo(externalScan);
dataverseName = datasetInfo.first;
datasetName = datasetInfo.second;
+ vars = externalScan.getScanVariables();
break;
case COLLECTION_SCAN:
if (i != 0) {
@@ -303,7 +314,7 @@
return false;
}
// Find the dataset corresponding to the datasource in the metadata.
- ds = metadataProvider.findDataset(dataverseName, datasetName);
+ Dataset ds = metadataProvider.findDataset(dataverseName, datasetName);
if (ds == null) {
throw new CompilationException(ErrorCode.NO_METADATA_FOR_DATASET, root.getSourceLocation(),
datasetName);
@@ -318,26 +329,31 @@
getIxJoinOuterAdditionalRecordTypes().add(null);
}
}
- rType = (ARecordType) itemType;
+ ARecordType rType = (ARecordType) itemType;
// Get the meta record type for that dataset.
- IAType metaItemType =
- metadataProvider.findType(ds.getMetaItemTypeDataverseName(), ds.getMetaItemTypeName());
+ ARecordType metaItemType = (ARecordType) metadataProvider.findType(ds.getMetaItemTypeDataverseName(),
+ ds.getMetaItemTypeName());
// First index is always the primary datasource in this subtree.
if (i == 0) {
setDataset(ds);
setRecordType(rType);
- setMetaRecordType((ARecordType) metaItemType);
+ setMetaRecordType(metaItemType);
} else {
getIxJoinOuterAdditionalDatasets().add(ds);
getIxJoinOuterAdditionalRecordTypes().add(rType);
}
- dataverseName = null;
- datasetName = null;
- ds = null;
- rType = null;
+ if (!vars.isEmpty()) {
+ int numVars = vars.size();
+ if (ds.hasMetaPart()) {
+ varsToRecordType.put(vars.get(numVars - 2), new RecordTypeSource(rType, 0));
+ varsToRecordType.put(vars.get(numVars - 1), new RecordTypeSource(metaItemType, 1));
+ } else {
+ varsToRecordType.put(vars.get(numVars - 1), new RecordTypeSource(rType, 0));
+ }
+ }
}
return true;
@@ -364,17 +380,6 @@
return getDataSourceType() == DataSourceType.DATASOURCE_SCAN;
}
- public boolean hasIxJoinOuterAdditionalDataSourceScan() {
- if (getIxJoinOuterAdditionalDataSourceTypes() != null) {
- for (int i = 0; i < getIxJoinOuterAdditionalDataSourceTypes().size(); i++) {
- if (getIxJoinOuterAdditionalDataSourceTypes().get(i) == DataSourceType.DATASOURCE_SCAN) {
- return true;
- }
- }
- }
- return false;
- }
-
public void reset() {
setRoot(null);
setRootRef(null);
@@ -392,6 +397,7 @@
setIxJoinOuterAdditionalRecordTypes(null);
lastMatchedDataSourceVars.first = -1;
lastMatchedDataSourceVars.second = -1;
+ varsToRecordType.clear();
}
/**
@@ -545,6 +551,10 @@
return recordType;
}
+ public RecordTypeSource getRecordTypeFor(LogicalVariable var) {
+ return varsToRecordType.get(var);
+ }
+
public void setRecordType(ARecordType recordType) {
this.recordType = recordType;
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeAccessMethod.java
index 6959543..3de78f7 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeAccessMethod.java
@@ -381,7 +381,7 @@
}
@Override
- public boolean exprIsOptimizable(Index index, IOptimizableFuncExpr optFuncExpr) {
+ public boolean exprIsOptimizable(Index index, IOptimizableFuncExpr optFuncExpr, boolean checkApplicableOnly) {
if (AccessMethodUtils.skipSecondaryIndexRequestedByAnnotation(index, optFuncExpr)) {
return false;
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/AbstractOperatorFromSubplanRewrite.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/AbstractOperatorFromSubplanRewrite.java
index ba70aff..acd557e 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/AbstractOperatorFromSubplanRewrite.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/AbstractOperatorFromSubplanRewrite.java
@@ -100,20 +100,44 @@
this.context = context;
}
- protected LogicalVariable getConditioningVariable(ILogicalExpression condition) {
+ protected void gatherBooleanVariables(ILogicalExpression condition, List<VariableReferenceExpression> outputList,
+ List<ILogicalExpression> miscExpressions) {
List<Mutable<ILogicalExpression>> selectConjuncts = new ArrayList<>();
if (splitIntoConjuncts(condition, selectConjuncts)) {
for (Mutable<ILogicalExpression> conjunct : selectConjuncts) {
if (conjunct.getValue().getExpressionTag().equals(LogicalExpressionTag.VARIABLE)) {
- return ((VariableReferenceExpression) conjunct.getValue()).getVariableReference();
+ outputList.add(((VariableReferenceExpression) conjunct.getValue()));
+ } else {
+ miscExpressions.add(conjunct.getValue());
}
}
-
} else if (condition.getExpressionTag().equals(LogicalExpressionTag.VARIABLE)) {
- return ((VariableReferenceExpression) condition).getVariableReference();
-
+ outputList.add(((VariableReferenceExpression) condition));
+ } else {
+ miscExpressions.add(condition);
}
- return null;
+ }
+
+ protected void gatherSubplanOperators(ILogicalOperator rootOperator, List<SubplanOperator> outputList) {
+ for (Mutable<ILogicalOperator> inputOpRef : rootOperator.getInputs()) {
+ LogicalOperatorTag operatorTag = inputOpRef.getValue().getOperatorTag();
+ switch (operatorTag) {
+ case SUBPLAN:
+ outputList.add((SubplanOperator) inputOpRef.getValue());
+ gatherSubplanOperators(inputOpRef.getValue(), outputList);
+ break;
+
+ case ASSIGN:
+ case UNNEST:
+ case SELECT:
+ gatherSubplanOperators(inputOpRef.getValue(), outputList);
+ break;
+
+ default:
+ // We will break early if we encounter any other operator.
+ return;
+ }
+ }
}
protected Pair<SelectOperator, UnnestOperator> traverseSubplanBranch(SubplanOperator subplanOperator,
@@ -132,7 +156,7 @@
// Ensure that this SELECT represents a predicate for an existential query, and is a query we can optimize.
ILogicalExpression normalizedSelectCondition =
normalizeCondition(workingSubplanRootAsAggregate, optimizableSelect.getCondition().getValue());
- normalizedSelectCondition = keepOptimizableFunctions(normalizedSelectCondition).cloneExpression();
+ normalizedSelectCondition = keepOptimizableFunctions(normalizedSelectCondition);
// Create a copy of this SELECT, and set this to our rewrite root.
SelectOperator rewriteRootSelect = new SelectOperator(new MutableObject<>(normalizedSelectCondition),
@@ -156,9 +180,7 @@
switch (workingOriginalOperator.getOperatorTag()) {
case UNNEST:
UnnestOperator originalUnnest = (UnnestOperator) workingOriginalOperator;
- UnnestOperator newUnnest =
- new UnnestOperator(originalUnnest.getVariable(), originalUnnest.getExpressionRef());
- newUnnest.setSourceLocation(sourceLocation);
+ UnnestOperator newUnnest = (UnnestOperator) OperatorManipulationUtil.deepCopy(originalUnnest);
workingNewOperator.getInputs().add(new MutableObject<>(newUnnest));
workingNewOperator = newUnnest;
bottommostNewUnnest = (UnnestOperator) workingNewOperator;
@@ -166,8 +188,7 @@
case ASSIGN:
AssignOperator originalAssign = (AssignOperator) workingOriginalOperator;
- AssignOperator newAssign =
- new AssignOperator(originalAssign.getVariables(), originalAssign.getExpressions());
+ AssignOperator newAssign = (AssignOperator) OperatorManipulationUtil.deepCopy(originalAssign);
newAssign.setSourceLocation(sourceLocation);
workingNewOperator.getInputs().add(new MutableObject<>(newAssign));
workingNewOperator = newAssign;
@@ -207,8 +228,12 @@
// Add the inputs from our subplan.
Mutable<ILogicalOperator> traversalOperator = traversalOutput.first.getInputs().get(0);
while (traversalOperator != null) {
- workingNewOperator.getInputs().add(new MutableObject<>(
- OperatorManipulationUtil.deepCopy(traversalOperator.getValue())));
+ ILogicalOperator traversalOperatorDeepCopy =
+ OperatorManipulationUtil.deepCopy(traversalOperator.getValue());
+ if (traversalOperator.getValue().equals(traversalOutput.second)) {
+ traversalOutput.second = (UnnestOperator) traversalOperatorDeepCopy;
+ }
+ workingNewOperator.getInputs().add(new MutableObject<>(traversalOperatorDeepCopy));
workingNewOperator = workingNewOperator.getInputs().get(0).getValue();
traversalOperator = (traversalOperator.getValue().getInputs().isEmpty()) ? null
: traversalOperator.getValue().getInputs().get(0);
@@ -282,7 +307,7 @@
break;
}
- return combinedCondition;
+ return combinedCondition.cloneExpression();
}
private SelectOperator getSelectFromPlan(AggregateOperator subplanRoot) {
@@ -373,7 +398,7 @@
return ConstantExpression.TRUE;
}
- private AggregateOperator getAggregateFromSubplan(SubplanOperator subplanOperator) {
+ protected AggregateOperator getAggregateFromSubplan(SubplanOperator subplanOperator) {
// We only expect one plan, and one root.
if (subplanOperator.getNestedPlans().size() > 1
|| subplanOperator.getNestedPlans().get(0).getRoots().size() > 1) {
@@ -486,6 +511,9 @@
} else {
// We are working with a strict universal quantification query.
+ if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+ return expr;
+ }
ScalarFunctionCallExpression notFunction = (ScalarFunctionCallExpression) expr;
if (!notFunction.getFunctionIdentifier().equals(BuiltinFunctions.NOT)) {
return expr;
@@ -496,7 +524,8 @@
if (!ifMissingOrNullFunction.getFunctionIdentifier().equals(BuiltinFunctions.IF_MISSING_OR_NULL)) {
return expr;
}
- return ifMissingOrNullFunction.getArguments().get(0).getValue().cloneExpression();
+ return ifMissingOrNullFunction.getArguments().get(0).getValue();
+
}
}
@@ -508,7 +537,9 @@
if (splitIntoConjuncts(conjunct.getValue(), innerExprConjuncts)) {
conjuncts.addAll(innerExprConjuncts);
} else {
- conjuncts.add(conjunct);
+
+ conjuncts.add(new MutableObject<>(conjunct.getValue()));
+
}
}
return true;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/ArrayIndexStructureMatcher.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/ArrayIndexStructureMatcher.java
index 62b266a..ddfdcac 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/ArrayIndexStructureMatcher.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/ArrayIndexStructureMatcher.java
@@ -23,6 +23,7 @@
import java.util.Collections;
import java.util.List;
+import org.apache.asterix.metadata.entities.Index;
import org.apache.asterix.metadata.utils.ArrayIndexUtil;
import org.apache.asterix.om.base.AInt32;
import org.apache.asterix.om.base.AString;
@@ -72,8 +73,9 @@
}
@Override
- public void executeActionOnFinalArrayStep(ARecordType startingStepRecordType, List<String> fieldName,
- boolean isNonArrayStep, boolean requiresOnlyOneUnnest) {
+ public void executeActionOnFinalArrayStep(Index.ArrayIndexElement workingElement, ARecordType baseRecordType,
+ ARecordType startingStepRecordType, List<String> fieldName, boolean isNonArrayStep,
+ boolean requiresOnlyOneUnnest) {
if (isNonArrayStep) {
isStructureMatched = isStructureMatched && matchAssignVarAndFieldName(startingStepRecordType, fieldName);
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/JoinFromSubplanRewrite.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/JoinFromSubplanRewrite.java
index aa80462..7f003a5 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/JoinFromSubplanRewrite.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/JoinFromSubplanRewrite.java
@@ -74,7 +74,7 @@
* | (potential) index branch ...
* |----------------- probe branch ...
* </pre>
- *
+ * <p>
* If we are given the pattern (a universal quantification over a cross product):
* <pre>
* SELECT_1(some variable AND array is not empty)
@@ -96,7 +96,7 @@
* | (potential) index branch ...
* |----------------- probe branch ...
* </pre>
- *
+ * <p>
* In the case of nested-subplans, we return a copy of the innermost SELECT followed by all relevant UNNEST/ASSIGNs.
*/
public class JoinFromSubplanRewrite extends AbstractOperatorFromSubplanRewrite<AbstractBinaryJoinOperator> {
@@ -123,6 +123,7 @@
return;
}
+ // TODO (GLENN): These assumptions should be relaxed in the future, but for now we'll roll with it.
// We expect a) the operator immediately above to be a SUBPLAN, and b) the next operator above to be a SELECT.
Mutable<ILogicalOperator> afterJoinOpRef1 = afterJoinRefs.get(afterJoinRefs.size() - 1);
Mutable<ILogicalOperator> afterJoinOpRef2 = afterJoinRefs.get(afterJoinRefs.size() - 2);
@@ -136,8 +137,11 @@
}
// Additionally, verify that our SELECT is conditioning on a variable.
+ List<VariableReferenceExpression> booleanVariables = new ArrayList<>();
joinContext.selectAfterSubplan = (SelectOperator) afterJoinOp2;
- if (getConditioningVariable(joinContext.selectAfterSubplan.getCondition().getValue()) == null) {
+ gatherBooleanVariables(joinContext.selectAfterSubplan.getCondition().getValue(), booleanVariables,
+ new ArrayList<>());
+ if (booleanVariables.isEmpty()) {
return;
}
@@ -224,7 +228,7 @@
*/
@Override
public AbstractBinaryJoinOperator restoreBeforeRewrite(List<Mutable<ILogicalOperator>> afterOperatorRefs,
- IOptimizationContext context) throws AlgebricksException {
+ IOptimizationContext context) {
JoinFromSubplanContext joinContext = contextStack.pop();
if (joinContext.removedAfterJoinOperators != null) {
afterOperatorRefs.addAll(joinContext.removedAfterJoinOperators);
@@ -251,9 +255,9 @@
LogicalVariable newVar = context.newVar();
VariableReferenceExpression newVarRef = new VariableReferenceExpression(newVar);
newVarRef.setSourceLocation(joinOp.getSourceLocation());
- AssignOperator newAssign =
- new AssignOperator(newVar, new MutableObject<>(arg.getValue().cloneExpression()));
- newAssign.setSourceLocation(arg.getValue().getSourceLocation());
+ Mutable<ILogicalExpression> clonedArgRef = new MutableObject<>(arg.getValue().cloneExpression());
+ AssignOperator newAssign = new AssignOperator(newVar, clonedArgRef);
+ newAssign.setSourceLocation(clonedArgRef.getValue().getSourceLocation());
newAssign.setExecutionMode(joinOp.getExecutionMode());
// Place the new ASSIGN in the appropriate join branch.
@@ -265,14 +269,14 @@
VariableUtilities.getUsedVariables(newAssign, usedVarsFromFunc);
VariableUtilities.getProducedVariablesInDescendantsAndSelf(leftBranchRoot, varsFromLeftBranch);
VariableUtilities.getProducedVariablesInDescendantsAndSelf(rightBranchRoot, varsFromRightBranch);
- if (varsFromLeftBranch.containsAll(usedVarsFromFunc)) {
+ if (new HashSet<>(varsFromLeftBranch).containsAll(usedVarsFromFunc)) {
newAssign.getInputs().add(new MutableObject<>(leftBranchRoot));
context.computeAndSetTypeEnvironmentForOperator(newAssign);
joinOp.getInputs().get(0).setValue(newAssign);
context.computeAndSetTypeEnvironmentForOperator(joinOp);
arg.setValue(newVarRef);
- } else if (varsFromRightBranch.containsAll(usedVarsFromFunc)) {
+ } else if (new HashSet<>(varsFromRightBranch).containsAll(usedVarsFromFunc)) {
newAssign.getInputs().add(new MutableObject<>(rightBranchRoot));
context.computeAndSetTypeEnvironmentForOperator(newAssign);
joinOp.getInputs().get(1).setValue(newAssign);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/SelectFromSubplanRewrite.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/SelectFromSubplanRewrite.java
index 594ba41..c5794b8 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/SelectFromSubplanRewrite.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/SelectFromSubplanRewrite.java
@@ -19,22 +19,31 @@
package org.apache.asterix.optimizer.rules.am.array;
import java.util.ArrayDeque;
+import java.util.ArrayList;
import java.util.Deque;
import java.util.HashSet;
+import java.util.Iterator;
import java.util.List;
+import java.util.Optional;
import java.util.Set;
+import org.apache.asterix.om.functions.BuiltinFunctions;
import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
-import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import org.apache.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
/**
* For use in writing a "throwaway" branch which removes NTS and subplan operators. The result of this invocation is to
@@ -57,7 +66,7 @@
* UNNEST(on variable)
* (parent branch input)
* </pre>
- *
+ * <p>
* If we are given the pattern (a universal quantification query):
* <pre>
* SELECT_1(some variable AND array is not empty)
@@ -75,7 +84,7 @@
* UNNEST(on variable)
* (parent branch input)
* </pre>
- *
+ * <p>
* In the case of nested-subplans, we return a copy of the innermost SELECT followed by all relevant UNNEST/ASSIGNs.
*/
public class SelectFromSubplanRewrite extends AbstractOperatorFromSubplanRewrite<SelectOperator> {
@@ -98,7 +107,7 @@
* UNNEST
* ...
* </pre>
- *
+ * <p>
* Operators are *created* here, rather than just reconnected from the original branch.
*/
@Override
@@ -108,17 +117,97 @@
selectRootStack.push(originalOperator);
reset(originalOperator.getSourceLocation(), context, optimizableFunctions);
- // We expect a) a SUBPLAN as input to this SELECT, and b) our SELECT to be conditioning on a variable.
- LogicalVariable originalSelectVar = getConditioningVariable(originalOperator.getCondition().getValue());
- if (!originalOperator.getInputs().get(0).getValue().getOperatorTag().equals(LogicalOperatorTag.SUBPLAN)
- || originalSelectVar == null) {
+ // Gather all boolean variables and SUBPLANs.
+ List<VariableReferenceExpression> booleanVariables = new ArrayList<>();
+ List<ILogicalExpression> miscExpressions = new ArrayList<>();
+ List<SubplanOperator> subplanOperators = new ArrayList<>();
+ gatherBooleanVariables(originalOperator.getCondition().getValue(), booleanVariables, miscExpressions);
+ gatherSubplanOperators(originalOperator, subplanOperators);
+ Iterator<SubplanOperator> subplanIterator = subplanOperators.listIterator();
+ if (booleanVariables.isEmpty() || subplanOperators.isEmpty()) {
return null;
}
- // Traverse our subplan and generate a SELECT branch if applicable.
- SubplanOperator subplanOperator = (SubplanOperator) originalOperator.getInputs().get(0).getValue();
- Pair<SelectOperator, UnnestOperator> traversalOutput = traverseSubplanBranch(subplanOperator, null, true);
- return (traversalOutput == null) ? null : traversalOutput.first;
+ // TODO (GLENN): We currently assume that SUBPLAN-SELECTs are back-to-back.
+ SubplanOperator bottommostSubplanOperator = subplanOperators.get(subplanOperators.size() - 1);
+
+ // We now need to match these variables to SUBPLANs downstream.
+ while (subplanIterator.hasNext()) {
+ SubplanOperator workingSubplanOperator = subplanIterator.next();
+ AggregateOperator aggregateFromSubplan = getAggregateFromSubplan(workingSubplanOperator);
+ if (aggregateFromSubplan == null) {
+ continue;
+ }
+
+ boolean isMatchingAggregateVariableFound = false;
+ for (LogicalVariable aggregateVariable : aggregateFromSubplan.getVariables()) {
+ Optional<VariableReferenceExpression> matchingBooleanVariable = booleanVariables.stream()
+ .filter(v -> v.getVariableReference().equals(aggregateVariable)).findFirst();
+ if (matchingBooleanVariable.isPresent()) {
+ isMatchingAggregateVariableFound = true;
+
+ // Note: we (currently) don't expect variables to shared in multiple subplan outputs.
+ booleanVariables.remove(matchingBooleanVariable.get());
+ }
+ }
+ if (!isMatchingAggregateVariableFound) {
+ subplanIterator.remove();
+ }
+ }
+ if (subplanOperators.isEmpty()) {
+ // No <boolean variable, SUBPLAN> pairs could be found.
+ return null;
+ }
+
+ // For all unused boolean variables, we'll add them back to our misc. expression set.
+ miscExpressions.addAll(booleanVariables);
+
+ // For each subplan, traverse and generate a SELECT branch if applicable.
+ List<Pair<SelectOperator, UnnestOperator>> traversalOutputs = new ArrayList<>();
+ for (SubplanOperator subplanBranch : subplanOperators) {
+ Pair<SelectOperator, UnnestOperator> traversalOutput = traverseSubplanBranch(subplanBranch, null, false);
+ if (traversalOutput != null) {
+ traversalOutputs.add(traversalOutput);
+ }
+ }
+ if (traversalOutputs.size() == 0) {
+ return null;
+
+ } else if (traversalOutputs.size() == 1) {
+ Pair<SelectOperator, UnnestOperator> traversalOutput = traversalOutputs.get(0);
+ ILogicalOperator bottommostOperator = traversalOutput.second;
+ SelectOperator selectRewriteOperator = traversalOutput.first;
+ bottommostOperator.getInputs().addAll(bottommostSubplanOperator.getInputs());
+ return finalizeSelectOperator(selectRewriteOperator, miscExpressions, context);
+
+ } else {
+ ScalarFunctionCallExpression workingSelectCondition =
+ new ScalarFunctionCallExpression(BuiltinFunctions.getBuiltinFunctionInfo(BuiltinFunctions.AND));
+ SelectOperator mergedSelectOperator = new SelectOperator(new MutableObject<>(workingSelectCondition));
+ ILogicalOperator workingLeafOperator = mergedSelectOperator;
+ for (Pair<SelectOperator, UnnestOperator> traversalOutput : traversalOutputs) {
+ SelectOperator selectRewriteOperator = traversalOutput.first;
+ ILogicalExpression selectRewriteExpr = selectRewriteOperator.getCondition().getValue();
+
+ // First, we coalesce our SELECT conditions.
+ List<Mutable<ILogicalExpression>> selectRewriteExprConjuncts = new ArrayList<>();
+ if (selectRewriteExpr.splitIntoConjuncts(selectRewriteExprConjuncts)) {
+ for (Mutable<ILogicalExpression> conjunct : selectRewriteExprConjuncts) {
+ workingSelectCondition.getArguments().add(new MutableObject<>(conjunct.getValue()));
+ }
+ } else {
+ workingSelectCondition.getArguments().add(new MutableObject<>(selectRewriteExpr));
+ }
+
+ // Next, we connect the bottommost operator back to the current leaf.
+ workingLeafOperator.getInputs().add(new MutableObject<>(traversalOutput.second));
+ workingLeafOperator = traversalOutput.second;
+ }
+
+ // Finally, we connect the leaf to the bottommost subplan input.
+ workingLeafOperator.getInputs().addAll(bottommostSubplanOperator.getInputs());
+ return finalizeSelectOperator(mergedSelectOperator, miscExpressions, context);
+ }
}
/**
@@ -127,7 +216,31 @@
*/
@Override
public SelectOperator restoreBeforeRewrite(List<Mutable<ILogicalOperator>> afterOperatorRefs,
- IOptimizationContext context) throws AlgebricksException {
+ IOptimizationContext context) {
return selectRootStack.pop();
}
+
+ private SelectOperator finalizeSelectOperator(SelectOperator selectOp, List<ILogicalExpression> auxiliaryExprs,
+ IOptimizationContext context) throws AlgebricksException {
+ if (auxiliaryExprs.isEmpty()) {
+ // There are no auxiliary expressions to add.
+ OperatorManipulationUtil.computeTypeEnvironmentBottomUp(selectOp, context);
+ return selectOp;
+ }
+
+ // Otherwise... we need to build a new SELECT.
+ ScalarFunctionCallExpression workingSelectCondition =
+ new ScalarFunctionCallExpression(BuiltinFunctions.getBuiltinFunctionInfo(BuiltinFunctions.AND));
+ if (!selectOp.getCondition().getValue().splitIntoConjuncts(workingSelectCondition.getArguments())) {
+ workingSelectCondition.getArguments().add(selectOp.getCondition());
+ }
+ for (ILogicalExpression auxiliaryExpr : auxiliaryExprs) {
+ workingSelectCondition.getArguments().add(new MutableObject<>(auxiliaryExpr));
+ //workingSelectCondition.getArguments().add(auxiliaryExpr); // MMK
+ }
+ SelectOperator mergedSelectOperator = new SelectOperator(new MutableObject<>(workingSelectCondition));
+ mergedSelectOperator.getInputs().addAll(selectOp.getInputs());
+ OperatorManipulationUtil.computeTypeEnvironmentBottomUp(mergedSelectOperator, context);
+ return mergedSelectOperator;
+ }
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/EnumerateJoinsRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/EnumerateJoinsRule.java
new file mode 100644
index 0000000..81b18df
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/EnumerateJoinsRule.java
@@ -0,0 +1,631 @@
+/*
+ * 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.optimizer.rules.cbo;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.asterix.common.annotations.IndexedNLJoinExpressionAnnotation;
+import org.apache.asterix.common.annotations.SkipSecondaryIndexSearchExpressionAnnotation;
+import org.apache.asterix.metadata.entities.Index;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableInt;
+import org.apache.commons.lang3.mutable.MutableObject;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.base.OperatorAnnotations;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.BroadcastExpressionAnnotation;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.HashJoinExpressionAnnotation;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionAnnotation;
+import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import org.apache.hyracks.algebricks.core.algebra.prettyprint.IPlanPrettyPrinter;
+import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+public class EnumerateJoinsRule implements IAlgebraicRewriteRule {
+
+ private static final Logger LOGGER = LogManager.getLogger();
+
+ protected final JoinEnum joinEnum;
+
+ public EnumerateJoinsRule(JoinEnum joinEnum) {
+ this.joinEnum = joinEnum;
+ }
+
+ @Override
+ public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
+ return false;
+ }
+
+ /**
+ * If this method returns false, that means CBO code will not be used to optimize the part of the join graph that
+ * was passed in. Currently, we do not optimize query graphs with outer joins in them. If the CBO code is activated
+ * a new join graph (with inputs possibly switched) will be created and the return value will be true.
+ */
+ @Override
+ public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+ throws AlgebricksException {
+ boolean cboMode = this.getCBOMode(context);
+ boolean cboTestMode = this.getCBOTestMode(context);
+ if (!(cboMode || cboTestMode)) {
+ return false;
+ }
+ // If we reach here, then either cboMode or cboTestMode is true.
+ // If cboTestMode is true, then we use predefined cardinalities for datasets for asterixdb regression tests.
+ // If cboMode is true, then all datasets need to have samples, otherwise the check in doAllDataSourcesHaveSamples()
+ // further below will return false.
+ ILogicalOperator op = opRef.getValue();
+ if (!((op.getOperatorTag() == LogicalOperatorTag.INNERJOIN)
+ || ((op.getOperatorTag() == LogicalOperatorTag.DISTRIBUTE_RESULT)))) {
+ return false;
+ }
+
+ // if this join has already been seen before, no need to apply the rule again
+ if (context.checkIfInDontApplySet(this, op)) {
+ return false;
+ }
+
+ List<ILogicalOperator> joinOps = new ArrayList<>();
+ List<ILogicalOperator> internalEdges = new ArrayList<>();
+ HashMap<EmptyTupleSourceOperator, ILogicalOperator> joinLeafInputsHashMap = new HashMap<>();
+ // The data scan operators. Will be in the order of the from clause.
+ // Important for position ordering when assigning bits to join expressions.
+ List<Pair<EmptyTupleSourceOperator, DataSourceScanOperator>> emptyTupleAndDataSourceOps = new ArrayList<>();
+ HashMap<DataSourceScanOperator, EmptyTupleSourceOperator> dataSourceEmptyTupleHashMap = new HashMap<>();
+
+ IPlanPrettyPrinter pp = context.getPrettyPrinter();
+ printPlan(pp, (AbstractLogicalOperator) op, "Original Whole plan1");
+ boolean canTransform = getJoinOpsAndLeafInputs(op, emptyTupleAndDataSourceOps, joinLeafInputsHashMap,
+ dataSourceEmptyTupleHashMap, internalEdges, joinOps);
+
+ if (!canTransform) {
+ return false;
+ }
+
+ // if this happens, something in the input plan is not acceptable to the new code.
+ if (emptyTupleAndDataSourceOps.size() != joinLeafInputsHashMap.size()) {
+ throw new IllegalStateException(
+ "ETS " + emptyTupleAndDataSourceOps.size() + " != LI " + joinLeafInputsHashMap.size());
+ }
+
+ printPlan(pp, (AbstractLogicalOperator) op, "Original Whole plan2");
+
+ int numberOfFromTerms = emptyTupleAndDataSourceOps.size();
+
+ joinEnum.initEnum((AbstractLogicalOperator) op, cboMode, cboTestMode, numberOfFromTerms,
+ emptyTupleAndDataSourceOps, joinLeafInputsHashMap, dataSourceEmptyTupleHashMap, internalEdges, joinOps,
+ context);
+
+ if (cboMode) {
+ if (!doAllDataSourcesHaveSamples(emptyTupleAndDataSourceOps, context)) {
+ return false;
+ }
+ }
+
+ if (internalEdges.size() > 0) {
+ pushAssignsIntoLeafInputs(joinLeafInputsHashMap, internalEdges);
+ }
+
+ int cheapestPlan = joinEnum.enumerateJoins(); // MAIN CALL INTO CBO
+ if (cheapestPlan == PlanNode.NO_PLAN) {
+ return false;
+ }
+
+ PlanNode cheapestPlanNode = joinEnum.allPlans.get(cheapestPlan);
+
+ if (numberOfFromTerms > 1) {
+ buildNewTree(cheapestPlanNode, joinLeafInputsHashMap, joinOps, new MutableInt(0), context);
+ printPlan(pp, (AbstractLogicalOperator) joinOps.get(0), "New Whole Plan after buildNewTree 1");
+ ILogicalOperator root = addConstantInternalEdgesAtTheTop(joinOps.get(0), internalEdges);
+ printPlan(pp, (AbstractLogicalOperator) joinOps.get(0), "New Whole Plan after buildNewTree 2");
+ printPlan(pp, (AbstractLogicalOperator) root, "New Whole Plan after buildNewTree");
+ // this will be the new root
+ opRef.setValue(root);
+ if (LOGGER.isTraceEnabled()) {
+ LOGGER.trace("---------------------------- Printing Leaf Inputs");
+ printLeafPlans(pp, joinLeafInputsHashMap);
+ // print joins starting from the bottom
+ for (int i = joinOps.size() - 1; i >= 0; i--) {
+ printPlan(pp, (AbstractLogicalOperator) joinOps.get(i), "join " + i);
+ }
+ printPlan(pp, (AbstractLogicalOperator) joinOps.get(0), "New Whole Plan");
+ printPlan(pp, (AbstractLogicalOperator) root, "New Whole Plan");
+ }
+
+ // turn of this rule for all joins in this set (subtree)
+ for (ILogicalOperator joinOp : joinOps) {
+ context.addToDontApplySet(this, joinOp);
+ }
+
+ } else {
+ buildNewTree(cheapestPlanNode, joinLeafInputsHashMap);
+ }
+
+ return true;
+ }
+
+ private boolean getCBOMode(IOptimizationContext context) {
+ PhysicalOptimizationConfig physOptConfig = context.getPhysicalOptimizationConfig();
+ return physOptConfig.getCBOMode();
+ }
+
+ private boolean getCBOTestMode(IOptimizationContext context) {
+ PhysicalOptimizationConfig physOptConfig = context.getPhysicalOptimizationConfig();
+ return physOptConfig.getCBOTestMode();
+ }
+
+ /**
+ * Should not see any kind of joins here. store the emptyTupeSourceOp and DataSource operators.
+ * Each leaf input will normally have both, but sometimes only emptyTupeSourceOp will be present (in lists)
+ */
+ private Pair<EmptyTupleSourceOperator, DataSourceScanOperator> containsLeafInputOnly(ILogicalOperator op) {
+ DataSourceScanOperator dataSourceOp = null;
+ ILogicalOperator currentOp = op;
+ while (currentOp.getInputs().size() == 1) {
+ if (currentOp.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
+ // we should not see two data scans in the same path
+ if (dataSourceOp != null) {
+ return null;
+ }
+ dataSourceOp = (DataSourceScanOperator) currentOp;
+ }
+ currentOp = currentOp.getInputs().get(0).getValue();
+ }
+ if (currentOp.getOperatorTag() == LogicalOperatorTag.EMPTYTUPLESOURCE) {
+ return new Pair<>((EmptyTupleSourceOperator) currentOp, dataSourceOp);
+ }
+ return null;
+ }
+
+ /**
+ * Check to see if there is only one assign here and nothing below that other than a join.
+ * have not seen cases where there is more than one assign in a leafinput.
+ */
+ private boolean onlyOneAssign(ILogicalOperator nextOp) {
+ if (nextOp.getOperatorTag() != LogicalOperatorTag.ASSIGN) {
+ return false;
+ }
+ List<Mutable<ILogicalOperator>> nextOpInputs = nextOp.getInputs();
+ return nextOpInputs.get(0).getValue().getOperatorTag() == LogicalOperatorTag.INNERJOIN;
+ }
+
+ private ILogicalOperator findSelectOrDataScan(ILogicalOperator op) {
+ LogicalOperatorTag tag;
+ while (true) {
+ if (op.getInputs().size() > 1) {
+ return null; // Assuming only a linear plan for single table queries (as leafInputs are linear).
+ }
+ tag = op.getOperatorTag();
+ if (tag == LogicalOperatorTag.EMPTYTUPLESOURCE) {
+ return null; // if this happens, there is nothing we can do in CBO code since there is no datasourcescan
+ }
+ if ((tag == LogicalOperatorTag.SELECT) || (tag == LogicalOperatorTag.DATASOURCESCAN)) {
+ return op;
+ }
+
+ op = op.getInputs().get(0).getValue();
+ }
+ }
+
+ /**
+ * This is the main routine that stores all the join operators and the leafInputs. We will later reuse the same
+ * join operators but switch the leafInputs (see buildNewTree). The whole scheme is based on the assumption that the
+ * leafInputs can be switched. The various data structures make the leafInputs accessible efficiently.
+ */
+ private boolean getJoinOpsAndLeafInputs(ILogicalOperator op,
+ List<Pair<EmptyTupleSourceOperator, DataSourceScanOperator>> emptyTupleAndDataSourceOps,
+ HashMap<EmptyTupleSourceOperator, ILogicalOperator> joinLeafInputsHashMap,
+ HashMap<DataSourceScanOperator, EmptyTupleSourceOperator> dataSourceEmptyTupleHashMap,
+ List<ILogicalOperator> internalEdges, List<ILogicalOperator> joinOps) {
+
+ if (op.getOperatorTag() == LogicalOperatorTag.LEFTOUTERJOIN) {
+ return false;
+ }
+
+ if (op.getOperatorTag() == LogicalOperatorTag.INNERJOIN) {
+ joinOps.add(op);
+ for (int i = 0; i < 2; i++) {
+ ILogicalOperator nextOp = op.getInputs().get(i).getValue();
+ boolean canTransform = getJoinOpsAndLeafInputs(nextOp, emptyTupleAndDataSourceOps,
+ joinLeafInputsHashMap, dataSourceEmptyTupleHashMap, internalEdges, joinOps);
+ if (!canTransform) {
+ return false;
+ }
+ }
+ } else {
+ Pair<EmptyTupleSourceOperator, DataSourceScanOperator> etsDataSource = containsLeafInputOnly(op);
+ if (etsDataSource != null) { // a leaf input
+ EmptyTupleSourceOperator etsOp = etsDataSource.first;
+ DataSourceScanOperator dataSourceOp = etsDataSource.second;
+ emptyTupleAndDataSourceOps.add(new Pair<>(etsOp, dataSourceOp));
+ if (op.getOperatorTag().equals(LogicalOperatorTag.DISTRIBUTE_RESULT)) {// single table query
+ ILogicalOperator selectOp = findSelectOrDataScan(op);
+ if (selectOp == null) {
+ return false;
+ } else {
+ joinLeafInputsHashMap.put(etsOp, selectOp);
+ }
+ } else {
+ joinLeafInputsHashMap.put(etsOp, op);
+ }
+ dataSourceEmptyTupleHashMap.put(dataSourceOp, etsOp);
+ } else { // This must be an internal edge
+ if (onlyOneAssign(op)) {
+ // currently, will handle only assign statement and nothing else in an internal Edge.
+ // we can lift this restriction later if the need arises. This just makes some code easier.
+ internalEdges.add(op);
+ boolean canTransform =
+ getJoinOpsAndLeafInputs(op.getInputs().get(0).getValue(), emptyTupleAndDataSourceOps,
+ joinLeafInputsHashMap, dataSourceEmptyTupleHashMap, internalEdges, joinOps);
+ if (!canTransform) {
+ return false;
+ }
+
+ //internalEdges.add(op); // better to store the parent; do this soon.
+ } else {
+ return false;
+ }
+ }
+ }
+
+ return true;
+ }
+
+ private void addCardCostAnnotations(ILogicalOperator op, PlanNode plan) {
+ op.getAnnotations().put(OperatorAnnotations.OP_OUTPUT_CARDINALITY,
+ (double) Math.round(plan.getJoinNode().getCardinality() * 100) / 100);
+ op.getAnnotations().put(OperatorAnnotations.OP_COST_TOTAL,
+ (double) Math.round(plan.computeTotalCost() * 100) / 100);
+ if (plan.IsScanNode()) {
+ op.getAnnotations().put(OperatorAnnotations.OP_INPUT_CARDINALITY,
+ (double) Math.round(plan.getJoinNode().getOrigCardinality() * 100) / 100);
+ op.getAnnotations().put(OperatorAnnotations.OP_COST_LOCAL,
+ (double) Math.round(plan.computeOpCost() * 100) / 100);
+ } else {
+ op.getAnnotations().put(OperatorAnnotations.OP_LEFT_EXCHANGE_COST,
+ (double) Math.round(plan.getLeftExchangeCost().computeTotalCost() * 100) / 100);
+ op.getAnnotations().put(OperatorAnnotations.OP_RIGHT_EXCHANGE_COST,
+ (double) Math.round(plan.getRightExchangeCost().computeTotalCost() * 100) / 100);
+ op.getAnnotations().put(OperatorAnnotations.OP_COST_LOCAL,
+ (double) Math.round((plan.computeOpCost()) * 100) / 100);
+ }
+
+ if (op.getOperatorTag().equals(LogicalOperatorTag.SELECT)) {
+ op.getAnnotations().put(OperatorAnnotations.OP_COST_LOCAL, 0.0);
+ }
+ }
+
+ /**
+ * Finds the DataSourceScanOperator given a leafInput
+ */
+ private ILogicalOperator findDataSourceScanOperator(ILogicalOperator op) {
+ ILogicalOperator origOp = op;
+ while (op != null && op.getOperatorTag() != LogicalOperatorTag.EMPTYTUPLESOURCE) {
+ if (op.getOperatorTag().equals(LogicalOperatorTag.DATASOURCESCAN)) {
+ return op;
+ }
+ op = op.getInputs().get(0).getValue();
+ }
+ return origOp;
+ }
+
+ private void removeJoinAnnotations(AbstractFunctionCallExpression afcExpr) {
+ afcExpr.removeAnnotation(BroadcastExpressionAnnotation.class);
+ afcExpr.removeAnnotation(IndexedNLJoinExpressionAnnotation.class);
+ afcExpr.removeAnnotation(HashJoinExpressionAnnotation.class);
+ }
+
+ private void setAnnotation(AbstractFunctionCallExpression afcExpr, IExpressionAnnotation anno) {
+ FunctionIdentifier fi = afcExpr.getFunctionIdentifier();
+ List<Mutable<ILogicalExpression>> arguments = afcExpr.getArguments();
+
+ if (fi.equals(AlgebricksBuiltinFunctions.AND)) {
+ for (Mutable<ILogicalExpression> iLogicalExpressionMutable : arguments) {
+ ILogicalExpression argument = iLogicalExpressionMutable.getValue();
+ AbstractFunctionCallExpression expr = (AbstractFunctionCallExpression) argument;
+ expr.putAnnotation(anno);
+ }
+ } else {
+ afcExpr.putAnnotation(anno);
+ }
+ }
+
+ //Internal edges are assign statements. The RHS has a variable in it.
+ // We need to find the internal edge that has a variable coming from this leaf leafInput.
+ private int findInternalEdge(ILogicalOperator leafInput, List<ILogicalOperator> internalEdges)
+ throws AlgebricksException {
+ int i = -1;
+
+ for (ILogicalOperator ie : internalEdges) {
+ i++;
+ // this will be an Assign, so no need to check
+ AssignOperator aOp = (AssignOperator) ie;
+ List<LogicalVariable> vars = new ArrayList<>();
+ aOp.getExpressions().get(0).getValue().getUsedVariables(vars);
+ HashSet<LogicalVariable> vars2 = new HashSet<>();
+ VariableUtilities.getLiveVariables(leafInput, vars2);
+ if (vars2.containsAll(vars)) { // note that this will fail if there variables from different leafInputs
+ return i;
+ }
+ }
+
+ return -1;
+ }
+
+ private ILogicalOperator addAssignToLeafInput(ILogicalOperator leafInput, ILogicalOperator internalEdge) {
+ ILogicalOperator root = leafInput;
+ // this will be an Assign, so no need to check
+ AssignOperator aOp = (AssignOperator) internalEdge;
+ aOp.getInputs().get(0).setValue(root);
+ return aOp;
+ }
+
+ private void skipAllIndexes(PlanNode plan, ILogicalOperator leafInput) {
+ if (plan.scanOp == PlanNode.ScanMethod.TABLE_SCAN && leafInput.getOperatorTag() == LogicalOperatorTag.SELECT) {
+ SelectOperator selOper = (SelectOperator) leafInput;
+ ILogicalExpression expr = selOper.getCondition().getValue();
+
+ List<Mutable<ILogicalExpression>> conjs = new ArrayList<>();
+
+ conjs.clear();
+ if (expr.splitIntoConjuncts(conjs)) {
+ conjs.remove(new MutableObject<ILogicalExpression>(ConstantExpression.TRUE));
+ for (Mutable<ILogicalExpression> conj : conjs) {
+ if (conj.getValue().getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL)) {
+ AbstractFunctionCallExpression afce = (AbstractFunctionCallExpression) conj.getValue();
+ // remove any annotations that may have been here from other parts of the code. We know we want a datascan.
+ afce.removeAnnotation(SkipSecondaryIndexSearchExpressionAnnotation.class);
+ afce.putAnnotation(SkipSecondaryIndexSearchExpressionAnnotation.INSTANCE_ANY_INDEX);
+ }
+ }
+ } else {
+ if ((expr.getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL))) {
+ AbstractFunctionCallExpression afce = (AbstractFunctionCallExpression) expr;
+ // remove any annotations that may have been here from other parts of the code. We know we want a datascan.
+ afce.removeAnnotation(SkipSecondaryIndexSearchExpressionAnnotation.class);
+ afce.putAnnotation(SkipSecondaryIndexSearchExpressionAnnotation.INSTANCE_ANY_INDEX);
+ }
+ }
+ }
+ }
+
+ // This is for single table queries
+ private void buildNewTree(PlanNode plan,
+ HashMap<EmptyTupleSourceOperator, ILogicalOperator> joinLeafInputsHashMap) {
+ ILogicalOperator leftInput = joinLeafInputsHashMap.get(plan.getEmptyTupleSourceOp());
+ skipAllIndexes(plan, leftInput);
+ ILogicalOperator selOp = findSelectOrDataScan(leftInput);
+ if (selOp != null) {
+ addCardCostAnnotations(selOp, plan);
+ }
+ addCardCostAnnotations(findDataSourceScanOperator(leftInput), plan);
+ }
+
+ // This one is for join queries
+ private void buildNewTree(PlanNode plan, HashMap<EmptyTupleSourceOperator, ILogicalOperator> joinLeafInputsHashMap,
+ List<ILogicalOperator> joinOps, MutableInt totalNumberOfJoins, IOptimizationContext context)
+ throws AlgebricksException {
+ // we have to move the inputs in op around so that they match the tree structure in pn
+ // we use the existing joinOps and switch the leafInputs appropriately.
+ List<PlanNode> allPlans = joinEnum.getAllPlans();
+ int leftIndex = plan.getLeftPlanIndex();
+ int rightIndex = plan.getRightPlanIndex();
+ PlanNode leftPlan = allPlans.get(leftIndex);
+ PlanNode rightPlan = allPlans.get(rightIndex);
+ ILogicalOperator joinOp = joinOps.get(totalNumberOfJoins.intValue());
+
+ if (plan.IsJoinNode()) {
+ AbstractBinaryJoinOperator abJoinOp = (AbstractBinaryJoinOperator) joinOp;
+ ILogicalExpression expr = plan.getJoinExpr();
+ abJoinOp.getCondition().setValue(expr);
+ // add the annotations
+ if (plan.getJoinOp() == PlanNode.JoinMethod.INDEX_NESTED_LOOP_JOIN) {
+ // this annotation is needed for the physical optimizer to replace this with the unnest operator later
+ AbstractFunctionCallExpression afcExpr = (AbstractFunctionCallExpression) expr;
+ removeJoinAnnotations(afcExpr);
+ setAnnotation(afcExpr, IndexedNLJoinExpressionAnnotation.INSTANCE_ANY_INDEX);
+ } else if (plan.getJoinOp() == PlanNode.JoinMethod.HYBRID_HASH_JOIN
+ || plan.getJoinOp() == PlanNode.JoinMethod.BROADCAST_HASH_JOIN
+ || plan.getJoinOp() == PlanNode.JoinMethod.CARTESIAN_PRODUCT_JOIN) {
+ if (plan.getJoinOp() == PlanNode.JoinMethod.BROADCAST_HASH_JOIN) {
+ // Broadcast the right branch.
+ BroadcastExpressionAnnotation bcast =
+ new BroadcastExpressionAnnotation(plan.side == HashJoinExpressionAnnotation.BuildSide.RIGHT
+ ? BroadcastExpressionAnnotation.BroadcastSide.RIGHT
+ : BroadcastExpressionAnnotation.BroadcastSide.LEFT);
+ AbstractFunctionCallExpression afcExpr = (AbstractFunctionCallExpression) expr;
+ removeJoinAnnotations(afcExpr);
+ setAnnotation(afcExpr, bcast);
+ } else if (plan.getJoinOp() == PlanNode.JoinMethod.HYBRID_HASH_JOIN) {
+ HashJoinExpressionAnnotation hjAnnotation = new HashJoinExpressionAnnotation(plan.side);
+ AbstractFunctionCallExpression afcExpr = (AbstractFunctionCallExpression) expr;
+ removeJoinAnnotations(afcExpr);
+ setAnnotation(afcExpr, hjAnnotation);
+ } else {
+ if (expr != ConstantExpression.TRUE) {
+ AbstractFunctionCallExpression afcExpr = (AbstractFunctionCallExpression) expr;
+ removeJoinAnnotations(afcExpr);
+ }
+ }
+ }
+ addCardCostAnnotations(joinOp, plan);
+ }
+
+ if (leftPlan.IsScanNode()) {
+ // leaf
+ ILogicalOperator leftInput = joinLeafInputsHashMap.get(leftPlan.getEmptyTupleSourceOp());
+ skipAllIndexes(leftPlan, leftInput);
+ ILogicalOperator selOp = findSelectOrDataScan(leftInput);
+ if (selOp != null) {
+ addCardCostAnnotations(selOp, leftPlan);
+ }
+ joinOp.getInputs().get(0).setValue(leftInput);
+ context.computeAndSetTypeEnvironmentForOperator(joinOp.getInputs().get(0).getValue());
+ addCardCostAnnotations(findDataSourceScanOperator(leftInput), leftPlan);
+ } else {
+ // join
+ totalNumberOfJoins.increment();
+ ILogicalOperator leftInput = joinOps.get(totalNumberOfJoins.intValue());
+ joinOp.getInputs().get(0).setValue(leftInput);
+ context.computeAndSetTypeEnvironmentForOperator(joinOp.getInputs().get(0).getValue());
+ buildNewTree(allPlans.get(leftIndex), joinLeafInputsHashMap, joinOps, totalNumberOfJoins, context);
+ }
+
+ if (rightPlan.IsScanNode()) {
+ // leaf
+ ILogicalOperator rightInput = joinLeafInputsHashMap.get(rightPlan.getEmptyTupleSourceOp());
+ skipAllIndexes(rightPlan, rightInput);
+ ILogicalOperator selOp = findSelectOrDataScan(rightInput);
+ if (selOp != null) {
+ addCardCostAnnotations(selOp, rightPlan);
+ }
+ joinOp.getInputs().get(1).setValue(rightInput);
+ context.computeAndSetTypeEnvironmentForOperator(joinOp.getInputs().get(1).getValue());
+ addCardCostAnnotations(findDataSourceScanOperator(rightInput), rightPlan);
+ } else {
+ // join
+ totalNumberOfJoins.increment();
+ ILogicalOperator rightInput = joinOps.get(totalNumberOfJoins.intValue());
+ joinOp.getInputs().get(1).setValue(rightInput);
+ context.computeAndSetTypeEnvironmentForOperator(joinOp.getInputs().get(1).getValue());
+ buildNewTree(allPlans.get(rightIndex), joinLeafInputsHashMap, joinOps, totalNumberOfJoins, context);
+ }
+ }
+
+ // in some very rare cases, there is an internal edge that has an assign statement such as $$var = 20 but this variable
+ // is not used anywhere in the current join graph but is used outside the current join graph. So we add this assign to the top of
+ // our plan, so the rest of the code will be happy. Strange that this assign appears in the join graph.
+ private ILogicalOperator addConstantInternalEdgesAtTheTop(ILogicalOperator op,
+ List<ILogicalOperator> internalEdges) {
+ if (internalEdges.size() == 0) {
+ return op;
+ }
+ ILogicalOperator root = op;
+ for (ILogicalOperator ie : internalEdges) {
+ // this will be an Assign, so no need to check
+ AssignOperator aOp = (AssignOperator) ie;
+ aOp.getInputs().get(0).setValue(root);
+ root = aOp;
+ }
+ return root;
+ }
+
+ public static void printPlan(IPlanPrettyPrinter pp, AbstractLogicalOperator op, String text)
+ throws AlgebricksException {
+ if (LOGGER.isTraceEnabled()) {
+ pp.reset();
+ pp.printOperator(op, true, false);
+ LOGGER.trace("---------------------------- {}\n{}\n----------------------------", text, pp);
+ }
+ }
+
+ private void printLeafPlans(IPlanPrettyPrinter pp,
+ HashMap<EmptyTupleSourceOperator, ILogicalOperator> joinLeafInputsHashMap) throws AlgebricksException {
+ Iterator<Map.Entry<EmptyTupleSourceOperator, ILogicalOperator>> li =
+ joinLeafInputsHashMap.entrySet().iterator();
+ int i = 0;
+ while (li.hasNext()) {
+ Map.Entry<EmptyTupleSourceOperator, ILogicalOperator> pair = li.next();
+ ILogicalOperator element = pair.getValue();
+ printPlan(pp, (AbstractLogicalOperator) element, "Printing Leaf Input" + i);
+ i++;
+ }
+ }
+
+ // for every internal edge assign (again assuming only 1 for now), find the corresponding leafInput and place the assign
+ // on top of that LeafInput. Modify the joinLeafInputsHashMap as well.
+ private void pushAssignsIntoLeafInputs(HashMap<EmptyTupleSourceOperator, ILogicalOperator> joinLeafInputsHashMap,
+ List<ILogicalOperator> internalEdges) throws AlgebricksException {
+
+ for (Map.Entry<EmptyTupleSourceOperator, ILogicalOperator> mapElement : joinLeafInputsHashMap.entrySet()) {
+ ILogicalOperator joinLeafInput = mapElement.getValue();
+ EmptyTupleSourceOperator ets = mapElement.getKey();
+ int internalEdgeNumber = findInternalEdge(joinLeafInput, internalEdges);
+ if (internalEdgeNumber != -1) {
+ joinLeafInput = addAssignToLeafInput(joinLeafInput, internalEdges.get(internalEdgeNumber));
+ joinLeafInputsHashMap.put(ets, joinLeafInput);
+ internalEdges.remove(internalEdgeNumber); // no longer needed
+ }
+ }
+
+ }
+
+ private boolean substituteVarOnce(ILogicalExpression exp, LogicalVariable oldVar, LogicalVariable newVar) {
+ switch (exp.getExpressionTag()) {
+ case FUNCTION_CALL:
+ AbstractFunctionCallExpression fun = (AbstractFunctionCallExpression) exp;
+ for (int i = 0; i < fun.getArguments().size(); i++) {
+ ILogicalExpression arg = fun.getArguments().get(i).getValue();
+ if (substituteVarOnce(arg, oldVar, newVar)) {
+ return true;
+ }
+ }
+ return false;
+ case VARIABLE:
+ VariableReferenceExpression varExpr = (VariableReferenceExpression) exp;
+ if (varExpr.getVariableReference().equals(oldVar)) {
+ varExpr.setVariable(newVar);
+ return true;
+ }
+ return false;
+ default:
+ return false;
+ }
+ }
+
+ // check to see if every dataset has a sample. If not, CBO code cannot run. A warning message must be issued as well.
+ private boolean doAllDataSourcesHaveSamples(
+ List<Pair<EmptyTupleSourceOperator, DataSourceScanOperator>> emptyTupleAndDataSourceOps,
+ IOptimizationContext context) throws AlgebricksException {
+ for (Pair<EmptyTupleSourceOperator, DataSourceScanOperator> emptyTupleAndDataSourceOp : emptyTupleAndDataSourceOps) {
+ if (emptyTupleAndDataSourceOp.getSecond() != null) {
+ DataSourceScanOperator scanOp = emptyTupleAndDataSourceOp.getSecond();
+ Index index = joinEnum.getStatsHandle().findSampleIndex(scanOp, context);
+ if (index == null) {
+ return false;
+ }
+ }
+ }
+ return true;
+ }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/EstimatedCostComputationVisitor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/EstimatedCostComputationVisitor.java
new file mode 100644
index 0000000..3943cf4
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/EstimatedCostComputationVisitor.java
@@ -0,0 +1,362 @@
+/*
+ * 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.optimizer.rules.cbo;
+
+import java.util.Map;
+
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.base.OperatorAnnotations;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DelegateOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ForwardOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteUpsertOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteUpsertOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.IntersectOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterUnnestMapOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterUnnestOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.LimitOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.MaterializeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.RunningAggregateOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SinkOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SplitOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.TokenizeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.WindowOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteResultOperator;
+import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+
+/**
+ * A visitor that annotates an operator with its estimated cardinality and estimated cost.
+ */
+public class EstimatedCostComputationVisitor implements ILogicalOperatorVisitor<Pair<Double, Double>, Double> {
+
+ public EstimatedCostComputationVisitor() {
+ }
+
+ @Override
+ public Pair<Double, Double> visitAggregateOperator(AggregateOperator op, Double arg) throws AlgebricksException {
+ return annotate(this, op, arg);
+ }
+
+ @Override
+ public Pair<Double, Double> visitRunningAggregateOperator(RunningAggregateOperator op, Double arg)
+ throws AlgebricksException {
+ return annotate(this, op, arg);
+ }
+
+ @Override
+ public Pair<Double, Double> visitEmptyTupleSourceOperator(EmptyTupleSourceOperator op, Double arg)
+ throws AlgebricksException {
+ // Empty tuple source operator sends an empty tuple to downstream operators.
+ return new Pair<>(0.0, 0.0);
+ }
+
+ @Override
+ public Pair<Double, Double> visitGroupByOperator(GroupByOperator op, Double arg) throws AlgebricksException {
+ // Needs more work in the cardinality estimation code to estimate group by cardinality and cost.
+ return annotate(this, op, arg);
+ }
+
+ @Override
+ public Pair<Double, Double> visitLimitOperator(LimitOperator op, Double arg) throws AlgebricksException {
+ return annotate(this, op, arg);
+ }
+
+ @Override
+ public Pair<Double, Double> visitInnerJoinOperator(InnerJoinOperator op, Double arg) throws AlgebricksException {
+ return visitInnerJoin(op, arg);
+ }
+
+ @Override
+ public Pair<Double, Double> visitLeftOuterJoinOperator(LeftOuterJoinOperator op, Double arg)
+ throws AlgebricksException {
+ return visitLeftOuterUnnest(op, arg);
+ }
+
+ @Override
+ public Pair<Double, Double> visitNestedTupleSourceOperator(NestedTupleSourceOperator op, Double arg)
+ throws AlgebricksException {
+ Pair<Double, Double> cardCost = annotate(this, op, arg);
+ return op.getDataSourceReference().getValue().getOperatorTag() == LogicalOperatorTag.SUBPLAN ? cardCost
+ : new Pair<>(0.0, 0.0);
+ }
+
+ @Override
+ public Pair<Double, Double> visitOrderOperator(OrderOperator op, Double arg) throws AlgebricksException {
+ return annotate(this, op, arg);
+ }
+
+ @Override
+ public Pair<Double, Double> visitAssignOperator(AssignOperator op, Double arg) throws AlgebricksException {
+ return annotate(this, op, arg);
+ }
+
+ @Override
+ public Pair<Double, Double> visitWindowOperator(WindowOperator op, Double arg) throws AlgebricksException {
+ return annotate(this, op, arg);
+ }
+
+ @Override
+ public Pair<Double, Double> visitSelectOperator(SelectOperator op, Double arg) throws AlgebricksException {
+ Pair<Double, Double> cardCost = op.getInputs().get(0).getValue().accept(this, arg);
+ for (Map.Entry<String, Object> anno : op.getAnnotations().entrySet()) {
+ if (anno.getValue() != null && anno.getKey().equals(OperatorAnnotations.OP_OUTPUT_CARDINALITY)) {
+ cardCost.setFirst((Double) anno.getValue());
+ } else if (anno.getValue() != null && anno.getKey().equals(OperatorAnnotations.OP_COST_TOTAL)) {
+ cardCost.setSecond((Double) anno.getValue());
+ }
+ }
+
+ return cardCost;
+ }
+
+ @Override
+ public Pair<Double, Double> visitDelegateOperator(DelegateOperator op, Double arg) throws AlgebricksException {
+ return annotate(this, op, arg);
+ }
+
+ @Override
+ public Pair<Double, Double> visitProjectOperator(ProjectOperator op, Double arg) throws AlgebricksException {
+ return annotate(this, op, arg);
+ }
+
+ @Override
+ public Pair<Double, Double> visitReplicateOperator(ReplicateOperator op, Double arg) throws AlgebricksException {
+ return annotate(this, op, arg);
+ }
+
+ @Override
+ public Pair<Double, Double> visitSplitOperator(SplitOperator op, Double arg) throws AlgebricksException {
+ return annotate(this, op, arg);
+ }
+
+ @Override
+ public Pair<Double, Double> visitMaterializeOperator(MaterializeOperator op, Double arg)
+ throws AlgebricksException {
+ return annotate(this, op, arg);
+ }
+
+ @Override
+ public Pair<Double, Double> visitScriptOperator(ScriptOperator op, Double arg) throws AlgebricksException {
+ return annotate(this, op, arg);
+ }
+
+ @Override
+ public Pair<Double, Double> visitSubplanOperator(SubplanOperator op, Double arg) throws AlgebricksException {
+ return annotate(this, op, arg);
+ }
+
+ @Override
+ public Pair<Double, Double> visitSinkOperator(SinkOperator op, Double arg) throws AlgebricksException {
+ return annotate(this, op, arg);
+ }
+
+ @Override
+ public Pair<Double, Double> visitUnionOperator(UnionAllOperator op, Double arg) throws AlgebricksException {
+ // Needs more work.
+ return annotate(this, op, arg);
+ }
+
+ @Override
+ public Pair<Double, Double> visitUnnestOperator(UnnestOperator op, Double arg) throws AlgebricksException {
+ return annotate(this, op, arg);
+ }
+
+ @Override
+ public Pair<Double, Double> visitLeftOuterUnnestOperator(LeftOuterUnnestOperator op, Double arg)
+ throws AlgebricksException {
+ return visitLeftOuterUnnest(op, arg);
+ }
+
+ @Override
+ public Pair<Double, Double> visitUnnestMapOperator(UnnestMapOperator op, Double arg) throws AlgebricksException {
+ Pair<Double, Double> cardCost = new Pair<>(0.0, 0.0);
+
+ for (Map.Entry<String, Object> anno : op.getAnnotations().entrySet()) {
+ if (anno.getValue() != null && anno.getKey().equals(OperatorAnnotations.OP_OUTPUT_CARDINALITY)) {
+ cardCost.setFirst((Double) anno.getValue());
+ } else if (anno.getValue() != null && anno.getKey().equals(OperatorAnnotations.OP_COST_TOTAL)) {
+ cardCost.setSecond((Double) anno.getValue());
+ }
+ }
+
+ return cardCost;
+ }
+
+ @Override
+ public Pair<Double, Double> visitLeftOuterUnnestMapOperator(LeftOuterUnnestMapOperator op, Double arg)
+ throws AlgebricksException {
+ return visitLeftOuterUnnest(op, arg);
+ }
+
+ @Override
+ public Pair<Double, Double> visitDataScanOperator(DataSourceScanOperator op, Double arg)
+ throws AlgebricksException {
+ Pair<Double, Double> cardCost = new Pair<>(0.0, 0.0);
+
+ for (Map.Entry<String, Object> anno : op.getAnnotations().entrySet()) {
+ if (anno.getValue() != null && anno.getKey().equals(OperatorAnnotations.OP_INPUT_CARDINALITY)) {
+ cardCost.setFirst((Double) anno.getValue());
+ } else if (anno.getValue() != null && anno.getKey().equals(OperatorAnnotations.OP_COST_TOTAL)) {
+ cardCost.setSecond((Double) anno.getValue());
+ }
+ }
+
+ return cardCost;
+ }
+
+ @Override
+ public Pair<Double, Double> visitDistinctOperator(DistinctOperator op, Double arg) throws AlgebricksException {
+ return annotate(this, op, arg);
+ }
+
+ @Override
+ public Pair<Double, Double> visitExchangeOperator(ExchangeOperator op, Double arg) throws AlgebricksException {
+ double exchCost = 0;
+ if (arg != null) {
+ exchCost = arg;
+ }
+
+ Pair<Double, Double> cardCost = op.getInputs().get(0).getValue().accept(this, arg);
+ if (exchCost != 0) {
+ op.getAnnotations().put(OperatorAnnotations.OP_COST_LOCAL, exchCost);
+ op.getAnnotations().put(OperatorAnnotations.OP_COST_TOTAL, exchCost + cardCost.getSecond());
+ } else {
+ op.getAnnotations().put(OperatorAnnotations.OP_COST_LOCAL, 0.0);
+ op.getAnnotations().put(OperatorAnnotations.OP_COST_TOTAL, cardCost.getSecond());
+ }
+ op.getAnnotations().put(OperatorAnnotations.OP_OUTPUT_CARDINALITY, cardCost.getFirst());
+ return cardCost;
+ }
+
+ @Override
+ public Pair<Double, Double> visitWriteOperator(WriteOperator op, Double arg) throws AlgebricksException {
+ return annotate(this, op, arg);
+ }
+
+ @Override
+ public Pair<Double, Double> visitDistributeResultOperator(DistributeResultOperator op, Double arg)
+ throws AlgebricksException {
+ return annotate(this, op, arg);
+ }
+
+ @Override
+ public Pair<Double, Double> visitWriteResultOperator(WriteResultOperator op, Double arg)
+ throws AlgebricksException {
+ return annotate(this, op, arg);
+ }
+
+ @Override
+ public Pair<Double, Double> visitInsertDeleteUpsertOperator(InsertDeleteUpsertOperator op, Double arg)
+ throws AlgebricksException {
+ return annotate(this, op, arg);
+ }
+
+ @Override
+ public Pair<Double, Double> visitIndexInsertDeleteUpsertOperator(IndexInsertDeleteUpsertOperator op, Double arg)
+ throws AlgebricksException {
+ return annotate(this, op, arg);
+ }
+
+ @Override
+ public Pair<Double, Double> visitTokenizeOperator(TokenizeOperator op, Double arg) throws AlgebricksException {
+ return annotate(this, op, arg);
+ }
+
+ @Override
+ public Pair<Double, Double> visitForwardOperator(ForwardOperator op, Double arg) throws AlgebricksException {
+ return annotate(this, op, arg);
+ }
+
+ @Override
+ public Pair<Double, Double> visitIntersectOperator(IntersectOperator op, Double arg) throws AlgebricksException {
+ // Needs more work.
+ return annotate(this, op, arg);
+ }
+
+ private static Pair<Double, Double> annotate(EstimatedCostComputationVisitor visitor, ILogicalOperator op,
+ Double arg) throws AlgebricksException {
+ if (op.getInputs().isEmpty()) {
+ return new Pair<>(0.0, 0.0);
+ }
+ Pair<Double, Double> cardCost = op.getInputs().get(0).getValue().accept(visitor, arg);
+ op.getAnnotations().put(OperatorAnnotations.OP_OUTPUT_CARDINALITY, cardCost.getFirst());
+ op.getAnnotations().put(OperatorAnnotations.OP_COST_TOTAL, cardCost.getSecond());
+ op.getAnnotations().put(OperatorAnnotations.OP_COST_LOCAL, 0.0);
+ return cardCost;
+ }
+
+ // Visits an operator that has the left outer semantics, e.g.,
+ // left outer join, left outer unnest, left outer unnest map.
+ private Pair<Double, Double> visitLeftOuterUnnest(ILogicalOperator operator, Double arg)
+ throws AlgebricksException {
+ // Needs more work.
+ return operator.getInputs().get(0).getValue().accept(this, arg);
+ }
+
+ // Visits an inner join operator.
+ private Pair<Double, Double> visitInnerJoin(InnerJoinOperator joinOperator, Double arg) throws AlgebricksException {
+ Pair<Double, Double> cardCost = new Pair<>(0.0, 0.0);
+
+ ILogicalOperator left = joinOperator.getInputs().get(0).getValue();
+ ILogicalOperator right = joinOperator.getInputs().get(1).getValue();
+ double leftExchangeCost = 0;
+ double rightExchangeCost = 0;
+
+ for (Map.Entry<String, Object> anno : joinOperator.getAnnotations().entrySet()) {
+ if (anno.getValue() != null && anno.getKey().equals(OperatorAnnotations.OP_OUTPUT_CARDINALITY)) {
+ cardCost.setFirst((Double) anno.getValue());
+ } else if (anno.getValue() != null && anno.getKey().equals(OperatorAnnotations.OP_COST_TOTAL)) {
+ cardCost.setSecond((Double) anno.getValue());
+ } else if (anno.getValue() != null && anno.getKey().equals(OperatorAnnotations.OP_LEFT_EXCHANGE_COST)) {
+ leftExchangeCost = (double) anno.getValue();
+ } else if (anno.getValue() != null && anno.getKey().equals(OperatorAnnotations.OP_RIGHT_EXCHANGE_COST)) {
+ rightExchangeCost = (double) anno.getValue();
+ }
+ }
+
+ // Visit the left and right branches.
+ left.accept(this, leftExchangeCost);
+ right.accept(this, rightExchangeCost);
+
+ return cardCost;
+ }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/JoinCondition.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/JoinCondition.java
new file mode 100644
index 0000000..bc7ea3f
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/JoinCondition.java
@@ -0,0 +1,49 @@
+/*
+ * 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.optimizer.rules.cbo;
+
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+
+public class JoinCondition {
+
+ public static final int NO_JC = -1;
+
+ public ILogicalExpression joinCondition;
+ public boolean derived = false;
+ public boolean partOfComposite = false;
+ public int numberOfVars = 0; // how many variables
+ public int componentNumber = 0; // for identifying if join graph is connected
+ public int datasetBits;
+ // used for triangle detection; we strictly do not mean left and right here.
+ // first and second sides would be more appropriate
+ public int leftSideBits;
+ public int rightSideBits;
+ public double selectivity;
+ public comparisonOp comparisonType;
+
+ public enum comparisonOp {
+ OP_EQ,
+ OP_OTHER
+ }
+
+ public ILogicalExpression getJoinCondition() {
+ return joinCondition;
+ }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/JoinEnum.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/JoinEnum.java
new file mode 100644
index 0000000..e7a620d
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/JoinEnum.java
@@ -0,0 +1,809 @@
+/*
+ * 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.optimizer.rules.cbo;
+
+import java.time.LocalDateTime;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.asterix.common.annotations.IndexedNLJoinExpressionAnnotation;
+import org.apache.asterix.common.annotations.SecondaryIndexSearchPreferenceAnnotation;
+import org.apache.asterix.metadata.declared.DataSource;
+import org.apache.asterix.metadata.declared.DataSourceId;
+import org.apache.asterix.metadata.entities.Index;
+import org.apache.asterix.om.base.AOrderedList;
+import org.apache.asterix.om.base.IAObject;
+import org.apache.asterix.om.constants.AsterixConstantValue;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.optimizer.cost.Cost;
+import org.apache.asterix.optimizer.cost.CostMethods;
+import org.apache.asterix.optimizer.cost.ICost;
+import org.apache.asterix.optimizer.cost.ICostMethods;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.BroadcastExpressionAnnotation;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.HashJoinExpressionAnnotation;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.UnnestingFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import org.apache.hyracks.algebricks.core.algebra.prettyprint.IPlanPrettyPrinter;
+import org.apache.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
+import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+public class JoinEnum {
+
+ private static final Logger LOGGER = LogManager.getLogger();
+
+ protected List<JoinCondition> joinConditions; // "global" list of join conditions
+ protected List<PlanNode> allPlans; // list of all plans
+ protected JoinNode[] jnArray; // array of all join nodes
+ protected int jnArraySize;
+ protected List<Pair<EmptyTupleSourceOperator, DataSourceScanOperator>> emptyTupleAndDataSourceOps;
+ protected Map<EmptyTupleSourceOperator, ILogicalOperator> joinLeafInputsHashMap;
+ protected Map<DataSourceScanOperator, EmptyTupleSourceOperator> dataSourceEmptyTupleHashMap;
+ protected List<ILogicalExpression> singleDatasetPreds;
+ protected List<ILogicalOperator> internalEdges;
+ protected List<ILogicalOperator> joinOps;
+ protected ILogicalOperator localJoinOp; // used in nestedLoopsApplicable code.
+ protected IOptimizationContext optCtx;
+ protected Stats stats;
+ protected PhysicalOptimizationConfig physOptConfig;
+ protected boolean cboMode;
+ protected boolean cboTestMode;
+ protected int numberOfTerms;
+ protected AbstractLogicalOperator op;
+ protected boolean connectedJoinGraph;
+ protected boolean forceJoinOrderMode;
+ protected String queryPlanShape;
+ protected ICost cost;
+ protected ICostMethods costMethods;
+
+ public JoinEnum() {
+ }
+
+ public void initEnum(AbstractLogicalOperator op, boolean cboMode, boolean cboTestMode, int numberOfFromTerms,
+ List<Pair<EmptyTupleSourceOperator, DataSourceScanOperator>> emptyTupleAndDataSourceOps,
+ Map<EmptyTupleSourceOperator, ILogicalOperator> joinLeafInputsHashMap,
+ Map<DataSourceScanOperator, EmptyTupleSourceOperator> dataSourceEmptyTupleHashMap,
+ List<ILogicalOperator> internalEdges, List<ILogicalOperator> joinOps, IOptimizationContext context) {
+ this.singleDatasetPreds = new ArrayList<>();
+ this.joinConditions = new ArrayList<>();
+ this.internalEdges = new ArrayList<>();
+ this.allPlans = new ArrayList<>();
+ this.numberOfTerms = numberOfFromTerms;
+ this.cboMode = cboMode;
+ this.cboTestMode = cboTestMode;
+ this.connectedJoinGraph = true;
+ this.optCtx = context;
+ this.physOptConfig = context.getPhysicalOptimizationConfig();
+ this.emptyTupleAndDataSourceOps = emptyTupleAndDataSourceOps;
+ this.joinLeafInputsHashMap = joinLeafInputsHashMap;
+ this.dataSourceEmptyTupleHashMap = dataSourceEmptyTupleHashMap;
+ this.internalEdges = internalEdges;
+ this.joinOps = joinOps;
+ this.op = op;
+ this.forceJoinOrderMode = getForceJoinOrderMode(context);
+ this.queryPlanShape = getQueryPlanShape(context);
+ initCostHandleAndJoinNodes(context);
+ }
+
+ protected void initCostHandleAndJoinNodes(IOptimizationContext context) {
+ this.cost = new Cost();
+ this.costMethods = new CostMethods(context);
+ this.stats = new Stats(optCtx, this);
+ this.jnArraySize = (int) Math.pow(2.0, this.numberOfTerms);
+ this.jnArray = new JoinNode[this.jnArraySize];
+ // initialize all the join nodes
+ for (int i = 0; i < this.jnArraySize; i++) {
+ this.jnArray[i] = new JoinNode(i, this);
+ }
+ }
+
+ public List<JoinCondition> getJoinConditions() {
+ return joinConditions;
+ }
+
+ public List<PlanNode> getAllPlans() {
+ return allPlans;
+ }
+
+ public JoinNode[] getJnArray() {
+ return jnArray;
+ }
+
+ public Cost getCostHandle() {
+ return (Cost) cost;
+ }
+
+ public CostMethods getCostMethodsHandle() {
+ return (CostMethods) costMethods;
+ }
+
+ public Stats getStatsHandle() {
+ return stats;
+ }
+
+ public Map<EmptyTupleSourceOperator, ILogicalOperator> getJoinLeafInputsHashMap() {
+ return joinLeafInputsHashMap;
+ }
+
+ public Map<DataSourceScanOperator, EmptyTupleSourceOperator> getDataSourceEmptyTupleHashMap() {
+ return dataSourceEmptyTupleHashMap;
+ }
+
+ public ILogicalOperator findLeafInput(List<LogicalVariable> logicalVars) throws AlgebricksException {
+ Set<LogicalVariable> vars = new HashSet<>();
+ for (Pair<EmptyTupleSourceOperator, DataSourceScanOperator> emptyTupleAndDataSourceOp : emptyTupleAndDataSourceOps) {
+ EmptyTupleSourceOperator emptyOp = emptyTupleAndDataSourceOp.getFirst();
+ ILogicalOperator op = joinLeafInputsHashMap.get(emptyOp);
+ vars.clear();
+ // this is expensive to do. So store this once and reuse
+ VariableUtilities.getLiveVariables(op, vars);
+ if (vars.containsAll(logicalVars)) {
+ return op;
+ }
+ }
+ // this will never happen, but keep compiler happy
+ return null;
+ }
+
+ public ILogicalExpression combineAllConditions(List<Integer> newJoinConditions) {
+ if (newJoinConditions.size() == 0) {
+ // this is a cartesian product
+ return ConstantExpression.TRUE;
+ }
+ if (newJoinConditions.size() == 1) {
+ JoinCondition jc = joinConditions.get(newJoinConditions.get(0));
+ return jc.joinCondition;
+ }
+ ScalarFunctionCallExpression andExpr = new ScalarFunctionCallExpression(
+ BuiltinFunctions.getBuiltinFunctionInfo(AlgebricksBuiltinFunctions.AND));
+
+ for (int joinNum : newJoinConditions) {
+ // Need to AND all the expressions.
+ JoinCondition jc = joinConditions.get(joinNum);
+ andExpr.getArguments().add(new MutableObject<>(jc.joinCondition));
+ }
+ return andExpr;
+ }
+
+ public ILogicalExpression getNestedLoopJoinExpr(List<Integer> newJoinConditions) {
+ if (newJoinConditions.size() != 1) {
+ // may remove this restriction later if possible
+ return null;
+ }
+ JoinCondition jc = joinConditions.get(newJoinConditions.get(0));
+ return jc.joinCondition;
+ }
+
+ public ILogicalExpression getHashJoinExpr(List<Integer> newJoinConditions) {
+ if (newJoinConditions.size() == 0) {
+ // this is a cartesian product
+ return ConstantExpression.TRUE;
+ }
+ if (newJoinConditions.size() == 1) {
+ JoinCondition jc = joinConditions.get(newJoinConditions.get(0));
+ if (jc.comparisonType == JoinCondition.comparisonOp.OP_EQ) {
+ return jc.joinCondition;
+ }
+ return null;
+ }
+ ScalarFunctionCallExpression andExpr = new ScalarFunctionCallExpression(
+ BuiltinFunctions.getBuiltinFunctionInfo(AlgebricksBuiltinFunctions.AND));
+
+ // at least one equality predicate needs to be present for a hash join to be possible.
+ boolean eqPredFound = false;
+ for (int joinNum : newJoinConditions) {
+ // need to AND all the expressions.
+ JoinCondition jc = joinConditions.get(joinNum);
+ if (jc.comparisonType == JoinCondition.comparisonOp.OP_EQ) {
+ eqPredFound = true;
+ }
+ andExpr.getArguments().add(new MutableObject<>(jc.joinCondition));
+ }
+ // return null if no equality predicates were found
+ return eqPredFound ? andExpr : null;
+ }
+
+ public HashJoinExpressionAnnotation findHashJoinHint(List<Integer> newJoinConditions) {
+ for (int i : newJoinConditions) {
+ JoinCondition jc = joinConditions.get(i);
+ if (jc.comparisonType != JoinCondition.comparisonOp.OP_EQ) {
+ return null;
+ }
+ ILogicalExpression expr = jc.joinCondition;
+ if (expr.getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL)) {
+ AbstractFunctionCallExpression AFCexpr = (AbstractFunctionCallExpression) expr;
+ HashJoinExpressionAnnotation hjea = AFCexpr.getAnnotation(HashJoinExpressionAnnotation.class);
+ if (hjea != null) {
+ return hjea;
+ }
+ }
+ }
+ return null;
+ }
+
+ public BroadcastExpressionAnnotation findBroadcastHashJoinHint(List<Integer> newJoinConditions) {
+ for (int i : newJoinConditions) {
+ JoinCondition jc = joinConditions.get(i);
+ if (jc.comparisonType != JoinCondition.comparisonOp.OP_EQ) {
+ return null;
+ }
+ ILogicalExpression expr = jc.joinCondition;
+ if (expr.getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL)) {
+ AbstractFunctionCallExpression AFCexpr = (AbstractFunctionCallExpression) expr;
+ BroadcastExpressionAnnotation bcasthjea = AFCexpr.getAnnotation(BroadcastExpressionAnnotation.class);
+ if (bcasthjea != null) {
+ return bcasthjea;
+ }
+ }
+ }
+ return null;
+ }
+
+ public IndexedNLJoinExpressionAnnotation findNLJoinHint(List<Integer> newJoinConditions) {
+ for (int i : newJoinConditions) {
+ JoinCondition jc = joinConditions.get(i);
+ ILogicalExpression expr = jc.joinCondition;
+ if (expr.getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL)) {
+ AbstractFunctionCallExpression AFCexpr = (AbstractFunctionCallExpression) expr;
+ IndexedNLJoinExpressionAnnotation inljea =
+ AFCexpr.getAnnotation(IndexedNLJoinExpressionAnnotation.class);
+ if (inljea != null) {
+ return inljea;
+ }
+ }
+ }
+ return null;
+ }
+
+ public boolean findUseIndexHint(AbstractFunctionCallExpression condition) {
+ if (condition.getFunctionIdentifier().equals(AlgebricksBuiltinFunctions.AND)) {
+ for (int i = 0; i < condition.getArguments().size(); i++) {
+ ILogicalExpression expr = condition.getArguments().get(i).getValue();
+ if (expr.getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL)) {
+ AbstractFunctionCallExpression AFCexpr = (AbstractFunctionCallExpression) expr;
+ if (AFCexpr.hasAnnotation(SecondaryIndexSearchPreferenceAnnotation.class)) {
+ return true;
+ }
+ }
+ }
+ } else if (condition.getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL)) {
+ if (condition.hasAnnotation(SecondaryIndexSearchPreferenceAnnotation.class)) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ public int findJoinNodeIndexByName(String name) {
+ for (int i = 1; i <= this.numberOfTerms; i++) {
+ if (name.equals(jnArray[i].datasetNames.get(0))) {
+ return i;
+ } else if (name.equals(jnArray[i].aliases.get(0))) {
+ return i;
+ }
+ }
+ // should never happen; keep compiler happy.
+ return JoinNode.NO_JN;
+ }
+
+ public int findJoinNodeIndex(LogicalVariable lv) throws AlgebricksException {
+ List<Pair<EmptyTupleSourceOperator, DataSourceScanOperator>> emptyTupleAndDataSourceOps =
+ this.emptyTupleAndDataSourceOps;
+ Map<EmptyTupleSourceOperator, ILogicalOperator> joinLeafInputsHashMap = this.joinLeafInputsHashMap;
+
+ for (Map.Entry<EmptyTupleSourceOperator, ILogicalOperator> mapElement : joinLeafInputsHashMap.entrySet()) {
+ ILogicalOperator joinLeafInput = mapElement.getValue();
+ HashSet<LogicalVariable> vars = new HashSet<>();
+ // this should get the variables from the inputs only, since the join condition is itself set to null
+ VariableUtilities.getLiveVariables(joinLeafInput, vars);
+ if (vars.contains(lv)) {
+ EmptyTupleSourceOperator key = mapElement.getKey();
+ for (int i = 0; i < emptyTupleAndDataSourceOps.size(); i++) {
+ if (key.equals(emptyTupleAndDataSourceOps.get(i).getFirst())) {
+ return i;
+ }
+ }
+ }
+ }
+ return JoinNode.NO_JN;
+ }
+
+ private int findBits(LogicalVariable lv) throws AlgebricksException {
+ int idx = findJoinNodeIndex(lv);
+ if (idx >= 0) {
+ return 1 << idx;
+ }
+
+ // so this variable must be in an internal edge in an assign statement. Find the RHS variables there
+ for (ILogicalOperator op : this.internalEdges) {
+ if (op.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+ List<LogicalVariable> vars2 = new ArrayList<>();
+ VariableUtilities.getUsedVariables(op, vars2);
+ int bits = 0;
+ for (LogicalVariable lv2 : vars2) {
+ bits |= findBits(lv2);
+ }
+ return bits;
+ }
+ }
+ // should never reach this because every variable must exist in some leaf input.
+ return JoinNode.NO_JN;
+ }
+
+ // This finds all the join Conditions in the whole query. This is a global list of all join predicates.
+ // It also fills in the dataset Bits for each join predicate.
+ protected void findJoinConditions() throws AlgebricksException {
+ List<Mutable<ILogicalExpression>> conjs = new ArrayList<>();
+ for (ILogicalOperator jOp : joinOps) {
+ AbstractBinaryJoinOperator joinOp = (AbstractBinaryJoinOperator) jOp;
+ ILogicalExpression expr = joinOp.getCondition().getValue();
+ conjs.clear();
+ if (expr.splitIntoConjuncts(conjs)) {
+ conjs.remove(new MutableObject<ILogicalExpression>(ConstantExpression.TRUE));
+ for (Mutable<ILogicalExpression> conj : conjs) {
+ JoinCondition jc = new JoinCondition();
+ jc.joinCondition = conj.getValue().cloneExpression();
+ joinConditions.add(jc);
+ jc.selectivity = stats.getSelectivityFromAnnotationMain(jc.joinCondition, true);
+ }
+ } else {
+ if ((expr.getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL))) {
+ JoinCondition jc = new JoinCondition();
+ // change to not a true condition
+ jc.joinCondition = expr.cloneExpression();
+ joinConditions.add(jc);
+ jc.selectivity = stats.getSelectivityFromAnnotationMain(jc.joinCondition, true);
+ }
+ }
+ }
+
+ // now patch up any join conditions that have variables referenced in any internal assign statements.
+ List<LogicalVariable> usedVars = new ArrayList<>();
+ for (JoinCondition jc : joinConditions) {
+ usedVars.clear();
+ ILogicalExpression expr = jc.joinCondition;
+ expr.getUsedVariables(usedVars);
+ for (ILogicalOperator ie : internalEdges) {
+ AssignOperator aOp = (AssignOperator) ie;
+ for (int i = 0; i < aOp.getVariables().size(); i++) {
+ if (usedVars.contains(aOp.getVariables().get(i))) {
+ OperatorManipulationUtil.replaceVarWithExpr((AbstractFunctionCallExpression) expr,
+ aOp.getVariables().get(i), aOp.getExpressions().get(i).getValue());
+ jc.joinCondition = expr;
+ jc.selectivity = stats.getSelectivityFromAnnotationMain(jc.joinCondition, true);
+ }
+ }
+ }
+ }
+
+ // now fill the datasetBits for each join condition.
+ for (JoinCondition jc : joinConditions) {
+ ILogicalExpression joinExpr = jc.joinCondition;
+ /*
+ if (joinExpr.getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL)) {
+ AbstractFunctionCallExpression afce = (AbstractFunctionCallExpression) joinExpr;
+ // remove all the join method type annotations.
+ afce.removeAnnotation(BroadcastExpressionAnnotation.class);
+ afce.removeAnnotation(IndexedNLJoinExpressionAnnotation.class);
+ afce.removeAnnotation(HashJoinExpressionAnnotation.class);
+ }
+ */
+ usedVars.clear();
+ joinExpr.getUsedVariables(usedVars);
+ // We only set these for join predicates that have exactly two tables
+ jc.leftSideBits = jc.rightSideBits = JoinCondition.NO_JC;
+ if (((AbstractFunctionCallExpression) joinExpr).getFunctionIdentifier()
+ .equals(AlgebricksBuiltinFunctions.EQ)) {
+ jc.comparisonType = JoinCondition.comparisonOp.OP_EQ;
+ } else {
+ jc.comparisonType = JoinCondition.comparisonOp.OP_OTHER;
+ }
+ jc.numberOfVars = usedVars.size();
+
+ for (int i = 0; i < jc.numberOfVars; i++) {
+ int bits = findBits(usedVars.get(i)); // rename to findInWhichLeaf
+ if (bits != JoinCondition.NO_JC) {
+ if (i == 0) {
+ jc.leftSideBits = bits;
+ } else if (i == 1) {
+ jc.rightSideBits = bits;
+ } else {
+ // have to deal with preds such as r.a + s.a = 5 OR r.a + s.a = t.a
+ }
+ jc.datasetBits |= bits;
+ }
+ }
+ }
+ }
+
+ // in case we have l.partkey = ps.partkey and l.suppkey = ps.suppkey, we will only use the first one for cardinality computations.
+ // treat it like a Pk-Fk join; simplifies cardinality computation
+ private void markCompositeJoinPredicates() {
+ // can use dataSetBits??? This will be simpler.
+ for (int i = 0; i < joinConditions.size() - 1; i++) {
+ for (int j = i + 1; j < joinConditions.size(); j++) {
+ if (joinConditions.get(i).datasetBits == joinConditions.get(j).datasetBits) {
+ joinConditions.get(j).partOfComposite = true;
+ }
+ }
+ }
+ }
+
+ private boolean verticesMatch(JoinCondition jc1, JoinCondition jc2) {
+ return jc1.leftSideBits == jc2.leftSideBits || jc1.leftSideBits == jc2.rightSideBits
+ || jc1.rightSideBits == jc2.leftSideBits || jc1.rightSideBits == jc2.rightSideBits;
+ }
+
+ private void markComponents(int startingJoinCondition) {
+ List<JoinCondition> joinConditions = this.getJoinConditions();
+ // see if all the joinCondition can be reached starting with the first.
+ JoinCondition jc1 = joinConditions.get(startingJoinCondition);
+ for (int i = 0; i < joinConditions.size(); i++) {
+ JoinCondition jc2 = joinConditions.get(i);
+ if (i != startingJoinCondition && jc2.componentNumber == 0) {
+ // a new edge not visited before
+ if (verticesMatch(jc1, jc2)) {
+ jc2.componentNumber = 1;
+ markComponents(i);
+ }
+ }
+ }
+ }
+
+ private void findIfJoinGraphIsConnected() {
+ int numJoinConditions = joinConditions.size();
+ if (numJoinConditions < numberOfTerms - 1) {
+ /// not enough join predicates
+ connectedJoinGraph = false;
+ return;
+ }
+ if (numJoinConditions > 0) {
+ joinConditions.get(0).componentNumber = 1;
+ markComponents(0);
+ for (int i = 1; i < numJoinConditions; i++) {
+ if (joinConditions.get(i).componentNumber == 0) {
+ connectedJoinGraph = false;
+ return;
+ }
+ }
+ }
+ }
+
+ private double findInListCard(ILogicalOperator op) {
+ if (op.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+ return 1.0;
+ }
+
+ if (op.getOperatorTag() == LogicalOperatorTag.UNNEST) {
+ UnnestOperator unnestOp = (UnnestOperator) op;
+ ILogicalExpression unnestExpr = unnestOp.getExpressionRef().getValue();
+ UnnestingFunctionCallExpression unnestingFuncExpr = (UnnestingFunctionCallExpression) unnestExpr;
+
+ if (unnestingFuncExpr.getFunctionIdentifier().equals(BuiltinFunctions.SCAN_COLLECTION)) {
+ if (unnestingFuncExpr.getArguments().get(0).getValue()
+ .getExpressionTag() == LogicalExpressionTag.CONSTANT) {
+ ConstantExpression constantExpr =
+ (ConstantExpression) unnestingFuncExpr.getArguments().get(0).getValue();
+ AsterixConstantValue constantValue = (AsterixConstantValue) constantExpr.getValue();
+ IAObject v = constantValue.getObject();
+ if (v.getType().getTypeTag() == ATypeTag.ARRAY) {
+ AOrderedList array = (AOrderedList) v;
+ return array.size();
+ }
+ }
+ }
+ }
+ // just a guess
+ return 10.0;
+ }
+
+ private String findAlias(DataSourceScanOperator scanOp) {
+ DataSource ds = (DataSource) scanOp.getDataSource();
+ List<LogicalVariable> allVars = scanOp.getVariables();
+ LogicalVariable dataRecVarInScan = ds.getDataRecordVariable(allVars);
+ return dataRecVarInScan.toString().substring(2);
+ }
+
+ private int addNonBushyJoinNodes(int level, int jnNumber, int[] startJnAtLevel) throws AlgebricksException {
+ // adding joinNodes of level (2, 3, ..., numberOfTerms)
+ int startJnSecondLevel = startJnAtLevel[2];
+ int startJnPrevLevel = startJnAtLevel[level - 1];
+ int startJnNextLevel = startJnAtLevel[level];
+ int i, j, addPlansToThisJn;
+
+ // walking thru the previous level
+ for (i = startJnPrevLevel; i < startJnNextLevel; i++) {
+ JoinNode jnI = jnArray[i];
+ jnI.jnArrayIndex = i;
+ if (jnI.highestDatasetId == 0) {
+ // this jn can be skipped
+ continue;
+ }
+
+ // walk thru the first level here
+ for (j = 1; j < startJnSecondLevel; j++) {
+ if (level == 2 && i > j) {
+ // don't want to generate x y and y x. we will do this in plan generation.
+ continue;
+ }
+ JoinNode jnJ = jnArray[j];
+ jnJ.jnArrayIndex = j;
+ if ((jnI.datasetBits & jnJ.datasetBits) > 0) {
+ // these already have some common table
+ continue;
+ }
+ int newBits = jnI.datasetBits | jnJ.datasetBits;
+ JoinNode jnNewBits = jnArray[newBits];
+ jnNewBits.jnArrayIndex = newBits;
+ // visiting this join node for the first time
+ if (jnNewBits.jnIndex == 0) {
+ jnNumber++;
+ JoinNode jn = jnArray[jnNumber];
+ jn.jnArrayIndex = jnNumber;
+ // if we want to locate the joinNode num (say 33) which has tables 1, 2, and 5.
+ // if these bits are turned on, we get 19. Then jn[19].jn_index will equal 33.
+ // Then jn[33].highestKeyspaceId will equal 5
+ // if this joinNode ever gets removed, then set jn[19].highestKeyspaceId = 0
+ jn.datasetBits = newBits;
+ jnNewBits.jnIndex = addPlansToThisJn = jnNumber;
+ jn.level = level;
+ jn.highestDatasetId = Math.max(jnI.highestDatasetId, j);
+
+ jn.datasetIndexes = new ArrayList<>();
+ jn.datasetIndexes.addAll(jnI.datasetIndexes);
+ jn.datasetIndexes.addAll(jnJ.datasetIndexes);
+ Collections.sort(jn.datasetIndexes);
+
+ jn.datasetNames = new ArrayList<>();
+ jn.datasetNames.addAll(jnI.datasetNames);
+ jn.datasetNames.addAll(jnJ.datasetNames);
+ Collections.sort(jn.datasetNames);
+ jn.aliases = new ArrayList<>();
+ jn.aliases.addAll(jnI.aliases);
+ jn.aliases.addAll(jnJ.aliases);
+ Collections.sort(jn.aliases);
+ jn.size = jnI.size + jnJ.size;
+ jn.cardinality = jn.computeJoinCardinality();
+ if (jn.cardinality < 2.1) {
+ jn.cardinality = 2.1; // for keeping CP and HJ cost formulas happy.
+ }
+ } else {
+ addPlansToThisJn = jnNewBits.jnIndex;
+ }
+
+ JoinNode jnIJ = jnArray[addPlansToThisJn];
+ jnIJ.jnArrayIndex = addPlansToThisJn;
+ jnIJ.addMultiDatasetPlans(jnI, jnJ);
+ if (forceJoinOrderMode) {
+ break;
+ }
+ }
+ if (forceJoinOrderMode) {
+ break;
+ }
+ }
+
+ return jnNumber;
+ }
+
+ private int enumerateHigherLevelJoinNodes() throws AlgebricksException {
+ int jnNumber = this.numberOfTerms;
+ int[] firstJnAtLevel;
+ firstJnAtLevel = new int[numberOfTerms + 1];
+ firstJnAtLevel[1] = 1;
+ IPlanPrettyPrinter pp = optCtx.getPrettyPrinter();
+ // after implementing greedy plan, we can start at level 3;
+ int startLevel = 2;
+ if (LOGGER.isTraceEnabled()) {
+ EnumerateJoinsRule.printPlan(pp, op, "Original Whole plan in JN 4");
+ }
+ for (int level = startLevel; level <= numberOfTerms; level++) {
+ firstJnAtLevel[level] = jnNumber + 1;
+ jnNumber = addNonBushyJoinNodes(level, jnNumber, firstJnAtLevel);
+ }
+ if (LOGGER.isTraceEnabled()) {
+ EnumerateJoinsRule.printPlan(pp, op, "Original Whole plan in JN 5");
+ }
+ return jnNumber;
+ }
+
+ protected int enumerateBaseLevelJoinNodes() throws AlgebricksException {
+ int lastBaseLevelJnNum = initializeBaseLevelJoinNodes();
+ if (lastBaseLevelJnNum == PlanNode.NO_PLAN) {
+ return PlanNode.NO_PLAN;
+ }
+ int dataScanPlan = PlanNode.NO_PLAN;
+ for (int i = 1; i <= numberOfTerms; i++) {
+ JoinNode jn = jnArray[i];
+ EmptyTupleSourceOperator ets = emptyTupleAndDataSourceOps.get(i - 1).getFirst();
+ ILogicalOperator leafInput = joinLeafInputsHashMap.get(ets);
+ dataScanPlan = jn.addSingleDatasetPlans();
+ if (dataScanPlan == PlanNode.NO_PLAN) {
+ return PlanNode.NO_PLAN;
+ }
+ // We may not add any index plans, so need to check for NO_PLAN
+ jn.addIndexAccessPlans(leafInput);
+ }
+ return numberOfTerms;
+ }
+
+ protected int initializeBaseLevelJoinNodes() throws AlgebricksException {
+ // join nodes have been allocated in the JoinEnum
+ // add a dummy Plan Node; we do not want planNode at position 0 to be a valid plan
+ PlanNode pn = new PlanNode(0, this);
+ pn.jn = null;
+ pn.jnIndexes[0] = pn.jnIndexes[1] = JoinNode.NO_JN;
+ pn.planIndexes[0] = pn.planIndexes[1] = PlanNode.NO_PLAN;
+ pn.opCost = pn.totalCost = new Cost(0);
+ allPlans.add(pn);
+
+ boolean noCards = false;
+ // initialize the level 1 join nodes
+ for (int i = 1; i <= numberOfTerms; i++) {
+ JoinNode jn = jnArray[i];
+ jn.jnArrayIndex = i;
+ jn.datasetBits = 1 << (i - 1);
+ jn.datasetIndexes = new ArrayList<>(Collections.singleton(i));
+ EmptyTupleSourceOperator ets = emptyTupleAndDataSourceOps.get(i - 1).getFirst();
+ ILogicalOperator leafInput = joinLeafInputsHashMap.get(ets);
+
+ DataSourceScanOperator scanOp = emptyTupleAndDataSourceOps.get(i - 1).getSecond();
+ if (scanOp != null) {
+ DataSourceId id = (DataSourceId) scanOp.getDataSource().getId();
+ jn.aliases = new ArrayList<>(Collections.singleton(findAlias(scanOp)));
+ jn.datasetNames = new ArrayList<>(Collections.singleton(id.getDatasourceName()));
+ Index.SampleIndexDetails idxDetails;
+ Index index = stats.findSampleIndex(scanOp, optCtx);
+ if (index != null) {
+ idxDetails = (Index.SampleIndexDetails) index.getIndexDetails();
+ } else {
+ idxDetails = null;
+ }
+
+ jn.idxDetails = idxDetails;
+ if (cboTestMode) {
+ // to make asterix tests run
+ jn.origCardinality = 1000000;
+ jn.size = 500;
+ } else {
+ if (idxDetails == null) {
+ return PlanNode.NO_PLAN;
+ }
+ jn.setOrigCardinality(idxDetails.getSourceCardinality());
+ jn.setAvgDocSize(idxDetails.getSourceAvgItemSize());
+ }
+ // multiply by the respective predicate selectivities
+ jn.cardinality = jn.origCardinality * stats.getSelectivity(leafInput, false);
+ if (jn.cardinality < 2.1) {
+ jn.cardinality = 2.1; // for keeping CP and HJ cost formulas happy.
+ }
+ } else {
+ // could be unnest or assign
+ jn.datasetNames = new ArrayList<>(Collections.singleton("unnestOrAssign"));
+ jn.aliases = new ArrayList<>(Collections.singleton("unnestOrAssign"));
+ jn.origCardinality = jn.cardinality = findInListCard(leafInput);
+ // just a guess
+ jn.size = 10;
+ }
+
+ if (jn.origCardinality >= Cost.MAX_CARD) {
+ noCards = true;
+ }
+ jn.correspondingEmptyTupleSourceOp = emptyTupleAndDataSourceOps.get(i - 1).getFirst();
+ jn.highestDatasetId = i;
+ jn.level = 1;
+ }
+ if (noCards) {
+ return PlanNode.NO_PLAN;
+ }
+ return numberOfTerms;
+ }
+
+ // main entry point in this file
+ public int enumerateJoins() throws AlgebricksException {
+ // create a localJoinOp for use in calling existing nested loops code.
+ InnerJoinOperator dummyInput = new InnerJoinOperator(null, null, null);
+ localJoinOp = new InnerJoinOperator(new MutableObject<>(ConstantExpression.TRUE),
+ new MutableObject<>(dummyInput), new MutableObject<>(dummyInput));
+
+ int lastBaseLevelJnNum = enumerateBaseLevelJoinNodes();
+ if (lastBaseLevelJnNum == PlanNode.NO_PLAN) {
+ return PlanNode.NO_PLAN;
+ }
+
+ IPlanPrettyPrinter pp = optCtx.getPrettyPrinter();
+ if (LOGGER.isTraceEnabled()) {
+ EnumerateJoinsRule.printPlan(pp, op, "Original Whole plan in JN 1");
+ }
+
+ findJoinConditions();
+ findIfJoinGraphIsConnected();
+
+ if (LOGGER.isTraceEnabled()) {
+ EnumerateJoinsRule.printPlan(pp, op, "Original Whole plan in JN 2");
+ }
+
+ markCompositeJoinPredicates();
+ int lastJnNum = enumerateHigherLevelJoinNodes();
+ JoinNode lastJn = jnArray[lastJnNum];
+ if (LOGGER.isTraceEnabled()) {
+ EnumerateJoinsRule.printPlan(pp, op, "Original Whole plan in JN END");
+ LOGGER.trace(dumpJoinNodes(lastJnNum));
+ }
+
+ // find the cheapest plan
+ int cheapestPlanIndex = lastJn.cheapestPlanIndex;
+ if (LOGGER.isTraceEnabled() && cheapestPlanIndex > 0) {
+ LOGGER.trace("Cheapest Plan is {} number of terms is {} joinNodes {}", cheapestPlanIndex, numberOfTerms,
+ lastJnNum);
+ }
+
+ return cheapestPlanIndex;
+ }
+
+ private String dumpJoinNodes(int numJoinNodes) {
+ StringBuilder sb = new StringBuilder(128);
+ sb.append(LocalDateTime.now());
+ for (int i = 1; i <= numJoinNodes; i++) {
+ JoinNode jn = jnArray[i];
+ sb.append(jn);
+ }
+ sb.append('\n').append("Printing cost of all Final Plans").append('\n');
+ jnArray[numJoinNodes].printCostOfAllPlans(sb);
+ return sb.toString();
+ }
+
+ public static boolean getForceJoinOrderMode(IOptimizationContext context) {
+ PhysicalOptimizationConfig physOptConfig = context.getPhysicalOptimizationConfig();
+ return physOptConfig.getForceJoinOrderMode();
+ }
+
+ public static String getQueryPlanShape(IOptimizationContext context) {
+ PhysicalOptimizationConfig physOptConfig = context.getPhysicalOptimizationConfig();
+ return physOptConfig.getQueryPlanShapeMode();
+ }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/JoinNode.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/JoinNode.java
new file mode 100644
index 0000000..1f9300d
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/JoinNode.java
@@ -0,0 +1,984 @@
+/*
+ * 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.optimizer.rules.cbo;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.TreeMap;
+
+import org.apache.asterix.common.annotations.IndexedNLJoinExpressionAnnotation;
+import org.apache.asterix.common.annotations.SkipSecondaryIndexSearchExpressionAnnotation;
+import org.apache.asterix.metadata.entities.Index;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.optimizer.cost.Cost;
+import org.apache.asterix.optimizer.cost.ICost;
+import org.apache.asterix.optimizer.rules.am.AccessMethodAnalysisContext;
+import org.apache.asterix.optimizer.rules.am.IAccessMethod;
+import org.apache.asterix.optimizer.rules.am.IOptimizableFuncExpr;
+import org.apache.asterix.optimizer.rules.am.IntroduceJoinAccessMethodRule;
+import org.apache.asterix.optimizer.rules.am.IntroduceSelectAccessMethodRule;
+import org.apache.commons.lang3.mutable.MutableObject;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.BroadcastExpressionAnnotation;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.HashJoinExpressionAnnotation;
+import org.apache.hyracks.algebricks.core.algebra.expressions.PredicateCardinalityAnnotation;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import org.apache.hyracks.algebricks.core.config.AlgebricksConfig;
+import org.apache.hyracks.api.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.Warning;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+public class JoinNode {
+ private static final Logger LOGGER = LogManager.getLogger();
+
+ protected JoinEnum joinEnum;
+ protected int jnArrayIndex;
+ protected int datasetBits; // this is bitmap of all the keyspaceBits present in this joinNode
+ protected List<Integer> datasetIndexes;
+ protected List<String> datasetNames;
+ protected List<String> aliases;
+ protected int cheapestPlanIndex;
+ protected ICost cheapestPlanCost;
+ protected double origCardinality; // without any selections
+ protected double cardinality;
+ protected double size;
+ protected List<Integer> planIndexesArray; // indexes into the PlanNode array in enumerateJoins
+ protected int jnIndex, level, highestDatasetId;
+ protected JoinNode rightJn, leftJn;
+ protected List<Integer> applicableJoinConditions;
+ protected EmptyTupleSourceOperator correspondingEmptyTupleSourceOp; // There is a 1-1 relationship between the LVs and the dataSourceScanOps and the leafInputs.
+ protected List<Pair<IAccessMethod, Index>> chosenIndexes;
+ protected Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs;
+ protected Index.SampleIndexDetails idxDetails;
+ protected static int NO_JN = -1;
+ protected static int NO_CARDS = -2;
+
+ public JoinNode(int i) {
+ this.jnArrayIndex = i;
+ planIndexesArray = new ArrayList<>();
+ cheapestPlanIndex = PlanNode.NO_PLAN;
+ size = 1; // for now, will be the size of the doc for this joinNode
+ }
+
+ public JoinNode(int i, JoinEnum joinE) {
+ this(i);
+ joinEnum = joinE;
+ cheapestPlanCost = joinEnum.getCostHandle().maxCost();
+ }
+
+ public boolean IsBaseLevelJoinNode() {
+ return this.jnArrayIndex <= joinEnum.numberOfTerms;
+ }
+
+ public boolean IsHigherLevelJoinNode() {
+ return !IsBaseLevelJoinNode();
+ }
+
+ public double computeJoinCardinality() {
+ JoinNode[] jnArray = joinEnum.getJnArray();
+ List<JoinCondition> joinConditions = joinEnum.getJoinConditions();
+
+ this.applicableJoinConditions = new ArrayList<>();
+ findApplicableJoinConditions();
+
+ if (LOGGER.isTraceEnabled() && this.applicableJoinConditions.size() == 0) {
+ LOGGER.trace("applicable Join Conditions size is 0 in join Node " + this.jnArrayIndex);
+ }
+
+ // Wonder if this computation will result in an overflow exception. Better to multiply them with selectivities also.
+ double productJoinCardinality = 1.0;
+ for (int idx : this.datasetIndexes) {
+ productJoinCardinality *= jnArray[idx].cardinality;
+ }
+
+ double productJoinSels = 1.0;
+ for (int idx : this.applicableJoinConditions) {
+ if (!joinConditions.get(idx).partOfComposite) {
+ productJoinSels *= joinConditions.get(idx).selectivity;
+ }
+ }
+ return productJoinCardinality * productJoinSels;
+ }
+
+ public double getCardinality() {
+ return cardinality;
+ }
+
+ public void setCardinality(double card) {
+ cardinality = card;
+ }
+
+ public double getOrigCardinality() {
+ return origCardinality;
+ }
+
+ public void setOrigCardinality(double card) {
+ origCardinality = card;
+ }
+
+ public void setAvgDocSize(double avgDocSize) {
+ size = avgDocSize;
+ }
+
+ public double getInputSize() {
+ return size;
+ }
+
+ public double getOutputSize() {
+ return size; // need to change this to account for projections
+ }
+
+ public JoinNode getLeftJn() {
+ return leftJn;
+ }
+
+ public JoinNode getRightJn() {
+ return rightJn;
+ }
+
+ public List<String> getAliases() {
+ return aliases;
+ }
+
+ public List<String> getDatasetNames() {
+ return datasetNames;
+ }
+
+ public Index.SampleIndexDetails getIdxDetails() {
+ return idxDetails;
+ }
+
+ protected boolean nestedLoopsApplicable(ILogicalExpression joinExpr) throws AlgebricksException {
+
+ List<LogicalVariable> usedVarList = new ArrayList<>();
+ joinExpr.getUsedVariables(usedVarList);
+ if (usedVarList.size() != 2) {
+ return false;
+ }
+
+ if (joinExpr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+ return false;
+ }
+
+ LogicalVariable var0 = usedVarList.get(0);
+ LogicalVariable var1 = usedVarList.get(1);
+
+ // Find which joinLeafInput these vars belong to.
+ // go thru the leaf inputs and see where these variables came from
+ ILogicalOperator joinLeafInput0 = joinEnum.findLeafInput(Collections.singletonList(var0));
+ if (joinLeafInput0 == null) {
+ return false; // this should not happen unless an assignment is between two joins.
+ }
+
+ ILogicalOperator joinLeafInput1 = joinEnum.findLeafInput(Collections.singletonList(var1));
+ if (joinLeafInput1 == null) {
+ return false;
+ }
+
+ // We need to find out which one of these is the inner joinLeafInput. So for that get the joinLeafInput of this join node.
+ ILogicalOperator innerLeafInput = joinEnum.joinLeafInputsHashMap.get(this.correspondingEmptyTupleSourceOp);
+
+ // This must equal one of the two joinLeafInputsHashMap found above. check for sanity!!
+ if (innerLeafInput != joinLeafInput1 && innerLeafInput != joinLeafInput0) {
+ return false; // This should not happen. So debug to find out why this happened.
+ }
+
+ if (innerLeafInput == joinLeafInput0) {
+ joinEnum.localJoinOp.getInputs().get(0).setValue(joinLeafInput1);
+ } else {
+ joinEnum.localJoinOp.getInputs().get(0).setValue(joinLeafInput0);
+ }
+
+ joinEnum.localJoinOp.getInputs().get(1).setValue(innerLeafInput);
+
+ // We will always use the first join Op to provide the joinOp input for invoking rewritePre
+ AbstractBinaryJoinOperator joinOp = (AbstractBinaryJoinOperator) joinEnum.localJoinOp;
+ joinOp.getCondition().setValue(joinExpr);
+
+ // Now call the rewritePre code
+ IntroduceJoinAccessMethodRule tmp = new IntroduceJoinAccessMethodRule();
+ boolean retVal = tmp.checkApplicable(new MutableObject<>(joinEnum.localJoinOp), joinEnum.optCtx);
+
+ return retVal;
+ }
+
+ /** one is a subset of two */
+ private boolean subset(int one, int two) {
+ return (one & two) == one;
+ }
+
+ protected void findApplicableJoinConditions() {
+ List<JoinCondition> joinConditions = joinEnum.getJoinConditions();
+
+ int i = 0;
+ for (JoinCondition jc : joinConditions) {
+ if (subset(jc.datasetBits, this.datasetBits)) {
+ this.applicableJoinConditions.add(i);
+ }
+ i++;
+ }
+ }
+
+ protected List<Integer> getNewJoinConditionsOnly() {
+ List<Integer> newJoinConditions = new ArrayList<>();
+ JoinNode leftJn = this.leftJn;
+ JoinNode rightJn = this.rightJn;
+ // find the new table being added. This assume only zig zag trees for now.
+ int newTableBits = 0;
+ if (leftJn.jnArrayIndex <= joinEnum.numberOfTerms) {
+ newTableBits = leftJn.datasetBits;
+ } else if (rightJn.jnArrayIndex <= joinEnum.numberOfTerms) {
+ newTableBits = rightJn.datasetBits;
+ }
+
+ if (LOGGER.isTraceEnabled() && newTableBits == 0) {
+ LOGGER.trace("newTable Bits == 0");
+ }
+
+ // All the new join predicates will have these bits turned on
+ for (int idx : this.applicableJoinConditions) {
+ if ((joinEnum.joinConditions.get(idx).datasetBits & newTableBits) > 0) {
+ newJoinConditions.add(idx);
+ }
+ }
+
+ return newJoinConditions; // this can be of size 0 because this may be a cartesian join
+ }
+
+ public int addSingleDatasetPlans() {
+ List<PlanNode> allPlans = joinEnum.allPlans;
+ ICost opCost, totalCost;
+
+ opCost = joinEnum.getCostMethodsHandle().costFullScan(this);
+ totalCost = opCost;
+ if (this.cheapestPlanIndex == PlanNode.NO_PLAN || opCost.costLT(this.cheapestPlanCost)) {
+ // for now just add one plan
+ PlanNode pn = new PlanNode(allPlans.size(), joinEnum);
+ pn.jn = this;
+ pn.datasetName = this.datasetNames.get(0);
+ pn.correspondingEmptyTupleSourceOp = this.correspondingEmptyTupleSourceOp;
+ pn.jnIndexes[0] = this.jnArrayIndex;
+ pn.jnIndexes[1] = JoinNode.NO_JN;
+ pn.planIndexes[0] = PlanNode.NO_PLAN; // There ane no plans below this plan.
+ pn.planIndexes[1] = PlanNode.NO_PLAN; // There ane no plans below this plan.
+ pn.opCost = opCost;
+ pn.scanOp = PlanNode.ScanMethod.TABLE_SCAN;
+ pn.totalCost = totalCost;
+
+ allPlans.add(pn);
+ this.planIndexesArray.add(allPlans.size() - 1);
+ this.cheapestPlanCost = totalCost;
+ this.cheapestPlanIndex = allPlans.size() - 1;
+ return this.cheapestPlanIndex;
+ }
+ return PlanNode.NO_PLAN;
+ }
+
+ protected void buildIndexPlan(boolean forceIndexPlan) {
+ List<PlanNode> allPlans = joinEnum.allPlans;
+ ICost opCost, totalCost;
+
+ opCost = joinEnum.getCostMethodsHandle().costIndexScan(this);
+ totalCost = opCost;
+ if (this.cheapestPlanIndex == PlanNode.NO_PLAN || opCost.costLT(this.cheapestPlanCost) || forceIndexPlan) {
+ // for now just add one plan
+ PlanNode pn = new PlanNode(allPlans.size(), joinEnum);
+ pn.jn = this;
+ pn.datasetName = this.datasetNames.get(0);
+ pn.correspondingEmptyTupleSourceOp = this.correspondingEmptyTupleSourceOp;
+ pn.jnIndexes[0] = this.jnArrayIndex;
+ pn.jnIndexes[1] = JoinNode.NO_JN;
+ pn.planIndexes[0] = PlanNode.NO_PLAN; // There ane no plans below this plan.
+ pn.planIndexes[1] = PlanNode.NO_PLAN; // There ane no plans below this plan.
+ pn.opCost = opCost;
+ pn.scanOp = PlanNode.ScanMethod.INDEX_SCAN;
+ pn.totalCost = totalCost;
+
+ allPlans.add(pn);
+ this.planIndexesArray.add(allPlans.size() - 1);
+ this.cheapestPlanCost = totalCost;
+ this.cheapestPlanIndex = allPlans.size() - 1;
+ }
+ }
+
+ protected void costAndChooseIndexPlans(ILogicalOperator leafInput,
+ Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs) throws AlgebricksException {
+ // Skip indexes with selectivity greater than 0.1, add the SKIP_SECONDARY_INDEX annotation to its expression.
+ double sel;
+ int exprIndex;
+ for (Map.Entry<IAccessMethod, AccessMethodAnalysisContext> amEntry : analyzedAMs.entrySet()) {
+ AccessMethodAnalysisContext analysisCtx = amEntry.getValue();
+ Iterator<Map.Entry<Index, List<Pair<Integer, Integer>>>> indexIt =
+ analysisCtx.getIteratorForIndexExprsAndVars();
+ List<IOptimizableFuncExpr> exprs = analysisCtx.getMatchedFuncExprs();
+ while (indexIt.hasNext()) {
+ Map.Entry<Index, List<Pair<Integer, Integer>>> indexEntry = indexIt.next();
+ Index chosenIndex = indexEntry.getKey();
+ exprIndex = indexEntry.getValue().get(0).getFirst();
+ IOptimizableFuncExpr expr = exprs.get(exprIndex);
+ AbstractFunctionCallExpression afce = expr.getFuncExpr();
+ PredicateCardinalityAnnotation selectivityAnnotation =
+ afce.getAnnotation(PredicateCardinalityAnnotation.class);
+ if (joinEnum.findUseIndexHint(afce)) {
+ buildIndexPlan(true);
+ } else if (selectivityAnnotation != null) {
+ sel = selectivityAnnotation.getSelectivity();
+ if (sel >= joinEnum.stats.SELECTIVITY_FOR_SECONDARY_INDEX_SELECTION) {
+ afce.putAnnotation(SkipSecondaryIndexSearchExpressionAnnotation
+ .newInstance(Collections.singleton(chosenIndex.getIndexName())));
+ } else {
+ buildIndexPlan(false);
+ }
+ }
+ }
+ }
+ }
+
+ private SelectOperator copySelExprsAndSetTrue(List<ILogicalExpression> selExprs, List<SelectOperator> selOpers,
+ ILogicalOperator leafInput) {
+ ILogicalOperator op = leafInput;
+ SelectOperator firstSelOp = null;
+ boolean firstSel = true;
+ while (op != null && op.getOperatorTag() != LogicalOperatorTag.EMPTYTUPLESOURCE) {
+ if (op.getOperatorTag() == LogicalOperatorTag.SELECT) {
+ SelectOperator selOp = (SelectOperator) op;
+ if (firstSel) {
+ firstSelOp = selOp;
+ firstSel = false;
+ }
+ selOpers.add(selOp);
+ selExprs.add(selOp.getCondition().getValue());
+ selOp.getCondition().setValue(ConstantExpression.TRUE); // we will switch these back later
+ }
+ op = op.getInputs().get(0).getValue();
+ }
+ return firstSelOp;
+ }
+
+ private void restoreSelExprs(List<ILogicalExpression> selExprs, List<SelectOperator> selOpers) {
+ for (int i = 0; i < selExprs.size(); i++) {
+ selOpers.get(i).getCondition().setValue(selExprs.get(i));
+ }
+ }
+
+ private ILogicalExpression andAlltheExprs(List<ILogicalExpression> selExprs) {
+ if (selExprs.size() == 1) {
+ return selExprs.get(0);
+ }
+
+ ScalarFunctionCallExpression andExpr = new ScalarFunctionCallExpression(
+ BuiltinFunctions.getBuiltinFunctionInfo(AlgebricksBuiltinFunctions.AND));
+
+ for (ILogicalExpression se : selExprs) {
+ andExpr.getArguments().add(new MutableObject<>(se));
+ }
+ return andExpr;
+ }
+
+ // Look for the pattern select, select, subplan and collapse to select, subplan
+ // This code does not belong in the CBO!!
+ private boolean combineDoubleSelectsBeforeSubPlans(ILogicalOperator op) {
+ boolean changes = false;
+ while (op != null && op.getOperatorTag() != LogicalOperatorTag.EMPTYTUPLESOURCE) {
+ if (op.getOperatorTag() == LogicalOperatorTag.SELECT) {
+ SelectOperator selOp1 = (SelectOperator) op;
+ if (selOp1.getInputs().get(0).getValue().getOperatorTag().equals(LogicalOperatorTag.SELECT)) {
+ SelectOperator selOp2 = (SelectOperator) (op.getInputs().get(0).getValue());
+ ILogicalOperator op2 = selOp2.getInputs().get(0).getValue();
+ if (op2.getOperatorTag() == LogicalOperatorTag.SUBPLAN) { // found the pattern we are looking for
+ selOp1.getInputs().get(0).setValue(op2);
+ ILogicalExpression exp1 = selOp1.getCondition().getValue();
+ ILogicalExpression exp2 = selOp2.getCondition().getValue();
+ ScalarFunctionCallExpression andExpr = new ScalarFunctionCallExpression(
+ BuiltinFunctions.getBuiltinFunctionInfo(AlgebricksBuiltinFunctions.AND));
+ andExpr.getArguments().add(new MutableObject<>(exp1));
+ andExpr.getArguments().add(new MutableObject<>(exp2));
+ selOp1.getCondition().setValue(andExpr);
+ op = op2.getInputs().get(0).getValue();
+ changes = true;
+ }
+ }
+ }
+ op = op.getInputs().get(0).getValue();
+ }
+ return changes;
+ }
+
+ public void addIndexAccessPlans(ILogicalOperator leafInput) throws AlgebricksException {
+ IntroduceSelectAccessMethodRule tmp = new IntroduceSelectAccessMethodRule();
+ List<Pair<IAccessMethod, Index>> chosenIndexes = new ArrayList<>();
+ Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs = new TreeMap<>();
+
+ while (combineDoubleSelectsBeforeSubPlans(leafInput));
+ List<ILogicalExpression> selExprs = new ArrayList<>();
+ List<SelectOperator> selOpers = new ArrayList<>();
+ SelectOperator firstSelop = copySelExprsAndSetTrue(selExprs, selOpers, leafInput);
+ if (firstSelop != null) { // if there are no selects, then there is no question of index selections either.
+ firstSelop.getCondition().setValue(andAlltheExprs(selExprs));
+ boolean index_access_possible =
+ tmp.checkApplicable(new MutableObject<>(leafInput), joinEnum.optCtx, chosenIndexes, analyzedAMs);
+ this.chosenIndexes = chosenIndexes;
+ this.analyzedAMs = analyzedAMs;
+ restoreSelExprs(selExprs, selOpers);
+ if (index_access_possible) {
+ costAndChooseIndexPlans(leafInput, analyzedAMs);
+ }
+ } else {
+ restoreSelExprs(selExprs, selOpers);
+ }
+ }
+
+ protected int buildHashJoinPlan(JoinNode leftJn, JoinNode rightJn, ILogicalExpression hashJoinExpr,
+ HashJoinExpressionAnnotation hintHashJoin) {
+ List<PlanNode> allPlans = joinEnum.allPlans;
+ PlanNode pn;
+ ICost hjCost, leftExchangeCost, rightExchangeCost, childCosts, totalCost;
+ this.leftJn = leftJn;
+ this.rightJn = rightJn;
+ int leftPlan = leftJn.cheapestPlanIndex;
+ int rightPlan = rightJn.cheapestPlanIndex;
+
+ if (hashJoinExpr == null || hashJoinExpr == ConstantExpression.TRUE) {
+ return PlanNode.NO_PLAN;
+ }
+
+ if (joinEnum.queryPlanShape.equals(AlgebricksConfig.QUERY_PLAN_SHAPE_LEFTDEEP)
+ && !leftJn.IsBaseLevelJoinNode()) {
+ return PlanNode.NO_PLAN;
+ }
+
+ if (joinEnum.queryPlanShape.equals(AlgebricksConfig.QUERY_PLAN_SHAPE_RIGHTDEEP)
+ && !rightJn.IsBaseLevelJoinNode()) {
+ return PlanNode.NO_PLAN;
+ }
+
+ if (rightJn.cardinality * rightJn.size <= leftJn.cardinality * leftJn.size || hintHashJoin != null
+ || joinEnum.forceJoinOrderMode
+ || !joinEnum.queryPlanShape.equals(AlgebricksConfig.QUERY_PLAN_SHAPE_ZIGZAG)) {
+ // We want to build with the smaller side.
+ hjCost = joinEnum.getCostMethodsHandle().costHashJoin(this);
+ leftExchangeCost = joinEnum.getCostMethodsHandle().computeHJProbeExchangeCost(this);
+ rightExchangeCost = joinEnum.getCostMethodsHandle().computeHJBuildExchangeCost(this);
+ childCosts = allPlans.get(leftPlan).totalCost.costAdd(allPlans.get(rightPlan).totalCost);
+ totalCost = hjCost.costAdd(leftExchangeCost).costAdd(rightExchangeCost).costAdd(childCosts);
+ if (this.cheapestPlanIndex == PlanNode.NO_PLAN || totalCost.costLT(this.cheapestPlanCost)) {
+ pn = new PlanNode(allPlans.size(), joinEnum);
+ pn.jn = this;
+ pn.jnIndexes[0] = leftJn.jnArrayIndex;
+ pn.jnIndexes[1] = rightJn.jnArrayIndex;
+ pn.planIndexes[0] = leftPlan;
+ pn.planIndexes[1] = rightPlan;
+ pn.joinOp = PlanNode.JoinMethod.HYBRID_HASH_JOIN; // need to check that all the conditions have equality predicates ONLY.
+ if (hintHashJoin != null) {
+ hintHashJoin.setBuildSide(HashJoinExpressionAnnotation.BuildSide.RIGHT);
+ }
+ pn.side = HashJoinExpressionAnnotation.BuildSide.RIGHT;
+ pn.joinExpr = hashJoinExpr;
+ pn.opCost = hjCost;
+ pn.totalCost = totalCost;
+ pn.leftExchangeCost = leftExchangeCost;
+ pn.rightExchangeCost = rightExchangeCost;
+ allPlans.add(pn);
+ this.planIndexesArray.add(allPlans.size() - 1);
+ this.cheapestPlanCost = totalCost;
+ this.cheapestPlanIndex = allPlans.size() - 1;
+ return this.cheapestPlanIndex;
+ }
+ }
+
+ return PlanNode.NO_PLAN;
+ }
+
+ protected int buildBroadcastHashJoinPlan(JoinNode leftJn, JoinNode rightJn, ILogicalExpression hashJoinExpr,
+ BroadcastExpressionAnnotation hintBroadcastHashJoin) {
+ List<PlanNode> allPlans = joinEnum.allPlans;
+ PlanNode pn;
+ ICost bcastHjCost, leftExchangeCost, rightExchangeCost, childCosts, totalCost;
+
+ this.leftJn = leftJn;
+ this.rightJn = rightJn;
+ int leftPlan = leftJn.cheapestPlanIndex;
+ int rightPlan = rightJn.cheapestPlanIndex;
+
+ if (hashJoinExpr == null || hashJoinExpr == ConstantExpression.TRUE) {
+ return PlanNode.NO_PLAN;
+ }
+
+ if (joinEnum.queryPlanShape.equals(AlgebricksConfig.QUERY_PLAN_SHAPE_LEFTDEEP)
+ && !leftJn.IsBaseLevelJoinNode()) {
+ return PlanNode.NO_PLAN;
+ }
+
+ if (joinEnum.queryPlanShape.equals(AlgebricksConfig.QUERY_PLAN_SHAPE_RIGHTDEEP)
+ && !rightJn.IsBaseLevelJoinNode()) {
+ return PlanNode.NO_PLAN;
+ }
+
+ if (rightJn.cardinality * rightJn.size <= leftJn.cardinality * leftJn.size || hintBroadcastHashJoin != null
+ || joinEnum.forceJoinOrderMode
+ || !joinEnum.queryPlanShape.equals(AlgebricksConfig.QUERY_PLAN_SHAPE_ZIGZAG)) {
+ // We want to broadcast and build with the smaller side.
+ bcastHjCost = joinEnum.getCostMethodsHandle().costBroadcastHashJoin(this);
+ leftExchangeCost = joinEnum.getCostHandle().zeroCost();
+ rightExchangeCost = joinEnum.getCostMethodsHandle().computeBHJBuildExchangeCost(this);
+ childCosts = allPlans.get(leftPlan).totalCost.costAdd(allPlans.get(rightPlan).totalCost);
+ totalCost = bcastHjCost.costAdd(rightExchangeCost).costAdd(childCosts);
+ if (this.cheapestPlanIndex == PlanNode.NO_PLAN || totalCost.costLT(this.cheapestPlanCost)) {
+ pn = new PlanNode(allPlans.size(), joinEnum);
+ pn.jn = this;
+ pn.jnIndexes[0] = leftJn.jnArrayIndex;
+ pn.jnIndexes[1] = rightJn.jnArrayIndex;
+ pn.planIndexes[0] = leftPlan;
+ pn.planIndexes[1] = rightPlan;
+ pn.joinOp = PlanNode.JoinMethod.BROADCAST_HASH_JOIN; // need to check that all the conditions have equality predicates ONLY.
+ if (hintBroadcastHashJoin != null) {
+ hintBroadcastHashJoin.setBroadcastSide(BroadcastExpressionAnnotation.BroadcastSide.RIGHT);
+ }
+ pn.side = HashJoinExpressionAnnotation.BuildSide.RIGHT;
+ pn.joinExpr = hashJoinExpr;
+ pn.opCost = bcastHjCost;
+ pn.totalCost = totalCost;
+ pn.leftExchangeCost = leftExchangeCost;
+ pn.rightExchangeCost = rightExchangeCost;
+
+ allPlans.add(pn);
+ this.planIndexesArray.add(allPlans.size() - 1);
+ this.cheapestPlanCost = totalCost;
+ this.cheapestPlanIndex = allPlans.size() - 1;
+ return this.cheapestPlanIndex;
+ }
+ }
+
+ return PlanNode.NO_PLAN;
+ }
+
+ protected int buildNLJoinPlan(JoinNode leftJn, JoinNode rightJn, ILogicalExpression nestedLoopJoinExpr)
+ throws AlgebricksException {
+ // Build a nested loops plan, first check if it is possible
+ // left right order must be preserved and right side should be a single data set
+ List<PlanNode> allPlans = joinEnum.allPlans;
+ int numberOfTerms = joinEnum.numberOfTerms;
+ PlanNode pn;
+ ICost nljCost, leftExchangeCost, rightExchangeCost, childCosts, totalCost;
+
+ this.leftJn = leftJn;
+ this.rightJn = rightJn;
+ int leftPlan = leftJn.cheapestPlanIndex;
+ int rightPlan = rightJn.cheapestPlanIndex;
+ if (rightJn.jnArrayIndex > numberOfTerms) {
+ // right side consists of more than one table
+ return PlanNode.NO_PLAN; // nested loop plan not possible.
+ }
+
+ if (nestedLoopJoinExpr == null || !rightJn.nestedLoopsApplicable(nestedLoopJoinExpr)) {
+ return PlanNode.NO_PLAN;
+ }
+
+ nljCost = joinEnum.getCostMethodsHandle().costIndexNLJoin(this);
+ leftExchangeCost = joinEnum.getCostMethodsHandle().computeNLJOuterExchangeCost(this);
+ rightExchangeCost = joinEnum.getCostHandle().zeroCost();
+ childCosts = allPlans.get(leftPlan).totalCost;
+ totalCost = nljCost.costAdd(leftExchangeCost).costAdd(childCosts);
+ if (this.cheapestPlanIndex == PlanNode.NO_PLAN || totalCost.costLT(this.cheapestPlanCost)) {
+ pn = new PlanNode(allPlans.size(), joinEnum);
+ pn.jn = this;
+ pn.jnIndexes[0] = leftJn.jnArrayIndex;
+ pn.jnIndexes[1] = rightJn.jnArrayIndex;
+ pn.planIndexes[0] = leftPlan;
+ pn.planIndexes[1] = rightPlan;
+ pn.joinOp = PlanNode.JoinMethod.INDEX_NESTED_LOOP_JOIN;
+ pn.joinExpr = nestedLoopJoinExpr; // save it so can be used to add the NESTED annotation in getNewTree.
+ pn.opCost = nljCost;
+ pn.totalCost = totalCost;
+ pn.leftExchangeCost = leftExchangeCost;
+ pn.rightExchangeCost = rightExchangeCost;
+
+ allPlans.add(pn);
+ this.planIndexesArray.add(allPlans.size() - 1);
+ this.cheapestPlanCost = totalCost;
+ this.cheapestPlanIndex = allPlans.size() - 1;
+ return allPlans.size() - 1;
+ }
+ return PlanNode.NO_PLAN;
+ }
+
+ protected int buildCPJoinPlan(JoinNode leftJn, JoinNode rightJn, ILogicalExpression hashJoinExpr,
+ ILogicalExpression nestedLoopJoinExpr) {
+ // Now build a cartesian product nested loops plan
+ List<PlanNode> allPlans = joinEnum.allPlans;
+ PlanNode pn;
+ ICost cpCost, leftExchangeCost, rightExchangeCost, childCosts, totalCost;
+
+ this.leftJn = leftJn;
+ this.rightJn = rightJn;
+ int leftPlan = leftJn.cheapestPlanIndex;
+ int rightPlan = rightJn.cheapestPlanIndex;
+
+ ILogicalExpression cpJoinExpr = null;
+ List<Integer> newJoinConditions = this.getNewJoinConditionsOnly();
+ if (hashJoinExpr == null && nestedLoopJoinExpr == null) {
+ cpJoinExpr = joinEnum.combineAllConditions(newJoinConditions);
+ } else if (hashJoinExpr != null && nestedLoopJoinExpr == null) {
+ cpJoinExpr = hashJoinExpr;
+ } else if (hashJoinExpr == null) {
+ cpJoinExpr = nestedLoopJoinExpr;
+ } else if (Objects.equals(hashJoinExpr, nestedLoopJoinExpr)) {
+ cpJoinExpr = hashJoinExpr;
+ } else {
+ ScalarFunctionCallExpression andExpr = new ScalarFunctionCallExpression(
+ BuiltinFunctions.getBuiltinFunctionInfo(AlgebricksBuiltinFunctions.AND));
+ andExpr.getArguments().add(new MutableObject<>(hashJoinExpr));
+ andExpr.getArguments().add(new MutableObject<>(nestedLoopJoinExpr));
+ cpJoinExpr = andExpr;
+ }
+
+ cpCost = joinEnum.getCostMethodsHandle().costCartesianProductJoin(this);
+ leftExchangeCost = joinEnum.getCostHandle().zeroCost();
+ rightExchangeCost = joinEnum.getCostMethodsHandle().computeCPRightExchangeCost(this);
+ childCosts = allPlans.get(leftPlan).totalCost.costAdd(allPlans.get(rightPlan).totalCost);
+ totalCost = cpCost.costAdd(rightExchangeCost).costAdd(childCosts);
+ if (this.cheapestPlanIndex == PlanNode.NO_PLAN || totalCost.costLT(this.cheapestPlanCost)) {
+ pn = new PlanNode(allPlans.size(), joinEnum);
+ pn.jn = this;
+ pn.jnIndexes[0] = leftJn.jnArrayIndex;
+ pn.jnIndexes[1] = rightJn.jnArrayIndex;
+ pn.planIndexes[0] = leftPlan;
+ pn.planIndexes[1] = rightPlan;
+ pn.joinOp = PlanNode.JoinMethod.CARTESIAN_PRODUCT_JOIN;
+ pn.joinExpr = Objects.requireNonNullElse(cpJoinExpr, ConstantExpression.TRUE);
+ pn.opCost = cpCost;
+ pn.totalCost = totalCost;
+ pn.leftExchangeCost = leftExchangeCost;
+ pn.rightExchangeCost = rightExchangeCost;
+
+ allPlans.add(pn);
+ this.planIndexesArray.add(allPlans.size() - 1);
+ this.cheapestPlanCost = totalCost;
+ this.cheapestPlanIndex = allPlans.size() - 1;
+ return allPlans.size() - 1;
+ }
+ return PlanNode.NO_PLAN;
+ }
+
+ protected Pair<Integer, ICost> addMultiDatasetPlans(JoinNode leftJn, JoinNode rightJn) throws AlgebricksException {
+ this.leftJn = leftJn;
+ this.rightJn = rightJn;
+ ICost noJoinCost = joinEnum.getCostHandle().maxCost();
+
+ if (leftJn.planIndexesArray.size() == 0 || rightJn.planIndexesArray.size() == 0) {
+ return new Pair<>(PlanNode.NO_PLAN, noJoinCost);
+ }
+
+ if (this.cardinality >= Cost.MAX_CARD) {
+ return new Pair<>(PlanNode.NO_PLAN, noJoinCost); // no card hint available, so do not add this plan
+ }
+
+ List<Integer> newJoinConditions = this.getNewJoinConditionsOnly(); // these will be a subset of applicable join conditions.
+ ILogicalExpression hashJoinExpr = joinEnum.getHashJoinExpr(newJoinConditions);
+ ILogicalExpression nestedLoopJoinExpr = joinEnum.getNestedLoopJoinExpr(newJoinConditions);
+
+ if ((newJoinConditions.size() == 0) && joinEnum.connectedJoinGraph) {
+ // at least one plan must be there at each level as the graph is fully connected.
+ if (leftJn.cardinality * rightJn.cardinality > 10000.0) {
+ return new Pair<>(PlanNode.NO_PLAN, noJoinCost);
+ }
+ }
+
+ double current_card = this.cardinality;
+ if (current_card >= Cost.MAX_CARD) {
+ return new Pair<>(PlanNode.NO_PLAN, noJoinCost); // no card hint available, so do not add this plan
+ }
+
+ int hjPlan, commutativeHjPlan, bcastHjPlan, commutativeBcastHjPlan, nljPlan, commutativeNljPlan, cpPlan,
+ commutativeCpPlan;
+ hjPlan = commutativeHjPlan = bcastHjPlan =
+ commutativeBcastHjPlan = nljPlan = commutativeNljPlan = cpPlan = commutativeCpPlan = PlanNode.NO_PLAN;
+
+ HashJoinExpressionAnnotation hintHashJoin = joinEnum.findHashJoinHint(newJoinConditions);
+ BroadcastExpressionAnnotation hintBroadcastHashJoin = joinEnum.findBroadcastHashJoinHint(newJoinConditions);
+ IndexedNLJoinExpressionAnnotation hintNLJoin = joinEnum.findNLJoinHint(newJoinConditions);
+
+ if (leftJn.cheapestPlanIndex == PlanNode.NO_PLAN || rightJn.cheapestPlanIndex == PlanNode.NO_PLAN) {
+ return new Pair<>(PlanNode.NO_PLAN, noJoinCost);
+ }
+
+ if (hintHashJoin != null) {
+ boolean build = (hintHashJoin.getBuildOrProbe() == HashJoinExpressionAnnotation.BuildOrProbe.BUILD);
+ boolean probe = (hintHashJoin.getBuildOrProbe() == HashJoinExpressionAnnotation.BuildOrProbe.PROBE);
+ boolean validBuildOrProbeObject = false;
+ String buildOrProbeObject = hintHashJoin.getName();
+ if (buildOrProbeObject != null && (rightJn.datasetNames.contains(buildOrProbeObject)
+ || rightJn.aliases.contains(buildOrProbeObject) || leftJn.datasetNames.contains(buildOrProbeObject)
+ || leftJn.aliases.contains(buildOrProbeObject))) {
+ validBuildOrProbeObject = true;
+ }
+ if (validBuildOrProbeObject) {
+ if ((build && (rightJn.datasetNames.contains(buildOrProbeObject)
+ || rightJn.aliases.contains(buildOrProbeObject)))
+ || (probe && (leftJn.datasetNames.contains(buildOrProbeObject)
+ || leftJn.aliases.contains(buildOrProbeObject)))) {
+ hjPlan = buildHashJoinPlan(leftJn, rightJn, hashJoinExpr, hintHashJoin);
+ } else if ((build && (leftJn.datasetNames.contains(buildOrProbeObject)
+ || leftJn.aliases.contains(buildOrProbeObject)))
+ || (probe && (rightJn.datasetNames.contains(buildOrProbeObject)
+ || rightJn.aliases.contains(buildOrProbeObject)))) {
+ commutativeHjPlan = buildHashJoinPlan(rightJn, leftJn, hashJoinExpr, hintHashJoin);
+ }
+ } else {
+ // Hints are attached to predicates, so newJoinConditions should not be empty, but adding the check to be safe.
+ if (!joinEnum.getJoinConditions().isEmpty() && !newJoinConditions.isEmpty()) {
+ IWarningCollector warningCollector = joinEnum.optCtx.getWarningCollector();
+ if (warningCollector.shouldWarn()) {
+ warningCollector.warn(Warning.of(
+ joinEnum.getJoinConditions().get(newJoinConditions.get(0)).joinCondition
+ .getSourceLocation(),
+ ErrorCode.INAPPLICABLE_HINT, "hash join",
+ (build ? "build " : "probe ") + "with " + buildOrProbeObject));
+ }
+ }
+ hjPlan = buildHashJoinPlan(leftJn, rightJn, hashJoinExpr, null);
+ if (!joinEnum.forceJoinOrderMode) {
+ commutativeHjPlan = buildHashJoinPlan(rightJn, leftJn, hashJoinExpr, null);
+ }
+ bcastHjPlan = buildBroadcastHashJoinPlan(leftJn, rightJn, hashJoinExpr, null);
+ if (!joinEnum.forceJoinOrderMode) {
+ commutativeBcastHjPlan = buildBroadcastHashJoinPlan(rightJn, leftJn, hashJoinExpr, null);
+ }
+ nljPlan = buildNLJoinPlan(leftJn, rightJn, nestedLoopJoinExpr);
+ if (!joinEnum.forceJoinOrderMode) {
+ commutativeNljPlan = buildNLJoinPlan(rightJn, leftJn, nestedLoopJoinExpr);
+ }
+ cpPlan = buildCPJoinPlan(leftJn, rightJn, hashJoinExpr, nestedLoopJoinExpr);
+ if (!joinEnum.forceJoinOrderMode) {
+ commutativeCpPlan = buildCPJoinPlan(rightJn, leftJn, hashJoinExpr, nestedLoopJoinExpr);
+ }
+ }
+ } else if (hintBroadcastHashJoin != null) {
+ boolean validBroadcastObject = false;
+ String broadcastObject = hintBroadcastHashJoin.getName();
+ if (broadcastObject != null && (rightJn.datasetNames.contains(broadcastObject)
+ || rightJn.aliases.contains(broadcastObject) || leftJn.datasetNames.contains(broadcastObject)
+ || leftJn.aliases.contains(broadcastObject))) {
+ validBroadcastObject = true;
+ }
+ if (validBroadcastObject) {
+ if (rightJn.datasetNames.contains(broadcastObject) || rightJn.aliases.contains(broadcastObject)) {
+ bcastHjPlan = buildBroadcastHashJoinPlan(leftJn, rightJn, hashJoinExpr, hintBroadcastHashJoin);
+ } else if (leftJn.datasetNames.contains(broadcastObject) || leftJn.aliases.contains(broadcastObject)) {
+ commutativeBcastHjPlan =
+ buildBroadcastHashJoinPlan(rightJn, leftJn, hashJoinExpr, hintBroadcastHashJoin);
+ }
+ } else if (broadcastObject == null) {
+ bcastHjPlan = buildBroadcastHashJoinPlan(leftJn, rightJn, hashJoinExpr, hintBroadcastHashJoin);
+ if (!joinEnum.forceJoinOrderMode) {
+ commutativeBcastHjPlan =
+ buildBroadcastHashJoinPlan(rightJn, leftJn, hashJoinExpr, hintBroadcastHashJoin);
+ }
+ } else {
+ // Hints are attached to predicates, so newJoinConditions should not be empty, but adding the check to be safe.
+ if (!joinEnum.getJoinConditions().isEmpty() && !newJoinConditions.isEmpty()) {
+ IWarningCollector warningCollector = joinEnum.optCtx.getWarningCollector();
+ if (warningCollector.shouldWarn()) {
+ warningCollector.warn(Warning.of(
+ joinEnum.getJoinConditions().get(newJoinConditions.get(0)).joinCondition
+ .getSourceLocation(),
+ ErrorCode.INAPPLICABLE_HINT, "broadcast hash join", "broadcast " + broadcastObject));
+ }
+ }
+
+ hjPlan = buildHashJoinPlan(leftJn, rightJn, hashJoinExpr, null);
+ if (!joinEnum.forceJoinOrderMode) {
+ commutativeHjPlan = buildHashJoinPlan(rightJn, leftJn, hashJoinExpr, null);
+ }
+ bcastHjPlan = buildBroadcastHashJoinPlan(leftJn, rightJn, hashJoinExpr, null);
+ if (!joinEnum.forceJoinOrderMode) {
+ commutativeBcastHjPlan = buildBroadcastHashJoinPlan(rightJn, leftJn, hashJoinExpr, null);
+ }
+ nljPlan = buildNLJoinPlan(leftJn, rightJn, nestedLoopJoinExpr);
+ if (!joinEnum.forceJoinOrderMode) {
+ commutativeNljPlan = buildNLJoinPlan(rightJn, leftJn, nestedLoopJoinExpr);
+ }
+ cpPlan = buildCPJoinPlan(leftJn, rightJn, hashJoinExpr, nestedLoopJoinExpr);
+ if (!joinEnum.forceJoinOrderMode) {
+ commutativeCpPlan = buildCPJoinPlan(rightJn, leftJn, hashJoinExpr, nestedLoopJoinExpr);
+ }
+ }
+ } else if (hintNLJoin != null) {
+ nljPlan = buildNLJoinPlan(leftJn, rightJn, nestedLoopJoinExpr);
+ if (!joinEnum.forceJoinOrderMode) {
+ commutativeNljPlan = buildNLJoinPlan(rightJn, leftJn, nestedLoopJoinExpr);
+ }
+ if (nljPlan == PlanNode.NO_PLAN && commutativeNljPlan == PlanNode.NO_PLAN) {
+ // Hints are attached to predicates, so newJoinConditions should not be empty, but adding the check to be safe.
+ if (!joinEnum.getJoinConditions().isEmpty() && !newJoinConditions.isEmpty()) {
+ IWarningCollector warningCollector = joinEnum.optCtx.getWarningCollector();
+ if (warningCollector.shouldWarn()) {
+ warningCollector.warn(Warning.of(
+ joinEnum.getJoinConditions().get(newJoinConditions.get(0)).joinCondition
+ .getSourceLocation(),
+ ErrorCode.INAPPLICABLE_HINT, "index nested loop join", "ignored"));
+ }
+ }
+ hjPlan = buildHashJoinPlan(leftJn, rightJn, hashJoinExpr, null);
+ if (!joinEnum.forceJoinOrderMode) {
+ commutativeHjPlan = buildHashJoinPlan(rightJn, leftJn, hashJoinExpr, null);
+ }
+ bcastHjPlan = buildBroadcastHashJoinPlan(leftJn, rightJn, hashJoinExpr, null);
+ if (!joinEnum.forceJoinOrderMode) {
+ commutativeBcastHjPlan = buildBroadcastHashJoinPlan(rightJn, leftJn, hashJoinExpr, null);
+ }
+ cpPlan = buildCPJoinPlan(leftJn, rightJn, hashJoinExpr, nestedLoopJoinExpr);
+ if (!joinEnum.forceJoinOrderMode) {
+ commutativeCpPlan = buildCPJoinPlan(rightJn, leftJn, hashJoinExpr, nestedLoopJoinExpr);
+ }
+ }
+ } else {
+ hjPlan = buildHashJoinPlan(leftJn, rightJn, hashJoinExpr, null);
+ if (!joinEnum.forceJoinOrderMode) {
+ commutativeHjPlan = buildHashJoinPlan(rightJn, leftJn, hashJoinExpr, null);
+ }
+ bcastHjPlan = buildBroadcastHashJoinPlan(leftJn, rightJn, hashJoinExpr, null);
+ if (!joinEnum.forceJoinOrderMode) {
+ commutativeBcastHjPlan = buildBroadcastHashJoinPlan(rightJn, leftJn, hashJoinExpr, null);
+ }
+ nljPlan = buildNLJoinPlan(leftJn, rightJn, nestedLoopJoinExpr);
+ if (!joinEnum.forceJoinOrderMode) {
+ commutativeNljPlan = buildNLJoinPlan(rightJn, leftJn, nestedLoopJoinExpr);
+ }
+ cpPlan = buildCPJoinPlan(leftJn, rightJn, hashJoinExpr, nestedLoopJoinExpr);
+ if (!joinEnum.forceJoinOrderMode) {
+ commutativeCpPlan = buildCPJoinPlan(rightJn, leftJn, hashJoinExpr, nestedLoopJoinExpr);
+ }
+ }
+
+ if (hjPlan == PlanNode.NO_PLAN && commutativeHjPlan == PlanNode.NO_PLAN && bcastHjPlan == PlanNode.NO_PLAN
+ && commutativeBcastHjPlan == PlanNode.NO_PLAN && nljPlan == PlanNode.NO_PLAN
+ && commutativeNljPlan == PlanNode.NO_PLAN && cpPlan == PlanNode.NO_PLAN
+ && commutativeCpPlan == PlanNode.NO_PLAN) {
+ return new Pair<>(PlanNode.NO_PLAN, noJoinCost);
+ }
+
+ //Reset as these might have changed when we tried the commutative joins.
+ this.leftJn = leftJn;
+ this.rightJn = rightJn;
+
+ return new Pair<>(this.cheapestPlanIndex, this.cheapestPlanCost);
+ }
+
+ @Override
+ public String toString() {
+ if (planIndexesArray.isEmpty()) {
+ return "";
+ }
+ List<PlanNode> allPlans = joinEnum.allPlans;
+ StringBuilder sb = new StringBuilder(128);
+ // This will avoid printing JoinNodes that have no plans
+ sb.append("Printing Join Node ").append(jnArrayIndex).append('\n');
+ sb.append("datasetNames ").append('\n');
+ for (String datasetName : datasetNames) {
+ // Need to not print newline
+ sb.append(datasetName).append(' ');
+ }
+ sb.append("datasetIndex ").append('\n');
+ for (int j = 0; j < datasetIndexes.size(); j++) {
+ sb.append(j).append(datasetIndexes.get(j)).append('\n');
+ }
+ sb.append("datasetBits is ").append(datasetBits).append('\n');
+ if (IsBaseLevelJoinNode()) {
+ sb.append("orig cardinality is ").append((double) Math.round(origCardinality * 100) / 100).append('\n');
+ }
+ sb.append("cardinality is ").append((double) Math.round(cardinality * 100) / 100).append('\n');
+ if (planIndexesArray.size() == 0) {
+ sb.append("No plans considered for this join node").append('\n');
+ }
+ for (int j = 0; j < planIndexesArray.size(); j++) {
+ int k = planIndexesArray.get(j);
+ PlanNode pn = allPlans.get(k);
+ sb.append("planIndexesArray [").append(j).append("] is ").append(k).append('\n');
+ sb.append("Printing PlanNode ").append(k).append('\n');
+ if (IsBaseLevelJoinNode()) {
+ sb.append("DATA_SOURCE_SCAN").append('\n');
+ } else {
+ sb.append("\n");
+ sb.append(pn.joinMethod().getFirst()).append('\n');
+ sb.append("Printing Join expr ").append('\n');
+ if (pn.joinExpr != null) {
+ sb.append(pn.joinExpr).append('\n');
+ } else {
+ sb.append("null").append('\n');
+ }
+ }
+ sb.append("card ").append((double) Math.round(cardinality * 100) / 100).append('\n');
+ sb.append("------------------").append('\n');
+ sb.append("operator cost ").append(pn.opCost.computeTotalCost()).append('\n');
+ sb.append("total cost ").append(pn.totalCost.computeTotalCost()).append('\n');
+ sb.append("jnIndexes ").append(pn.jnIndexes[0]).append(" ").append(pn.jnIndexes[1]).append('\n');
+ if (IsHigherLevelJoinNode()) {
+ PlanNode leftPlan = pn.getLeftPlanNode();
+ PlanNode rightPlan = pn.getRightPlanNode();
+ int l = leftPlan.allPlansIndex;
+ int r = rightPlan.allPlansIndex;
+ sb.append("planIndexes ").append(l).append(" ").append(r).append('\n');
+ sb.append("(lcost = ").append(leftPlan.totalCost.computeTotalCost()).append(") (rcost = ")
+ .append(rightPlan.totalCost.computeTotalCost()).append(")").append('\n');
+ }
+ sb.append("\n");
+ }
+ sb.append("jnIndex ").append(jnIndex).append('\n');
+ sb.append("datasetBits ").append(datasetBits).append('\n');
+ sb.append("cardinality ").append((double) Math.round(cardinality * 100) / 100).append('\n');
+ sb.append("size ").append((double) Math.round(size * 100) / 100).append('\n');
+ sb.append("level ").append(level).append('\n');
+ sb.append("highestDatasetId ").append(highestDatasetId).append('\n');
+ sb.append("--------------------------------------").append('\n');
+ return sb.toString();
+ }
+
+ public void printCostOfAllPlans(StringBuilder sb) {
+ List<PlanNode> allPlans = joinEnum.allPlans;
+ ICost minCost = joinEnum.getCostHandle().maxCost();
+ for (int planIndex : planIndexesArray) {
+ ICost planCost = allPlans.get(planIndex).totalCost;
+ sb.append("plan ").append(planIndex).append(" cost is ").append(planCost.computeTotalCost()).append('\n');
+ if (planCost.costLT(minCost)) {
+ minCost = planCost;
+ }
+ }
+ sb.append("LOWEST COST ").append(minCost.computeTotalCost()).append('\n');
+ }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/PlanNode.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/PlanNode.java
new file mode 100644
index 0000000..7e9c3ee
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/PlanNode.java
@@ -0,0 +1,220 @@
+/*
+ * 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.optimizer.rules.cbo;
+
+import org.apache.asterix.optimizer.cost.ICost;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.HashJoinExpressionAnnotation;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
+
+public class PlanNode {
+
+ public static int NO_PLAN = -1;
+
+ private final JoinEnum joinEnum;
+ int allPlansIndex;
+ int[] planIndexes;
+ int[] jnIndexes;
+ JoinNode jn;
+ String datasetName;
+ ICost opCost;
+ ICost totalCost;
+ ICost leftExchangeCost;
+ ICost rightExchangeCost;
+ JoinMethod joinOp;
+ // Used to indicate which side to build for HJ and which side to broadcast for BHJ.
+ HashJoinExpressionAnnotation.BuildSide side;
+ ScanMethod scanOp;
+ ILogicalExpression joinExpr;
+ DataSourceScanOperator correspondingDataSourceScanOp;
+ EmptyTupleSourceOperator correspondingEmptyTupleSourceOp;
+
+ public enum ScanMethod {
+ INDEX_SCAN,
+ TABLE_SCAN
+ }
+
+ public enum JoinMethod {
+ HYBRID_HASH_JOIN,
+ BROADCAST_HASH_JOIN,
+ INDEX_NESTED_LOOP_JOIN,
+ CARTESIAN_PRODUCT_JOIN
+ }
+
+ public PlanNode(int planIndex, JoinEnum joinE) {
+ this.allPlansIndex = planIndex;
+ joinEnum = joinE;
+ planIndexes = new int[2]; // 0 is for left, 1 is for right
+ jnIndexes = new int[2]; // join node index(es)
+ }
+
+ public int getIndex() {
+ return allPlansIndex;
+ }
+
+ public int[] getPlanIndexes() {
+ return planIndexes;
+ }
+
+ public int getLeftPlanIndex() {
+ return planIndexes[0];
+ }
+
+ public PlanNode getLeftPlanNode() {
+ if (planIndexes[0] == NO_PLAN) {
+ return null;
+ }
+ return joinEnum.allPlans.get(planIndexes[0]);
+ }
+
+ public PlanNode getRightPlanNode() {
+ if (planIndexes[1] == NO_PLAN) {
+ return null;
+ }
+ return joinEnum.allPlans.get(planIndexes[1]);
+ }
+
+ public JoinNode getJoinNode() {
+ return jn;
+ }
+
+ public void setJoinNode(JoinNode jn) {
+ this.jn = jn;
+ }
+
+ public int getRightPlanIndex() {
+ return planIndexes[1];
+ }
+
+ public void setRightPlanIndex(int index) {
+ this.planIndexes[1] = index;
+ }
+
+ public int getLeftJoinIndex() {
+ return jnIndexes[0];
+ }
+
+ public void setLeftPlanIndex(int index) {
+ this.planIndexes[0] = index;
+ }
+
+ public void setLeftJoinIndex(int index) {
+ this.jnIndexes[0] = index;
+ }
+
+ public void setRightJoinIndex(int index) {
+ this.jnIndexes[1] = index;
+ }
+
+ public int getRightJoinIndex() {
+ return jnIndexes[1];
+ }
+
+ public boolean IsScanNode() {
+ return getLeftPlanIndex() == NO_PLAN && getRightPlanIndex() == NO_PLAN;
+ }
+
+ public boolean IsJoinNode() {
+ return getLeftPlanIndex() != NO_PLAN && getRightPlanIndex() != NO_PLAN;
+ }
+
+ public Pair<String, String> joinMethod() {
+ if (this.joinOp == PlanNode.JoinMethod.HYBRID_HASH_JOIN) {
+ return new Pair<>("HASH JOIN", "HJ");
+ } else if (this.joinOp == PlanNode.JoinMethod.BROADCAST_HASH_JOIN) {
+ return new Pair<>("BROADCAST HASH JOIN", "BHJ");
+ } else if (this.joinOp == PlanNode.JoinMethod.INDEX_NESTED_LOOP_JOIN) {
+ return new Pair<>("INDEX NESTED LOOPS JOIN", "INLJ");
+ } else if (this.joinOp == PlanNode.JoinMethod.CARTESIAN_PRODUCT_JOIN) {
+ return new Pair<>("CARTESIAN PRODUCT JOIN", "CPJ");
+ }
+ return new Pair<>("", "");
+ }
+
+ public String getDatasetName() {
+ return datasetName;
+ }
+
+ public void setDatasetName(String dsName) {
+ this.datasetName = dsName;
+ }
+
+ public DataSourceScanOperator getDataSourceScanOp() {
+ return correspondingDataSourceScanOp; // This applies only to singleDataSetPlans
+ }
+
+ public EmptyTupleSourceOperator getEmptyTupleSourceOp() {
+ return correspondingEmptyTupleSourceOp; // This applies only to singleDataSetPlans
+ }
+
+ public void setEmptyTupleSourceOp(EmptyTupleSourceOperator emptyTupleSourceOp) {
+ this.correspondingEmptyTupleSourceOp = emptyTupleSourceOp; // This applies only to singleDataSetPlans
+ }
+
+ public ICost getOpCost() {
+ return opCost;
+ }
+
+ public void setOpCost(ICost cost) {
+ this.opCost = cost;
+ }
+
+ public double computeOpCost() {
+ return opCost.computeTotalCost();
+ }
+
+ public ICost getTotalCost() {
+ return totalCost;
+ }
+
+ public void setTotalCost(ICost tc) {
+ this.totalCost = tc;
+ }
+
+ public ICost getLeftExchangeCost() {
+ return leftExchangeCost;
+ }
+
+ public ICost getRightExchangeCost() {
+ return rightExchangeCost;
+ }
+
+ public double computeTotalCost() {
+ return totalCost.computeTotalCost();
+ }
+
+ public ScanMethod getScanOp() {
+ return scanOp;
+ }
+
+ public void setScanMethod(ScanMethod sm) {
+ this.scanOp = sm;
+ }
+
+ public JoinMethod getJoinOp() {
+ return joinOp;
+ }
+
+ public ILogicalExpression getJoinExpr() {
+ return joinExpr;
+ }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/Stats.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/Stats.java
new file mode 100644
index 0000000..38d817b
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/Stats.java
@@ -0,0 +1,244 @@
+/*
+ * 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.optimizer.rules.cbo;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.metadata.declared.DataSource;
+import org.apache.asterix.metadata.declared.DataSourceId;
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.metadata.entities.Index;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.JoinProductivityAnnotation;
+import org.apache.hyracks.algebricks.core.algebra.expressions.PredicateCardinalityAnnotation;
+import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
+import org.apache.hyracks.api.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.Warning;
+
+public class Stats {
+
+ public double SELECTIVITY_FOR_SECONDARY_INDEX_SELECTION = 0.1;
+
+ protected IOptimizationContext optCtx;
+ protected JoinEnum joinEnum;
+
+ public Stats(IOptimizationContext context, JoinEnum joinE) {
+ optCtx = context;
+ joinEnum = joinE;
+ }
+
+ public DataverseName findDataverseName(DataSourceScanOperator scanOp) {
+ if (scanOp == null) {
+ // this should rarely happen (IN lists may cause this)
+ return null;
+ }
+ DataSourceId dsid = (DataSourceId) scanOp.getDataSource().getId();
+ return dsid.getDataverseName();
+ }
+
+ public Index findSampleIndex(DataSourceScanOperator scanOp, IOptimizationContext context)
+ throws AlgebricksException {
+ DataverseName dataverseName = findDataverseName(scanOp);
+ DataSource ds = (DataSource) scanOp.getDataSource();
+ DataSourceId dsid = ds.getId();
+ MetadataProvider mdp = (MetadataProvider) context.getMetadataProvider();
+ return mdp.findSampleIndex(dataverseName, dsid.getDatasourceName());
+ }
+
+ private double findJoinSelectivity(JoinProductivityAnnotation anno, AbstractFunctionCallExpression joinExpr)
+ throws AlgebricksException {
+ List<LogicalVariable> exprUsedVars = new ArrayList<>();
+ joinExpr.getUsedVariables(exprUsedVars);
+ if (exprUsedVars.size() != 2) {
+ // Since there is a left and right dataset here, expecting only two variables.
+ return 1.0;
+ }
+ int idx1 = joinEnum.findJoinNodeIndex(exprUsedVars.get(0)) + 1;
+ int idx2 = joinEnum.findJoinNodeIndex(exprUsedVars.get(1)) + 1;
+ double card1 = joinEnum.getJnArray()[idx1].origCardinality;
+ double card2 = joinEnum.getJnArray()[idx2].origCardinality;
+ if (card1 == 0.0 || card2 == 0.0) // should not happen
+ {
+ return 1.0;
+ }
+
+ // join sel = leftside * productivity/(card1 * card2);
+ if (anno != null) {
+ int leftIndex = joinEnum.findJoinNodeIndexByName(anno.getLeftSideDataSet());
+ if (leftIndex != idx1 && leftIndex != idx2) {
+ // should not happen
+ IWarningCollector warningCollector = joinEnum.optCtx.getWarningCollector();
+ if (warningCollector.shouldWarn()) {
+ warningCollector.warn(Warning.of(joinExpr.getSourceLocation(), ErrorCode.INAPPLICABLE_HINT,
+ "productivity", "Invalid collection name/alias: " + anno.getLeftSideDataSet()));
+ }
+ return 1.0;
+ }
+ double productivity = anno.getJoinProductivity();
+ if (productivity <= 0) {
+ IWarningCollector warningCollector = joinEnum.optCtx.getWarningCollector();
+ if (warningCollector.shouldWarn()) {
+ warningCollector.warn(Warning.of(joinExpr.getSourceLocation(), ErrorCode.INAPPLICABLE_HINT,
+ "productivity",
+ "Productivity specified: " + productivity + ", has to be a decimal value greater than 0"));
+ }
+ return 1.0;
+ }
+ if (leftIndex == idx1) {
+ return productivity / card2;
+ } else {
+ return productivity / card1;
+ }
+ } else {
+ if (card1 < card2) {
+ // we are assuming that the smaller side is the primary side and that the join is Pk-Fk join.
+ return 1.0 / card1;
+ }
+ return 1.0 / card2;
+ }
+ }
+
+ // The expression we get may not be a base condition. It could be comprised of ors and ands and nots. So have to
+ //recursively find the overall selectivity.
+ protected double getSelectivityFromAnnotation(AbstractFunctionCallExpression afcExpr, boolean join)
+ throws AlgebricksException {
+ double sel = 1.0;
+
+ if (afcExpr.getFunctionIdentifier().equals(AlgebricksBuiltinFunctions.OR)) {
+ double orSel = 0.0;
+ for (int i = 0; i < afcExpr.getArguments().size(); i++) {
+ ILogicalExpression lexpr = afcExpr.getArguments().get(i).getValue();
+ if (lexpr.getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL)) {
+ sel = getSelectivityFromAnnotation(
+ (AbstractFunctionCallExpression) afcExpr.getArguments().get(i).getValue(), join);
+ orSel = orSel + sel - orSel * sel;
+ }
+ }
+ return orSel;
+ } else if (afcExpr.getFunctionIdentifier().equals(AlgebricksBuiltinFunctions.AND)) {
+ double andSel = 1.0;
+ for (int i = 0; i < afcExpr.getArguments().size(); i++) {
+ ILogicalExpression lexpr = afcExpr.getArguments().get(i).getValue();
+ if (lexpr.getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL)) {
+ sel = getSelectivityFromAnnotation(
+ (AbstractFunctionCallExpression) afcExpr.getArguments().get(i).getValue(), join);
+ andSel *= sel;
+ }
+ }
+ return andSel;
+ } else if (afcExpr.getFunctionIdentifier().equals(AlgebricksBuiltinFunctions.NOT)) {
+ ILogicalExpression lexpr = afcExpr.getArguments().get(0).getValue();
+ if (lexpr.getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL)) {
+ sel = getSelectivityFromAnnotation(
+ (AbstractFunctionCallExpression) afcExpr.getArguments().get(0).getValue(), join);
+ return 1.0 - sel;
+ }
+ }
+
+ double s = 1.0;
+ PredicateCardinalityAnnotation pca = afcExpr.getAnnotation(PredicateCardinalityAnnotation.class);
+ if (pca != null) {
+ s = pca.getSelectivity();
+ if (s <= 0 || s >= 1) {
+ IWarningCollector warningCollector = joinEnum.optCtx.getWarningCollector();
+ if (warningCollector.shouldWarn()) {
+ warningCollector.warn(Warning.of(afcExpr.getSourceLocation(), ErrorCode.INAPPLICABLE_HINT,
+ "selectivity", "Selectivity specified: " + s
+ + ", has to be a decimal value greater than 0 and less than 1"));
+ }
+ } else {
+ sel *= s;
+ }
+ } else {
+ JoinProductivityAnnotation jpa = afcExpr.getAnnotation(JoinProductivityAnnotation.class);
+ s = findJoinSelectivity(jpa, afcExpr);
+ sel *= s;
+ }
+ if (join && s == 1.0) {
+ // assume no selectivity was assigned
+ joinEnum.singleDatasetPreds.add(afcExpr);
+ }
+ return sel;
+ }
+
+ public double getSelectivityFromAnnotationMain(ILogicalExpression leExpr, boolean join) throws AlgebricksException {
+ double sel = 1.0;
+
+ if (leExpr.getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL)) {
+ AbstractFunctionCallExpression afcExpr = (AbstractFunctionCallExpression) leExpr;
+ sel = getSelectivityFromAnnotation(afcExpr, join);
+ }
+
+ return sel;
+ }
+
+ // The next two routines should be combined and made more general
+ protected double getSelectivity(ILogicalOperator op, boolean join) throws AlgebricksException {
+ double sel = 1.0; // safe to return 1 if there is no annotation
+
+ if (op == null) {
+ return sel;
+ }
+
+ // find all the selectOperators here.
+ while (op.getOperatorTag() != LogicalOperatorTag.EMPTYTUPLESOURCE) {
+ if (op.getOperatorTag() == LogicalOperatorTag.SELECT) {
+ SelectOperator selOper = (SelectOperator) op;
+ sel *= getSelectivityFromAnnotationMain(selOper.getCondition().getValue(), join);
+ }
+ if (op.getOperatorTag() == LogicalOperatorTag.SUBPLAN) {
+ sel *= getSelectivity((SubplanOperator) op);
+ }
+ op = op.getInputs().get(0).getValue();
+ }
+ return sel;
+ }
+
+ protected double getSelectivity(SubplanOperator subplanOp) throws AlgebricksException {
+ double sel = 1.0; // safe to return 1 if there is no annotation
+ //ILogicalOperator op = subplanOp;
+ ILogicalOperator op = subplanOp.getNestedPlans().get(0).getRoots().get(0).getValue();
+ while (true) {
+ if (op.getOperatorTag() == LogicalOperatorTag.SELECT) {
+ SelectOperator selOper = (SelectOperator) op;
+ sel *= getSelectivityFromAnnotationMain(selOper.getCondition().getValue(), false);
+ }
+ if (op.getInputs().size() > 0) {
+ op = op.getInputs().get(0).getValue();
+ } else {
+ break;
+ }
+ }
+ return sel;
+ }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AbstractLangTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AbstractLangTranslator.java
index ac7fabe..6a24d59 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AbstractLangTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AbstractLangTranslator.java
@@ -40,6 +40,8 @@
import org.apache.asterix.common.functions.FunctionSignature;
import org.apache.asterix.common.metadata.DataverseName;
import org.apache.asterix.lang.common.base.Statement;
+import org.apache.asterix.lang.common.statement.AnalyzeDropStatement;
+import org.apache.asterix.lang.common.statement.AnalyzeStatement;
import org.apache.asterix.lang.common.statement.CreateAdapterStatement;
import org.apache.asterix.lang.common.statement.CreateDataverseStatement;
import org.apache.asterix.lang.common.statement.CreateFeedStatement;
@@ -371,6 +373,27 @@
dataverseName);
}
break;
+
+ case ANALYZE:
+ AnalyzeStatement analyzeStmt = (AnalyzeStatement) stmt;
+ if (analyzeStmt.getDataverseName() != null) {
+ dataverseName = analyzeStmt.getDataverseName();
+ }
+ invalidOperation = isMetadataDataverse(dataverseName);
+ if (invalidOperation) {
+ message = String.format(BAD_DATAVERSE_OBJECT_DDL_MESSAGE, "analyze", dataset(), dataverseName);
+ }
+ break;
+ case ANALYZE_DROP:
+ AnalyzeDropStatement analyzeDropStmt = (AnalyzeDropStatement) stmt;
+ if (analyzeDropStmt.getDataverseName() != null) {
+ dataverseName = analyzeDropStmt.getDataverseName();
+ }
+ invalidOperation = isMetadataDataverse(dataverseName);
+ if (invalidOperation) {
+ message = String.format(BAD_DATAVERSE_OBJECT_DDL_MESSAGE, "analyze drop", dataset(), dataverseName);
+ }
+ break;
}
if (invalidOperation) {
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/BaseClientRequest.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/BaseClientRequest.java
index 99cda09..0fafc47 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/BaseClientRequest.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/BaseClientRequest.java
@@ -20,7 +20,6 @@
import org.apache.asterix.common.api.IClientRequest;
import org.apache.asterix.common.api.IRequestReference;
-import org.apache.asterix.common.api.RequestReference;
import org.apache.asterix.common.dataflow.ICcApplicationContext;
import org.apache.asterix.om.base.ADateTime;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -93,15 +92,25 @@
return JSONUtil.convertNodeUnchecked(asJson());
}
- protected ObjectNode asJson() {
+ @Override
+ public ObjectNode asJson() {
+ return putJson();
+ }
+
+ @Override
+ public ObjectNode asRedactedJson() {
+ return putJson();
+ }
+
+ private ObjectNode putJson() {
ObjectNode json = JSONUtil.createObject();
json.put("uuid", requestReference.getUuid());
json.put("requestTime", new ADateTime(requestReference.getTime()).toSimpleString());
json.put("elapsedTime", getElapsedTimeInSecs());
json.put("node", requestReference.getNode());
json.put("state", state.getLabel());
- json.put("userAgent", ((RequestReference) requestReference).getUserAgent());
- json.put("remoteAddr", ((RequestReference) requestReference).getRemoteAddr());
+ json.put("userAgent", requestReference.getUserAgent());
+ json.put("remoteAddr", requestReference.getRemoteAddr());
json.put("cancellable", cancellable);
return json;
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ClientRequest.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ClientRequest.java
index c19bb02..31f1979 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ClientRequest.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ClientRequest.java
@@ -18,11 +18,22 @@
*/
package org.apache.asterix.translator;
+import static org.apache.hyracks.api.job.resource.IJobCapacityController.JobSubmissionStatus.QUEUE;
+
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
import org.apache.asterix.common.api.ICommonRequestParameters;
import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.asterix.om.base.AMutableDateTime;
import org.apache.hyracks.api.client.IHyracksClientConnection;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.api.job.JobStatus;
+import org.apache.hyracks.api.job.resource.IJobCapacityController;
+import org.apache.hyracks.api.job.resource.IReadOnlyClusterCapacity;
+import org.apache.hyracks.api.util.ExceptionUtils;
+import org.apache.hyracks.util.LogRedactionUtil;
import com.fasterxml.jackson.databind.node.ObjectNode;
@@ -32,13 +43,16 @@
protected final Thread executor;
protected final String statement;
protected final String clientContextId;
+ protected final JobState jobState;
protected volatile JobId jobId;
+ private volatile String plan; // can be null
public ClientRequest(ICommonRequestParameters requestParameters) {
super(requestParameters.getRequestReference());
this.clientContextId = requestParameters.getClientContextId();
this.statement = requestParameters.getStatement();
this.executor = Thread.currentThread();
+ this.jobState = new JobState();
}
@Override
@@ -46,6 +60,10 @@
return clientContextId;
}
+ public void setPlan(String plan) {
+ this.plan = plan;
+ }
+
public synchronized void setJobId(JobId jobId) {
this.jobId = jobId;
setRunning();
@@ -76,11 +94,93 @@
}
@Override
- protected ObjectNode asJson() {
+ public ObjectNode asJson() {
ObjectNode json = super.asJson();
- json.put("jobId", jobId != null ? jobId.toString() : null);
- json.put("statement", statement);
+ return asJson(json, false);
+ }
+
+ @Override
+ public ObjectNode asRedactedJson() {
+ ObjectNode json = super.asRedactedJson();
+ return asJson(json, true);
+ }
+
+ private ObjectNode asJson(ObjectNode json, boolean redact) {
+ putJobDetails(json, redact);
+ json.put("statement", redact ? LogRedactionUtil.statement(statement) : statement);
json.put("clientContextID", clientContextId);
+ if (plan != null) {
+ json.put("plan", redact ? LogRedactionUtil.userData(plan) : plan);
+ }
return json;
}
+
+ @Override
+ public void jobCreated(JobId jobId, IReadOnlyClusterCapacity requiredClusterCapacity,
+ IJobCapacityController.JobSubmissionStatus status) {
+ jobState.createTime = System.currentTimeMillis();
+ jobState.status = status == QUEUE ? JobStatus.PENDING : JobStatus.RUNNING;
+ jobState.requiredCPUs = requiredClusterCapacity.getAggregatedCores();
+ jobState.requiredMemoryInBytes = requiredClusterCapacity.getAggregatedMemoryByteSize();
+ }
+
+ @Override
+ public void jobStarted(JobId jobId) {
+ jobState.startTime = System.currentTimeMillis();
+ jobState.status = JobStatus.RUNNING;
+ }
+
+ @Override
+ public void jobFinished(JobId jobId, JobStatus jobStatus, List<Exception> exceptions) {
+ jobState.endTime = System.currentTimeMillis();
+ jobState.status = jobStatus;
+ if (exceptions != null && !exceptions.isEmpty()) {
+ jobState.errorMsg = processException(exceptions.get(0));
+ }
+ }
+
+ protected String processException(Exception e) {
+ return ExceptionUtils.unwrap(e).getMessage();
+ }
+
+ private void putJobDetails(ObjectNode json, boolean redact) {
+ try {
+ json.put("jobId", jobId != null ? jobId.toString() : null);
+ putJobState(json, jobState, redact);
+ } catch (Throwable th) {
+ // ignore
+ }
+ }
+
+ private static void putJobState(ObjectNode json, JobState state, boolean redact) {
+ AMutableDateTime dateTime = new AMutableDateTime(0);
+ putTime(json, state.createTime, "jobCreateTime", dateTime);
+ putTime(json, state.startTime, "jobStartTime", dateTime);
+ putTime(json, state.endTime, "jobEndTime", dateTime);
+ long queueTime = (state.startTime > 0 ? state.startTime : System.currentTimeMillis()) - state.createTime;
+ json.put("jobQueueTime", TimeUnit.MILLISECONDS.toSeconds(queueTime));
+ json.put("jobStatus", String.valueOf(state.status));
+ json.put("jobRequiredCPUs", state.requiredCPUs);
+ json.put("jobRequiredMemory", state.requiredMemoryInBytes);
+ if (state.errorMsg != null) {
+ json.put("error", redact ? LogRedactionUtil.userData(state.errorMsg) : state.errorMsg);
+ }
+ }
+
+ private static void putTime(ObjectNode json, long time, String label, AMutableDateTime dateTime) {
+ if (time > 0) {
+ dateTime.setValue(time);
+ json.put(label, dateTime.toSimpleString());
+ }
+ }
+
+ static class JobState {
+ volatile long createTime;
+ volatile long startTime;
+ volatile long endTime;
+ volatile long requiredMemoryInBytes;
+ volatile int requiredCPUs;
+ volatile JobStatus status;
+ volatile String errorMsg;
+ }
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
index 794a0b7..acbd13e 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
@@ -940,7 +940,8 @@
}
if (!function.isExternal()) {
// all non-external UDFs should've been inlined by now
- throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc, signature);
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc,
+ "UDF not inlined: " + signature);
}
IFunctionInfo finfo = ExternalFunctionCompilerUtil.getExternalFunctionInfo(metadataProvider, function);
AbstractFunctionCallExpression f = new ScalarFunctionCallExpression(finfo, args);
@@ -1716,8 +1717,7 @@
default:
if (expressionNeedsNoNesting(expr)) {
Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this, topOpRef);
- ILogicalExpression exp = ((AssignOperator) p.first).getExpressions().get(0).getValue();
- return new Pair<>(exp, p.first.getInputs().get(0));
+ return inlineAssignIfPossible((AssignOperator) p.first);
} else {
Mutable<ILogicalOperator> srcRef = new MutableObject<>();
Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this, srcRef);
@@ -1749,9 +1749,35 @@
}
}
+ /**
+ * TODO(wyk) I believe that inlining expressions should be done at the optimization level and not at the translation
+ * level. By inlining at the translation level, we could possibly miss optimizing inlined expressions in rules
+ * that do not inspect arguments of a function. I kept inlining all pure (a.k.a functional) functions for now to
+ * match the previous behavior. For non-pure functions, the assign should be kept as we do not inline them at
+ * first due to ASTERIXDB-3103
+ *
+ * @see org.apache.hyracks.algebricks.rewriter.rules.InlineVariablesRule
+ */
+ private Pair<ILogicalExpression, Mutable<ILogicalOperator>> inlineAssignIfPossible(AssignOperator assignOp) {
+ ILogicalExpression expr = assignOp.getExpressions().get(0).getValue();
+
+ if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+ return new Pair<>(expr, assignOp.getInputs().get(0));
+ }
+
+ AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
+ if (funcExpr.isFunctional()) {
+ return new Pair<>(expr, assignOp.getInputs().get(0));
+ }
+
+ //Do not inline non-functional expressions (e.g. uuid()) and keep the assign
+ return new Pair<>(new VariableReferenceExpression(assignOp.getVariables().get(0)),
+ new MutableObject<>(assignOp));
+ }
+
protected Pair<ILogicalOperator, LogicalVariable> aggListifyForSubquery(LogicalVariable var,
Mutable<ILogicalOperator> opRef, boolean bProject) {
- SourceLocation sourceLoc = opRef.getValue().getSourceLocation();
+ SourceLocation sourceLoc = opRef.getValue() != null ? opRef.getValue().getSourceLocation() : null;
AggregateFunctionCallExpression funAgg =
BuiltinFunctions.makeAggregateFunctionExpression(BuiltinFunctions.LISTIFY, new ArrayList<>());
funAgg.getArguments().add(new MutableObject<>(new VariableReferenceExpression(var)));
@@ -1902,8 +1928,7 @@
* Eliminate shared operator references in a query plan. Deep copy a new query
* plan subtree whenever there is a shared operator reference.
*
- * @param plan,
- * the query plan.
+ * @param plan, the query plan.
* @throws CompilationException
*/
protected void eliminateSharedOperatorReferenceForPlan(ILogicalPlan plan) throws CompilationException {
@@ -1918,12 +1943,10 @@
* <code>currentOpRef.getValue()</code>. Deep copy a new query plan subtree
* whenever there is a shared operator reference.
*
- * @param currentOpRef,
- * the operator reference to consider
- * @param opRefSet,
- * the set storing seen operator references so far.
+ * @param currentOpRef, the operator reference to consider
+ * @param opRefSet, the set storing seen operator references so far.
* @return a mapping that maps old variables to new variables, for the ancestors
- * of <code>currentOpRef</code> to replace variables properly.
+ * of <code>currentOpRef</code> to replace variables properly.
* @throws CompilationException
*/
private LinkedHashMap<LogicalVariable, LogicalVariable> eliminateSharedOperatorReference(
@@ -2005,14 +2028,11 @@
/**
* Constructs a subplan operator for a branch in a if-else (or case) expression.
*
- * @param inputOp,
- * the input operator.
- * @param selectExpr,
- * the expression to select tuples that are processed by this branch.
- * @param branchExpression,
- * the expression to be evaluated in this branch.
+ * @param inputOp, the input operator.
+ * @param selectExpr, the expression to select tuples that are processed by this branch.
+ * @param branchExpression, the expression to be evaluated in this branch.
* @return a pair of the constructed subplan operator and the output variable
- * for the branch.
+ * for the branch.
* @throws CompilationException
*/
protected Pair<ILogicalOperator, LogicalVariable> constructSubplanOperatorForBranch(ILogicalOperator inputOp,
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ResultMetadata.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ResultMetadata.java
new file mode 100644
index 0000000..78f84ff
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ResultMetadata.java
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.translator;
+
+import java.util.List;
+import java.util.Set;
+
+import org.apache.hyracks.api.exceptions.Warning;
+import org.apache.hyracks.api.result.IResultMetadata;
+
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
+public class ResultMetadata implements IResultMetadata {
+ private static final long serialVersionUID = 1905367559307369034L;
+
+ private final SessionConfig.OutputFormat format;
+ private long jobDuration;
+ private long processedObjects;
+ private ObjectNode profile;
+ private long diskIoCount;
+ private Set<Warning> warnings;
+ private long totalWarningsCount;
+ private transient List<Object> outputTypes;
+
+ public ResultMetadata(SessionConfig.OutputFormat format) {
+ this.format = format;
+ }
+
+ public SessionConfig.OutputFormat getFormat() {
+ return format;
+ }
+
+ public long getProcessedObjects() {
+ return processedObjects;
+ }
+
+ public void setProcessedObjects(long processedObjects) {
+ this.processedObjects = processedObjects;
+ }
+
+ public void setJobDuration(long jobDuration) {
+ this.jobDuration = jobDuration;
+ }
+
+ public void setWarnings(Set<Warning> warnings) {
+ this.warnings = warnings;
+ }
+
+ /**
+ * Sets the count of all warnings generated including unreported ones.
+ */
+ public void setTotalWarningsCount(long totalWarningsCount) {
+ this.totalWarningsCount = totalWarningsCount;
+ }
+
+ public long getJobDuration() {
+ return jobDuration;
+ }
+
+ public void setJobProfile(ObjectNode profile) {
+ this.profile = profile;
+ }
+
+ public ObjectNode getJobProfile() {
+ return profile;
+ }
+
+ /**
+ * @return The reported warnings.
+ */
+ public Set<Warning> getWarnings() {
+ return warnings;
+ }
+
+ /**
+ * @return Total count of all warnings generated including unreported ones.
+ */
+ public long getTotalWarningsCount() {
+ return totalWarningsCount;
+ }
+
+ @Override
+ public void setOutputTypes(List<Object> typeList) {
+ this.outputTypes = typeList;
+ }
+
+ public List<Object> getOutputTypes() {
+ return outputTypes;
+ }
+
+ @Override
+ public String toString() {
+ return "ResultMetadata{" + "format=" + format + ", jobDuration=" + jobDuration + ", processedObjects="
+ + processedObjects + ", diskIoCount=" + diskIoCount + '}';
+ }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java
index 3c4537f..037aca3 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java
@@ -18,6 +18,8 @@
*/
package org.apache.asterix.translator;
+import static org.apache.asterix.external.util.ExternalDataConstants.SUBPATH;
+
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
@@ -28,6 +30,7 @@
import java.util.function.Predicate;
import org.apache.asterix.algebra.base.ILangExpressionToPlanTranslator;
+import org.apache.asterix.common.annotations.ExternalSubpathAnnotation;
import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.common.functions.FunctionSignature;
@@ -203,7 +206,9 @@
}
Pair<ILogicalOperator, LogicalVariable> select =
selectExpression.getSelectSetOperation().accept(this, currentOpRef);
- currentOpRef = new MutableObject<>(select.first);
+ if (select.first != null) {
+ currentOpRef = new MutableObject<>(select.first);
+ }
if (selectExpression.hasOrderby()) {
currentOpRef = new MutableObject<>(selectExpression.getOrderbyClause().accept(this, currentOpRef).first);
}
@@ -325,6 +330,10 @@
} else {
unnestOp = new UnnestOperator(fromVar, new MutableObject<>(pUnnestExpr.first));
}
+ ExternalSubpathAnnotation hint = ((AbstractExpression) fromExpr).findHint(ExternalSubpathAnnotation.class);
+ if (hint != null) {
+ unnestOp.getAnnotations().put(SUBPATH, hint.getSubPath());
+ }
unnestOp.getInputs().add(pUnnestExpr.second);
unnestOp.setSourceLocation(sourceLoc);
@@ -576,6 +585,10 @@
outerUnnestMissingValue)
: new UnnestOperator(rightVar, new MutableObject<>(pUnnestExpr.first));
}
+ ExternalSubpathAnnotation hint = ((AbstractExpression) rightExpr).findHint(ExternalSubpathAnnotation.class);
+ if (hint != null) {
+ unnestOp.getAnnotations().put(SUBPATH, hint.getSubPath());
+ }
unnestOp.getInputs().add(pUnnestExpr.second);
unnestOp.setSourceLocation(binaryCorrelate.getRightVariable().getSourceLocation());
return new Pair<>(unnestOp, rightVar);
@@ -757,7 +770,9 @@
} else {
ProjectOperator pr = new ProjectOperator(resVar);
pr.getInputs().add(returnOpRef);
- pr.setSourceLocation(returnOpRef.getValue().getSourceLocation());
+ if (returnOpRef.getValue() != null) {
+ pr.setSourceLocation(returnOpRef.getValue().getSourceLocation());
+ }
return new Pair<>(pr, resVar);
}
}
@@ -1204,10 +1219,14 @@
boolean isWin = BuiltinFunctions.isWindowFunction(fi);
boolean isWinAgg = isWin && BuiltinFunctions.builtinFunctionHasProperty(fi,
BuiltinFunctions.WindowFunctionProperty.HAS_LIST_ARG);
- boolean prohibitOrderClause = isWin && BuiltinFunctions.builtinFunctionHasProperty(fi,
- BuiltinFunctions.WindowFunctionProperty.NO_ORDER_CLAUSE);
- boolean prohibitFrameClause = isWin && BuiltinFunctions.builtinFunctionHasProperty(fi,
- BuiltinFunctions.WindowFunctionProperty.NO_FRAME_CLAUSE);
+ boolean prohibitOrderClause = (isWin && BuiltinFunctions.builtinFunctionHasProperty(fi,
+ BuiltinFunctions.WindowFunctionProperty.NO_ORDER_CLAUSE))
+ || (!isWin && BuiltinFunctions.builtinFunctionHasProperty(fi,
+ BuiltinFunctions.AggregateFunctionProperty.NO_ORDER_CLAUSE));
+ boolean prohibitFrameClause = (isWin && BuiltinFunctions.builtinFunctionHasProperty(fi,
+ BuiltinFunctions.WindowFunctionProperty.NO_FRAME_CLAUSE))
+ || (!isWin && BuiltinFunctions.builtinFunctionHasProperty(fi,
+ BuiltinFunctions.AggregateFunctionProperty.NO_FRAME_CLAUSE));
boolean allowRespectIgnoreNulls = isWin && BuiltinFunctions.builtinFunctionHasProperty(fi,
BuiltinFunctions.WindowFunctionProperty.ALLOW_RESPECT_IGNORE_NULLS);
boolean allowFromFirstLast = isWin && BuiltinFunctions.builtinFunctionHasProperty(fi,
diff --git a/asterixdb/asterix-app/pom.xml b/asterixdb/asterix-app/pom.xml
index 18a1f58..2285d77 100644
--- a/asterixdb/asterix-app/pom.xml
+++ b/asterixdb/asterix-app/pom.xml
@@ -22,7 +22,7 @@
<parent>
<artifactId>apache-asterixdb</artifactId>
<groupId>org.apache.asterix</groupId>
- <version>0.9.8-SNAPSHOT</version>
+ <version>0.9.8.2-SNAPSHOT</version>
</parent>
<artifactId>asterix-app</artifactId>
<licenses>
@@ -39,6 +39,7 @@
<sonar.sources>pom.xml,src/main/java,src/main/resources</sonar.sources>
<pip.path>${project.build.directory}/bin/pip3</pip.path>
<shiv.path>${project.build.directory}/bin/shiv</shiv.path>
+ <asterix-test-datagenerator-maven-plugin.version>${project.version}</asterix-test-datagenerator-maven-plugin.version>
</properties>
<build>
<plugins>
@@ -108,7 +109,7 @@
<plugin>
<groupId>org.apache.asterix</groupId>
<artifactId>asterix-test-datagenerator-maven-plugin</artifactId>
- <version>${project.version}</version>
+ <version>${asterix-test-datagenerator-maven-plugin.version}</version>
<executions>
<execution>
<id>replace-template-data</id>
@@ -266,6 +267,7 @@
<excludes combine.children="append">
<exclude>src/test/resources/**/results_parser_sqlpp/**</exclude>
<exclude>src/test/resources/**/results/**</exclude>
+ <exclude>src/test/resources/**/results_cbo/**</exclude>
<exclude>src/test/resources/fuzzyjoin/pub/fuzzy-join-aql*.dot</exclude>
<exclude>src/test/resources/fuzzyjoin/pub/fuzzy-join-aql*.json</exclude>
<exclude>**/data/**</exclude>
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
index 3a9d54d..cb86e35 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
@@ -40,6 +40,7 @@
import org.apache.asterix.common.api.IResponsePrinter;
import org.apache.asterix.common.config.CompilerProperties;
import org.apache.asterix.common.config.OptimizationConfUtil;
+import org.apache.asterix.common.dataflow.ICcApplicationContext;
import org.apache.asterix.common.exceptions.ACIDException;
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.common.exceptions.CompilationException;
@@ -54,7 +55,6 @@
import org.apache.asterix.dataflow.data.common.MergeAggregationExpressionFactory;
import org.apache.asterix.dataflow.data.common.MissableTypeComputer;
import org.apache.asterix.dataflow.data.common.PartialAggregationTypeComputer;
-import org.apache.asterix.external.feed.watch.FeedActivityDetails;
import org.apache.asterix.formats.base.IDataFormat;
import org.apache.asterix.jobgen.QueryLogicalExpressionJobGen;
import org.apache.asterix.lang.common.base.IAstPrintVisitorFactory;
@@ -65,25 +65,18 @@
import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
import org.apache.asterix.lang.common.statement.FunctionDecl;
import org.apache.asterix.lang.common.statement.Query;
-import org.apache.asterix.lang.common.statement.StartFeedStatement;
import org.apache.asterix.lang.common.statement.ViewDecl;
import org.apache.asterix.lang.common.struct.VarIdentifier;
-import org.apache.asterix.lang.common.util.FunctionUtil;
-import org.apache.asterix.lang.sqlpp.rewrites.SqlppQueryRewriter;
import org.apache.asterix.metadata.declared.MetadataProvider;
import org.apache.asterix.om.base.IAObject;
import org.apache.asterix.optimizer.base.AsterixOptimizationContext;
-import org.apache.asterix.optimizer.base.FuzzyUtils;
-import org.apache.asterix.optimizer.rules.DisjunctivePredicateToJoinRule;
-import org.apache.asterix.optimizer.rules.SetAsterixPhysicalOperatorsRule;
-import org.apache.asterix.optimizer.rules.util.EquivalenceClassUtils;
import org.apache.asterix.runtime.job.listener.JobEventListenerFactory;
import org.apache.asterix.translator.CompiledStatements.ICompiledDmlStatement;
import org.apache.asterix.translator.ExecutionPlans;
import org.apache.asterix.translator.IRequestParameters;
+import org.apache.asterix.translator.ResultMetadata;
import org.apache.asterix.translator.SessionConfig;
import org.apache.asterix.translator.SessionOutput;
-import org.apache.asterix.translator.SqlppExpressionToPlanTranslator;
import org.apache.asterix.utils.ResourceUtils;
import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
@@ -106,6 +99,8 @@
import org.apache.hyracks.algebricks.core.rewriter.base.IOptimizationContextFactory;
import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
import org.apache.hyracks.algebricks.data.IPrinterFactoryProvider;
+import org.apache.hyracks.algebricks.runtime.serializer.ResultSerializerFactoryProvider;
+import org.apache.hyracks.algebricks.runtime.writers.PrinterBasedWriterFactory;
import org.apache.hyracks.api.client.IClusterInfoCollector;
import org.apache.hyracks.api.client.IHyracksClientConnection;
import org.apache.hyracks.api.client.NodeControllerInfo;
@@ -121,7 +116,6 @@
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.ObjectWriter;
-import com.google.common.collect.ImmutableSet;
/**
* Provides helper methods for compilation of a query into a JobSpec and submission
@@ -133,29 +127,11 @@
public static final String PREFIX_INTERNAL_PARAMETERS = "_internal";
- // A white list of supported configurable parameters.
- private static final Set<String> CONFIGURABLE_PARAMETER_NAMES = ImmutableSet.of(
- CompilerProperties.COMPILER_JOINMEMORY_KEY, CompilerProperties.COMPILER_GROUPMEMORY_KEY,
- CompilerProperties.COMPILER_SORTMEMORY_KEY, CompilerProperties.COMPILER_WINDOWMEMORY_KEY,
- CompilerProperties.COMPILER_TEXTSEARCHMEMORY_KEY, CompilerProperties.COMPILER_PARALLELISM_KEY,
- CompilerProperties.COMPILER_SORT_PARALLEL_KEY, CompilerProperties.COMPILER_SORT_SAMPLES_KEY,
- CompilerProperties.COMPILER_EXTERNALSCANMEMORY_KEY, CompilerProperties.COMPILER_INDEXONLY_KEY,
- CompilerProperties.COMPILER_INTERNAL_SANITYCHECK_KEY,
- CompilerProperties.COMPILER_EXTERNAL_FIELD_PUSHDOWN_KEY, CompilerProperties.COMPILER_SUBPLAN_MERGE_KEY,
- CompilerProperties.COMPILER_SUBPLAN_NESTEDPUSHDOWN_KEY, CompilerProperties.COMPILER_ARRAYINDEX_KEY,
- CompilerProperties.COMPILER_MIN_MEMORY_ALLOCATION_KEY, FunctionUtil.IMPORT_PRIVATE_FUNCTIONS,
- FuzzyUtils.SIM_FUNCTION_PROP_NAME, FuzzyUtils.SIM_THRESHOLD_PROP_NAME,
- StartFeedStatement.WAIT_FOR_COMPLETION, FeedActivityDetails.FEED_POLICY_NAME,
- FeedActivityDetails.COLLECT_LOCATIONS, SqlppQueryRewriter.INLINE_WITH_OPTION,
- SqlppExpressionToPlanTranslator.REWRITE_IN_AS_OR_OPTION, "hash_merge", "output-record-type",
- DisjunctivePredicateToJoinRule.REWRITE_OR_AS_JOIN_OPTION,
- SetAsterixPhysicalOperatorsRule.REWRITE_ATTEMPT_BATCH_ASSIGN,
- EquivalenceClassUtils.REWRITE_INTERNAL_QUERYUID_PK, SqlppQueryRewriter.SQL_COMPAT_OPTION);
-
private final IRewriterFactory rewriterFactory;
private final IAstPrintVisitorFactory astPrintVisitorFactory;
private final ILangExpressionToPlanTranslatorFactory translatorFactory;
private final IRuleSetFactory ruleSetFactory;
+ private final Set<String> configurableParameterNames;
private final ExecutionPlans executionPlans;
public APIFramework(ILangCompilationProvider compilationProvider) {
@@ -163,6 +139,7 @@
this.astPrintVisitorFactory = compilationProvider.getAstPrintVisitorFactory();
this.translatorFactory = compilationProvider.getExpressionToPlanTranslatorFactory();
this.ruleSetFactory = compilationProvider.getRuleSetFactory();
+ this.configurableParameterNames = compilationProvider.getCompilerOptions();
executionPlans = new ExecutionPlans();
}
@@ -181,11 +158,16 @@
IConflictingTypeResolver conflictingTypeResolver, PhysicalOptimizationConfig physicalOptimizationConfig,
AlgebricksPartitionConstraint clusterLocations, IWarningCollector warningCollector) {
IPlanPrettyPrinter prettyPrinter = PlanPrettyPrinter.createStringPlanPrettyPrinter();
- return new AsterixOptimizationContext(varCounter, expressionEvalSizeComputer,
+ return new AsterixOptimizationContext(this, varCounter, expressionEvalSizeComputer,
mergeAggregationExpressionFactory, expressionTypeComputer, missableTypeComputer,
conflictingTypeResolver, physicalOptimizationConfig, clusterLocations, prettyPrinter,
warningCollector);
}
+
+ @Override
+ public IOptimizationContext cloneOptimizationContext(IOptimizationContext oc) {
+ return new AsterixOptimizationContext((AsterixOptimizationContext) oc);
+ }
}
public Pair<IReturningStatement, Integer> reWriteQuery(List<FunctionDecl> declaredFunctions,
@@ -233,20 +215,18 @@
ILogicalPlan plan =
isLoad ? t.translateLoad(statement) : t.translate(query, outputDatasetName, statement, resultMetadata);
- if ((isQuery || isLoad) && !conf.is(SessionConfig.FORMAT_ONLY_PHYSICAL_OPS)
- && conf.is(SessionConfig.OOB_LOGICAL_PLAN)) {
- generateLogicalPlan(plan, output.config().getPlanFormat());
- }
- CompilerProperties compilerProperties = metadataProvider.getApplicationContext().getCompilerProperties();
+ ICcApplicationContext ccAppContext = metadataProvider.getApplicationContext();
+ CompilerProperties compilerProperties = ccAppContext.getCompilerProperties();
Map<String, Object> querySpecificConfig = validateConfig(metadataProvider.getConfig(), sourceLoc);
final PhysicalOptimizationConfig physOptConf =
OptimizationConfUtil.createPhysicalOptimizationConf(compilerProperties, querySpecificConfig, sourceLoc);
-
+ boolean cboMode = physOptConf.getCBOMode() || physOptConf.getCBOTestMode();
HeuristicCompilerFactoryBuilder builder =
new HeuristicCompilerFactoryBuilder(OptimizationContextFactory.INSTANCE);
builder.setPhysicalOptimizationConfig(physOptConf);
- builder.setLogicalRewrites(ruleSetFactory.getLogicalRewrites(metadataProvider.getApplicationContext()));
- builder.setPhysicalRewrites(ruleSetFactory.getPhysicalRewrites(metadataProvider.getApplicationContext()));
+ builder.setLogicalRewrites(() -> ruleSetFactory.getLogicalRewrites(ccAppContext));
+ builder.setLogicalRewritesByKind(kind -> ruleSetFactory.getLogicalRewrites(kind, ccAppContext));
+ builder.setPhysicalRewrites(() -> ruleSetFactory.getPhysicalRewrites(ccAppContext));
IDataFormat format = metadataProvider.getDataFormat();
ICompilerFactory compilerFactory = builder.create();
builder.setExpressionEvalSizeComputer(format.getExpressionEvalSizeComputer());
@@ -258,12 +238,35 @@
builder.setWarningCollector(warningCollector);
builder.setMaxWarnings(conf.getMaxWarnings());
+ if ((isQuery || isLoad) && !conf.is(SessionConfig.FORMAT_ONLY_PHYSICAL_OPS)
+ && conf.is(SessionConfig.OOB_LOGICAL_PLAN)) {
+ generateLogicalPlan(plan, output.config().getPlanFormat(), cboMode);
+ }
+
int parallelism = getParallelism((String) querySpecificConfig.get(CompilerProperties.COMPILER_PARALLELISM_KEY),
compilerProperties.getParallelism());
AlgebricksAbsolutePartitionConstraint computationLocations =
chooseLocations(clusterInfoCollector, parallelism, metadataProvider.getClusterLocations());
builder.setClusterLocations(computationLocations);
+ builder.setBinaryBooleanInspectorFactory(format.getBinaryBooleanInspectorFactory());
+ builder.setBinaryIntegerInspectorFactory(format.getBinaryIntegerInspectorFactory());
+ builder.setComparatorFactoryProvider(format.getBinaryComparatorFactoryProvider());
+ builder.setExpressionRuntimeProvider(
+ new ExpressionRuntimeProvider(new QueryLogicalExpressionJobGen(metadataProvider.getFunctionManager())));
+ builder.setHashFunctionFactoryProvider(format.getBinaryHashFunctionFactoryProvider());
+ builder.setHashFunctionFamilyProvider(format.getBinaryHashFunctionFamilyProvider());
+ builder.setMissingWriterFactory(format.getMissingWriterFactory());
+ builder.setNullWriterFactory(format.getNullWriterFactory());
+ builder.setUnnestingPositionWriterFactory(format.getUnnestingPositionWriterFactory());
+ builder.setPredicateEvaluatorFactoryProvider(format.getPredicateEvaluatorFactoryProvider());
+ builder.setPrinterProvider(getPrinterFactoryProvider(format, conf.fmt()));
+ builder.setWriterFactory(PrinterBasedWriterFactory.INSTANCE);
+ builder.setResultSerializerFactoryProvider(ResultSerializerFactoryProvider.INSTANCE);
+ builder.setSerializerDeserializerProvider(format.getSerdeProvider());
+ builder.setTypeTraitProvider(format.getTypeTraitProvider());
+ builder.setNormalizedKeyComputerFactoryProvider(format.getNormalizedKeyComputerFactoryProvider());
+
ICompiler compiler = compilerFactory.createCompiler(plan, metadataProvider, t.getVarCounter());
if (conf.isOptimize()) {
compiler.optimize();
@@ -275,7 +278,7 @@
output.out().write(buf.toString());
} else {
if (isQuery || isLoad) {
- generateOptimizedLogicalPlan(plan, output.config().getPlanFormat());
+ generateOptimizedLogicalPlan(plan, output.config().getPlanFormat(), cboMode);
}
}
}
@@ -309,42 +312,46 @@
}
if (!conf.isGenerateJobSpec()) {
+ if (isQuery || isLoad) {
+ generateOptimizedLogicalPlan(plan, output.config().getPlanFormat(), cboMode);
+ }
return null;
}
- builder.setBinaryBooleanInspectorFactory(format.getBinaryBooleanInspectorFactory());
- builder.setBinaryIntegerInspectorFactory(format.getBinaryIntegerInspectorFactory());
- builder.setComparatorFactoryProvider(format.getBinaryComparatorFactoryProvider());
- builder.setExpressionRuntimeProvider(
- new ExpressionRuntimeProvider(new QueryLogicalExpressionJobGen(metadataProvider.getFunctionManager())));
- builder.setHashFunctionFactoryProvider(format.getBinaryHashFunctionFactoryProvider());
- builder.setHashFunctionFamilyProvider(format.getBinaryHashFunctionFamilyProvider());
- builder.setMissingWriterFactory(format.getMissingWriterFactory());
- builder.setNullWriterFactory(format.getNullWriterFactory());
- builder.setUnnestingPositionWriterFactory(format.getUnnestingPositionWriterFactory());
- builder.setPredicateEvaluatorFactoryProvider(format.getPredicateEvaluatorFactoryProvider());
- builder.setPrinterProvider(getPrinterFactoryProvider(format, conf.fmt()));
- builder.setSerializerDeserializerProvider(format.getSerdeProvider());
- builder.setTypeTraitProvider(format.getTypeTraitProvider());
- builder.setNormalizedKeyComputerFactoryProvider(format.getNormalizedKeyComputerFactoryProvider());
-
JobEventListenerFactory jobEventListenerFactory =
new JobEventListenerFactory(txnId, metadataProvider.isWriteTransaction());
- JobSpecification spec = compiler.createJob(metadataProvider.getApplicationContext(), jobEventListenerFactory);
+ JobSpecification spec = compiler.createJob(ccAppContext, jobEventListenerFactory);
if (isQuery) {
- if (requestParameters == null || !requestParameters.isSkipAdmissionPolicy()) {
- // Sets a required capacity, only for read-only queries.
- // DDLs and DMLs are considered not that frequent.
- // limit the computation locations to the locations that will be used in the query
- final INodeJobTracker nodeJobTracker = metadataProvider.getApplicationContext().getNodeJobTracker();
- final AlgebricksAbsolutePartitionConstraint jobLocations =
- getJobLocations(spec, nodeJobTracker, computationLocations);
- final IClusterCapacity jobRequiredCapacity =
- ResourceUtils.getRequiredCapacity(plan, jobLocations, physOptConf);
- spec.setRequiredClusterCapacity(jobRequiredCapacity);
+ if (!compiler.skipJobCapacityAssignment()) {
+ if (requestParameters == null || !requestParameters.isSkipAdmissionPolicy()) {
+ // Sets a required capacity, only for read-only queries.
+ // DDLs and DMLs are considered not that frequent.
+ // limit the computation locations to the locations that will be used in the query
+ final INodeJobTracker nodeJobTracker = ccAppContext.getNodeJobTracker();
+ final AlgebricksAbsolutePartitionConstraint jobLocations =
+ getJobLocations(spec, nodeJobTracker, computationLocations);
+ final IClusterCapacity jobRequiredCapacity =
+ ResourceUtils.getRequiredCapacity(plan, jobLocations, physOptConf, compilerProperties);
+ spec.setRequiredClusterCapacity(jobRequiredCapacity);
+ }
}
}
+
+ if (conf.is(SessionConfig.OOB_OPTIMIZED_LOGICAL_PLAN) || isExplainOnly) {
+ if (isQuery || isLoad) {
+ generateOptimizedLogicalPlan(plan, output.config().getPlanFormat(), cboMode);
+ }
+ }
+
+ if (isExplainOnly) {
+ printPlanAsResult(metadataProvider, output, printer, printSignature);
+ if (!conf.is(SessionConfig.OOB_OPTIMIZED_LOGICAL_PLAN)) {
+ executionPlans.setOptimizedLogicalPlan(null);
+ }
+ return null;
+ }
+
if (isQuery && conf.is(SessionConfig.OOB_HYRACKS_JOB)) {
generateJob(spec);
}
@@ -502,10 +509,10 @@
}
// Validates if the query contains unsupported query parameters.
- private static Map<String, Object> validateConfig(Map<String, Object> config, SourceLocation sourceLoc)
+ private Map<String, Object> validateConfig(Map<String, Object> config, SourceLocation sourceLoc)
throws AlgebricksException {
for (String parameterName : config.keySet()) {
- if (!CONFIGURABLE_PARAMETER_NAMES.contains(parameterName)
+ if (!configurableParameterNames.contains(parameterName)
&& !parameterName.startsWith(PREFIX_INTERNAL_PARAMETERS)) {
throw AsterixException.create(ErrorCode.COMPILATION_UNSUPPORTED_QUERY_PARAMETER, sourceLoc,
parameterName);
@@ -530,13 +537,16 @@
}
}
- private void generateLogicalPlan(ILogicalPlan plan, SessionConfig.PlanFormat format) throws AlgebricksException {
- executionPlans.setLogicalPlan(getPrettyPrintVisitor(format).printPlan(plan).toString());
+ private void generateLogicalPlan(ILogicalPlan plan, SessionConfig.PlanFormat format,
+ boolean printOptimizerEstimates) throws AlgebricksException {
+ executionPlans
+ .setLogicalPlan(getPrettyPrintVisitor(format).printPlan(plan, printOptimizerEstimates).toString());
}
- private void generateOptimizedLogicalPlan(ILogicalPlan plan, SessionConfig.PlanFormat format)
- throws AlgebricksException {
- executionPlans.setOptimizedLogicalPlan(getPrettyPrintVisitor(format).printPlan(plan).toString());
+ private void generateOptimizedLogicalPlan(ILogicalPlan plan, SessionConfig.PlanFormat format,
+ boolean printOptimizerEstimates) throws AlgebricksException {
+ executionPlans.setOptimizedLogicalPlan(
+ getPrettyPrintVisitor(format).printPlan(plan, printOptimizerEstimates).toString());
}
private void generateJob(JobSpecification spec) {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/ResultMetadata.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/ResultMetadata.java
deleted file mode 100644
index 94360a1..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/ResultMetadata.java
+++ /dev/null
@@ -1,113 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.api.common;
-
-import java.util.List;
-import java.util.Set;
-
-import org.apache.asterix.translator.SessionConfig;
-import org.apache.hyracks.api.exceptions.Warning;
-import org.apache.hyracks.api.result.IResultMetadata;
-
-import com.fasterxml.jackson.databind.node.ObjectNode;
-
-public class ResultMetadata implements IResultMetadata {
- private static final long serialVersionUID = 1905367559307369034L;
-
- private final SessionConfig.OutputFormat format;
- private long jobDuration;
- private long processedObjects;
- private ObjectNode profile;
- private long diskIoCount;
- private Set<Warning> warnings;
- private long totalWarningsCount;
- private transient List<Object> outputTypes;
-
- public ResultMetadata(SessionConfig.OutputFormat format) {
- this.format = format;
- }
-
- public SessionConfig.OutputFormat getFormat() {
- return format;
- }
-
- public long getProcessedObjects() {
- return processedObjects;
- }
-
- public void setProcessedObjects(long processedObjects) {
- this.processedObjects = processedObjects;
- }
-
- public void setJobDuration(long jobDuration) {
- this.jobDuration = jobDuration;
- }
-
- public void setWarnings(Set<Warning> warnings) {
- this.warnings = warnings;
- }
-
- /**
- * Sets the count of all warnings generated including unreported ones.
- */
- public void setTotalWarningsCount(long totalWarningsCount) {
- this.totalWarningsCount = totalWarningsCount;
- }
-
- public long getJobDuration() {
- return jobDuration;
- }
-
- public void setJobProfile(ObjectNode profile) {
- this.profile = profile;
- }
-
- public ObjectNode getJobProfile() {
- return profile;
- }
-
- /**
- * @return The reported warnings.
- */
- public Set<Warning> getWarnings() {
- return warnings;
- }
-
- /**
- * @return Total count of all warnings generated including unreported ones.
- */
- public long getTotalWarningsCount() {
- return totalWarningsCount;
- }
-
- @Override
- public void setOutputTypes(List<Object> typeList) {
- this.outputTypes = typeList;
- }
-
- public List<Object> getOutputTypes() {
- return outputTypes;
- }
-
- @Override
- public String toString() {
- return "ResultMetadata{" + "format=" + format + ", jobDuration=" + jobDuration + ", processedObjects="
- + processedObjects + ", diskIoCount=" + diskIoCount + '}';
- }
-}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractQueryApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractQueryApiServlet.java
index 4dadf55..bf77c24 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractQueryApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractQueryApiServlet.java
@@ -28,8 +28,6 @@
import org.apache.hyracks.api.client.IHyracksClientConnection;
import org.apache.hyracks.api.result.IResultSet;
import org.apache.hyracks.http.server.AbstractServlet;
-import org.apache.hyracks.ipc.exceptions.IPCException;
-import org.apache.logging.log4j.Level;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
@@ -61,13 +59,7 @@
}
protected IResultSet getResultSet() throws Exception { // NOSONAR
- try {
- return ServletUtil.getResultSet(getHyracksClientConnection(), appCtx, ctx);
- } catch (IPCException e) {
- LOGGER.log(Level.WARN, "Failed getting hyracks dataset connection. Resetting hyracks connection.", e);
- ctx.put(HYRACKS_CONNECTION_ATTR, appCtx.getHcc());
- return ServletUtil.getResultSet(getHyracksClientConnection(), appCtx, ctx);
- }
+ return ServletUtil.getResultSet(appCtx, ctx);
}
protected IHyracksClientConnection getHyracksClientConnection() throws Exception { // NOSONAR
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractRequestsServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractRequestsServlet.java
new file mode 100644
index 0000000..b6c6a71
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractRequestsServlet.java
@@ -0,0 +1,68 @@
+/*
+ * 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.api.http.server;
+
+import java.util.Collection;
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.asterix.common.api.IClientRequest;
+import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.hyracks.http.api.IServletRequest;
+import org.apache.hyracks.http.api.IServletResponse;
+import org.apache.hyracks.http.server.AbstractServlet;
+import org.apache.hyracks.http.server.utils.HttpUtil;
+import org.apache.hyracks.util.JSONUtil;
+
+import com.fasterxml.jackson.databind.node.ArrayNode;
+
+import io.netty.handler.codec.http.HttpResponseStatus;
+
+public abstract class AbstractRequestsServlet extends AbstractServlet {
+
+ public static final String REDACT_PARAM = "redact";
+ protected final ICcApplicationContext appCtx;
+
+ public AbstractRequestsServlet(ConcurrentMap<String, Object> ctx, ICcApplicationContext appCtx, String... paths) {
+ super(ctx, paths);
+ this.appCtx = appCtx;
+ }
+
+ @Override
+ protected void get(IServletRequest request, IServletResponse response) throws Exception {
+ ArrayNode requestsJson = JSONUtil.createArray();
+ Collection<IClientRequest> requests = getRequests();
+ String redact = request.getParameter(REDACT_PARAM);
+ if (Boolean.parseBoolean(redact)) {
+ for (IClientRequest req : requests) {
+ requestsJson.add(req.asRedactedJson());
+ }
+ } else {
+ for (IClientRequest req : requests) {
+ requestsJson.add(req.asJson());
+ }
+ }
+ HttpUtil.setContentType(response, HttpUtil.ContentType.APPLICATION_JSON, request);
+ response.setStatus(HttpResponseStatus.OK);
+ JSONUtil.writeNode(response.writer(), requestsJson);
+ response.writer().flush();
+ }
+
+ abstract Collection<IClientRequest> getRequests();
+
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ActiveRequestsServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ActiveRequestsServlet.java
new file mode 100644
index 0000000..7e00e9a
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ActiveRequestsServlet.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.api.http.server;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.asterix.api.http.server.QueryServiceRequestParameters.Parameter;
+import org.apache.asterix.common.api.IClientRequest;
+import org.apache.asterix.common.api.IRequestTracker;
+import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.hyracks.http.api.IServletRequest;
+import org.apache.hyracks.http.api.IServletResponse;
+import org.apache.logging.log4j.Level;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import io.netty.handler.codec.http.HttpResponseStatus;
+
+/**
+ * The servlet provides a REST API for getting the running queries or cancelling an on-going one.
+ */
+public class ActiveRequestsServlet extends AbstractRequestsServlet {
+
+ public static final String REQUEST_UUID_PARAM_NAME = "request_id";
+ private static final Logger LOGGER = LogManager.getLogger();
+
+ public ActiveRequestsServlet(ConcurrentMap<String, Object> ctx, ICcApplicationContext appCtx, String... paths) {
+ super(ctx, appCtx, paths);
+ }
+
+ @Override
+ public Collection<IClientRequest> getRequests() {
+ return appCtx.getRequestTracker().getRunningRequests();
+ }
+
+ @Override
+ protected void delete(IServletRequest request, IServletResponse response) throws IOException {
+ String uuid = request.getParameter(REQUEST_UUID_PARAM_NAME);
+ String clientCtxId = request.getParameter(Parameter.CLIENT_ID.str());
+ LOGGER.debug("received cancel request, uuid={}, clientCtxId={}", uuid, clientCtxId);
+ if (uuid == null && clientCtxId == null) {
+ response.setStatus(HttpResponseStatus.BAD_REQUEST);
+ return;
+ }
+ final IRequestTracker requestTracker = appCtx.getRequestTracker();
+ IClientRequest req = uuid != null ? requestTracker.get(uuid) : requestTracker.getByClientContextId(clientCtxId);
+ if (req == null) {
+ response.setStatus(HttpResponseStatus.NOT_FOUND);
+ return;
+ }
+ if (!req.isCancellable()) {
+ response.setStatus(HttpResponseStatus.FORBIDDEN);
+ return;
+ }
+ try {
+ // Cancels the on-going job.
+ requestTracker.cancel(req.getId());
+ // response: OK
+ response.setStatus(HttpResponseStatus.OK);
+ } catch (Exception e) {
+ LOGGER.log(Level.WARN, "unexpected exception thrown from cancel", e);
+ // response: INTERNAL SERVER ERROR
+ response.setStatus(HttpResponseStatus.INTERNAL_SERVER_ERROR);
+ }
+ }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ApiServlet.java
index 081c69a..56ad88e 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ApiServlet.java
@@ -134,7 +134,7 @@
try {
// TODO: warnings should be retrieved from warnings collectors
IHyracksClientConnection hcc = (IHyracksClientConnection) ctx.get(HYRACKS_CONNECTION_ATTR);
- IResultSet resultSet = ServletUtil.getResultSet(hcc, appCtx, ctx);
+ IResultSet resultSet = ServletUtil.getResultSet(appCtx, ctx);
IParser parser = parserFactory.createParser(query);
List<Statement> statements = parser.parse();
SessionConfig sessionConfig = new SessionConfig(format, true, isSet(executeQuery), true, planFormat);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/CcQueryCancellationServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/CcQueryCancellationServlet.java
deleted file mode 100644
index 7ba2867..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/CcQueryCancellationServlet.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.api.http.server;
-
-import java.io.IOException;
-import java.util.concurrent.ConcurrentMap;
-
-import org.apache.asterix.api.http.server.QueryServiceRequestParameters.Parameter;
-import org.apache.asterix.common.api.IClientRequest;
-import org.apache.asterix.common.api.IRequestTracker;
-import org.apache.asterix.common.dataflow.ICcApplicationContext;
-import org.apache.hyracks.http.api.IServletRequest;
-import org.apache.hyracks.http.api.IServletResponse;
-import org.apache.hyracks.http.server.AbstractServlet;
-import org.apache.logging.log4j.Level;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
-import io.netty.handler.codec.http.HttpResponseStatus;
-
-/**
- * The servlet provides a REST API for cancelling an on-going query.
- */
-public class CcQueryCancellationServlet extends AbstractServlet {
-
- public static final String REQUEST_UUID_PARAM_NAME = "request_id";
- private static final Logger LOGGER = LogManager.getLogger();
- private final ICcApplicationContext appCtx;
-
- public CcQueryCancellationServlet(ConcurrentMap<String, Object> ctx, ICcApplicationContext appCtx,
- String... paths) {
- super(ctx, paths);
- this.appCtx = appCtx;
- }
-
- @Override
- protected void delete(IServletRequest request, IServletResponse response) throws IOException {
- String uuid = request.getParameter(REQUEST_UUID_PARAM_NAME);
- String clientCtxId = request.getParameter(Parameter.CLIENT_ID.str());
- if (uuid == null && clientCtxId == null) {
- response.setStatus(HttpResponseStatus.BAD_REQUEST);
- return;
- }
- final IRequestTracker requestTracker = appCtx.getRequestTracker();
- IClientRequest req = uuid != null ? requestTracker.get(uuid) : requestTracker.getByClientContextId(clientCtxId);
- if (req == null) {
- response.setStatus(HttpResponseStatus.NOT_FOUND);
- return;
- }
- if (!req.isCancellable()) {
- response.setStatus(HttpResponseStatus.FORBIDDEN);
- return;
- }
- try {
- // Cancels the on-going job.
- requestTracker.cancel(req.getId());
- // response: OK
- response.setStatus(HttpResponseStatus.OK);
- } catch (Exception e) {
- LOGGER.log(Level.WARN, "unexpected exception thrown from cancel", e);
- // response: INTERNAL SERVER ERROR
- response.setStatus(HttpResponseStatus.INTERNAL_SERVER_ERROR);
- }
- }
-}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ClusterApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ClusterApiServlet.java
index eaeb082..ed0a68a 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ClusterApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ClusterApiServlet.java
@@ -21,7 +21,6 @@
import static org.apache.asterix.api.http.server.ServletConstants.ASTERIX_APP_CONTEXT_INFO_ATTR;
import java.io.IOException;
-import java.io.PrintWriter;
import java.util.concurrent.ConcurrentMap;
import java.util.function.Predicate;
@@ -29,6 +28,7 @@
import org.apache.hyracks.api.config.IOption;
import org.apache.hyracks.api.config.Section;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IFormattedException;
import org.apache.hyracks.control.common.config.ConfigUtils;
import org.apache.hyracks.control.common.controllers.ControllerConfig;
import org.apache.hyracks.http.api.IServletRequest;
@@ -36,7 +36,6 @@
import org.apache.hyracks.http.server.AbstractServlet;
import org.apache.hyracks.http.server.utils.HttpUtil;
import org.apache.hyracks.util.JSONUtil;
-import org.apache.logging.log4j.Level;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
@@ -67,7 +66,6 @@
@Override
protected void get(IServletRequest request, IServletResponse response) throws IOException {
HttpUtil.setContentType(response, HttpUtil.ContentType.APPLICATION_JSON, request);
- PrintWriter responseWriter = response.writer();
try {
ObjectNode json;
response.setStatus(HttpResponseStatus.OK);
@@ -81,15 +79,18 @@
default:
throw new IllegalArgumentException();
}
- JSONUtil.writeNode(responseWriter, json);
+ JSONUtil.writeNode(response.writer(), json);
} catch (IllegalArgumentException e) { // NOSONAR - exception not logged or rethrown
sendError(response, HttpResponseStatus.NOT_FOUND);
} catch (Exception e) {
- LOGGER.log(Level.INFO, "exception thrown for " + request, e);
- response.setStatus(HttpResponseStatus.INTERNAL_SERVER_ERROR);
- responseWriter.write(e.toString());
+ LOGGER.info("exception thrown for {}", request, e);
+ if (e instanceof IFormattedException) {
+ sendError(response, HttpResponseStatus.INTERNAL_SERVER_ERROR, (IFormattedException) e);
+ } else {
+ sendError(response, HttpResponseStatus.INTERNAL_SERVER_ERROR, e.toString());
+
+ }
}
- responseWriter.flush();
}
@Override
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/CompletedRequestsServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/CompletedRequestsServlet.java
new file mode 100644
index 0000000..92eacbb
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/CompletedRequestsServlet.java
@@ -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.
+ */
+package org.apache.asterix.api.http.server;
+
+import java.util.Collection;
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.asterix.common.api.IClientRequest;
+import org.apache.asterix.common.dataflow.ICcApplicationContext;
+
+public class CompletedRequestsServlet extends AbstractRequestsServlet {
+
+ public CompletedRequestsServlet(ConcurrentMap<String, Object> ctx, ICcApplicationContext appCtx, String... paths) {
+ super(ctx, appCtx, paths);
+ }
+
+ @Override
+ public Collection<IClientRequest> getRequests() {
+ return appCtx.getRequestTracker().getCompletedRequests();
+ }
+
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryCancellationServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryCancellationServlet.java
index b2134dc..5dd9430 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryCancellationServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryCancellationServlet.java
@@ -18,7 +18,7 @@
*/
package org.apache.asterix.api.http.server;
-import static org.apache.asterix.api.http.server.CcQueryCancellationServlet.REQUEST_UUID_PARAM_NAME;
+import static org.apache.asterix.api.http.server.ActiveRequestsServlet.REQUEST_UUID_PARAM_NAME;
import static org.apache.asterix.app.message.ExecuteStatementRequestMessage.DEFAULT_NC_TIMEOUT_MILLIS;
import java.util.concurrent.ConcurrentMap;
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryResultApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryResultApiServlet.java
index 981cdc9..bfebfd6 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryResultApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryResultApiServlet.java
@@ -21,7 +21,6 @@
import java.io.IOException;
import java.util.concurrent.ConcurrentMap;
-import org.apache.asterix.api.common.ResultMetadata;
import org.apache.asterix.app.result.ResponseMetrics;
import org.apache.asterix.app.result.ResponsePrinter;
import org.apache.asterix.app.result.ResultHandle;
@@ -31,6 +30,7 @@
import org.apache.asterix.app.result.fields.ResultsPrinter;
import org.apache.asterix.common.api.IApplicationContext;
import org.apache.asterix.translator.IStatementExecutor.Stats;
+import org.apache.asterix.translator.ResultMetadata;
import org.apache.asterix.translator.SessionConfig;
import org.apache.asterix.translator.SessionOutput;
import org.apache.hyracks.api.exceptions.ErrorCode;
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceRequestParameters.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceRequestParameters.java
index b6913e4..563f498 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceRequestParameters.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceRequestParameters.java
@@ -19,6 +19,8 @@
package org.apache.asterix.api.http.server;
+import static org.apache.asterix.utils.RedactionUtil.REDACTED_SENSITIVE_ENTRY_VALUE;
+
import java.io.IOException;
import java.util.HashMap;
import java.util.Iterator;
@@ -43,6 +45,7 @@
import org.apache.hyracks.http.api.IServletRequest;
import org.apache.hyracks.http.server.utils.HttpUtil;
import org.apache.hyracks.util.JSONUtil;
+import org.apache.hyracks.util.LogRedactionUtil;
import com.fasterxml.jackson.core.JsonParseException;
import com.fasterxml.jackson.core.JsonProcessingException;
@@ -80,7 +83,8 @@
SIGNATURE("signature"),
MULTI_STATEMENT("multi-statement"),
MAX_WARNINGS("max-warnings"),
- SQL_COMPAT("sql-compat");
+ SQL_COMPAT("sql-compat"),
+ SOURCE("source");
private final String str;
@@ -123,7 +127,9 @@
private String path;
private String statement;
private String clientContextID;
+ private String requestId;
private String dataverse;
+ private String source;
private ClientType clientType = ClientType.ASTERIX;
private OutputFormat format = OutputFormat.CLEAN_JSON;
private ResultDelivery mode = ResultDelivery.IMMEDIATE;
@@ -172,6 +178,14 @@
this.statement = statement;
}
+ public String getSource() {
+ return source;
+ }
+
+ public void setSource(String source) {
+ this.source = source;
+ }
+
public OutputFormat getFormat() {
return format;
}
@@ -376,11 +390,20 @@
return maxWarnings;
}
+ public void setRequestId(String requestId) {
+ this.requestId = requestId;
+ }
+
+ public String getRequestId() {
+ return requestId;
+ }
+
public ObjectNode asJson() {
ObjectNode object = OBJECT_MAPPER.createObjectNode();
object.put("host", host);
object.put("path", path);
- object.put("statement", statement != null ? JSONUtil.escape(new StringBuilder(), statement).toString() : null);
+ object.put("statement", statement != null
+ ? LogRedactionUtil.statement(JSONUtil.escape(new StringBuilder(), statement).toString()) : null);
object.put("pretty", pretty);
object.put("mode", mode.getName());
object.put("clientContextID", clientContextID);
@@ -402,9 +425,10 @@
object.put("readOnly", readOnly);
object.put("maxWarnings", maxWarnings);
object.put("sqlCompat", sqlCompatMode);
+ object.put("source", source);
if (statementParams != null) {
for (Map.Entry<String, JsonNode> statementParam : statementParams.entrySet()) {
- object.set('$' + statementParam.getKey(), statementParam.getValue());
+ object.set('$' + statementParam.getKey(), REDACTED_SENSITIVE_ENTRY_VALUE);
}
}
return object;
@@ -486,6 +510,7 @@
setSignature(parseBoolean(req, Parameter.SIGNATURE.str(), valGetter, isSignature()));
setClientType(parseIfExists(req, Parameter.CLIENT_TYPE.str(), valGetter, getClientType(), clientTypes::get));
setSQLCompatMode(parseBoolean(req, Parameter.SQL_COMPAT.str(), valGetter, isSQLCompatMode()));
+ setSource(valGetter.apply(req, Parameter.SOURCE.str()));
}
protected void setExtraParams(JsonNode jsonRequest) throws HyracksDataException {
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 9f83aa8..8de7582 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
@@ -90,7 +90,7 @@
import org.apache.hyracks.http.api.IServletRequest;
import org.apache.hyracks.http.api.IServletResponse;
import org.apache.hyracks.http.server.utils.HttpUtil;
-import org.apache.hyracks.util.LogRedactionUtil;
+import org.apache.logging.log4j.Level;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
@@ -270,6 +270,7 @@
ResponsePrinter responsePrinter = new ResponsePrinter(sessionOutput);
ResultDelivery delivery = ResultDelivery.IMMEDIATE;
QueryServiceRequestParameters param = newQueryRequestParameters();
+ param.setRequestId(requestRef.getUuid());
RequestExecutionState executionState = newRequestExecutionState();
try {
// buffer the output until we are ready to set the status of the response message correctly
@@ -278,7 +279,10 @@
if (forceReadOnly) {
param.setReadOnly(true);
}
- LOGGER.info(() -> "handleRequest: " + LogRedactionUtil.statement(param.toString()));
+ if (LOGGER.isInfoEnabled()) {
+ LOGGER.info("handleRequest: uuid={}, clientContextID={}, {}", requestRef.getUuid(),
+ param.getClientContextID(), param.toString());
+ }
delivery = param.getMode();
setSessionConfig(sessionOutput, param, delivery);
final ResultProperties resultProperties = new ResultProperties(delivery, param.getMaxResultReads());
@@ -429,15 +433,15 @@
executionState.setStatus(ResultStatus.FATAL, HttpResponseStatus.BAD_REQUEST);
return true;
case REQUEST_TIMEOUT:
- LOGGER.info(() -> "handleException: request execution timed out: "
- + LogRedactionUtil.userData(param.toString()));
+ logException(Level.INFO, "request execution timed out", param.getRequestId(),
+ param.getClientContextID());
executionState.setStatus(ResultStatus.TIMEOUT, HttpResponseStatus.OK);
return true;
case REJECT_NODE_UNREGISTERED:
case REJECT_BAD_CLUSTER_STATE:
- LOGGER.warn(() -> "handleException: " + ex.getMessage() + ": "
- + LogRedactionUtil.userData(param.toString()));
+ logException(Level.WARN, ex.getMessage(), param.getRequestId(), param.getClientContextID());
executionState.setStatus(ResultStatus.FATAL, HttpResponseStatus.SERVICE_UNAVAILABLE);
+ return true;
default:
// fall-through
}
@@ -455,11 +459,9 @@
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()),
- t);
+ logException(Level.DEBUG, t.getMessage(), param.getRequestId(), param.getClientContextID(), t);
} else {
- LOGGER.info(() -> "handleException: " + t.getMessage() + ": "
- + LogRedactionUtil.statement(param.toString()));
+ logException(Level.INFO, t.getMessage(), param.getRequestId(), param.getClientContextID());
}
executionState.setStatus(ResultStatus.FATAL, HttpResponseStatus.BAD_REQUEST);
return;
@@ -471,7 +473,7 @@
return;
}
}
- LOGGER.warn(() -> "handleException: unexpected exception: " + LogRedactionUtil.userData(param.toString()), t);
+ logException(Level.WARN, "unexpected exception", param.getRequestId(), param.getClientContextID(), t);
executionState.setStatus(ResultStatus.FATAL, HttpResponseStatus.INTERNAL_SERVER_ERROR);
}
@@ -543,4 +545,12 @@
protected String getApplicationVersion() {
return ApplicationConfigurator.getApplicationVersion(appCtx.getBuildProperties());
}
+
+ private void logException(Level lvl, String msg, String clientCtxId, String uuid) {
+ LOGGER.log(lvl, "handleException: {}: uuid={}, clientContextID={}", msg, uuid, clientCtxId);
+ }
+
+ private void logException(Level lvl, String msg, String clientCtxId, String uuid, Throwable t) {
+ LOGGER.log(lvl, "handleException: {}: uuid={}, clientContextID={}", msg, uuid, clientCtxId, t);
+ }
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ServletUtil.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ServletUtil.java
index 3ac37bf..8d5e4db 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ServletUtil.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ServletUtil.java
@@ -24,29 +24,22 @@
import java.util.List;
import java.util.Map;
-import org.apache.asterix.app.result.ResultReader;
import org.apache.asterix.common.api.IApplicationContext;
import org.apache.asterix.common.metadata.DataverseName;
import org.apache.commons.codec.DecoderException;
import org.apache.commons.codec.net.URLCodec;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.api.client.IHyracksClientConnection;
import org.apache.hyracks.api.result.IResultSet;
-import org.apache.hyracks.client.result.ResultSet;
import org.apache.hyracks.http.api.IServletRequest;
public class ServletUtil {
- static IResultSet getResultSet(IHyracksClientConnection hcc, IApplicationContext appCtx,
- final Map<String, Object> ctx) throws Exception {
+ static IResultSet getResultSet(IApplicationContext appCtx, final Map<String, Object> ctx) throws Exception {
IResultSet resultSet = (IResultSet) ctx.get(RESULTSET_ATTR);
if (resultSet == null) {
synchronized (ctx) {
resultSet = (IResultSet) ctx.get(RESULTSET_ATTR);
if (resultSet == null) {
- resultSet = new ResultSet(hcc,
- appCtx.getServiceContext().getControllerService().getNetworkSecurityManager()
- .getSocketChannelFactory(),
- appCtx.getCompilerProperties().getFrameSize(), ResultReader.NUM_READERS);
+ resultSet = appCtx.getResultSet();
ctx.put(RESULTSET_ATTR, resultSet);
}
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveEntityEventsListener.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveEntityEventsListener.java
index 3fd339e..f32c730 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveEntityEventsListener.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveEntityEventsListener.java
@@ -105,6 +105,7 @@
protected int numDeRegistered;
protected volatile RecoveryTask rt;
protected volatile boolean suspended = false;
+ private long suspendCount;
// failures
protected Exception jobFailure;
protected Exception resumeFailure;
@@ -138,7 +139,7 @@
}
protected synchronized void setState(ActivityState newState) {
- LOGGER.log(level, "State of {} is being set to {} from {}", getEntityId(), newState, state);
+ LOGGER.log(level, "state of {} is being set from {} to {}", getEntityId(), state, newState);
this.prevState = state;
this.state = newState;
if (newState == ActivityState.STARTING || newState == ActivityState.RECOVERING
@@ -153,9 +154,8 @@
@Override
public synchronized void notify(ActiveEvent event) {
try {
- if (LOGGER.isEnabled(level)) {
- LOGGER.log(level, "EventListener is notified.");
- }
+ LOGGER.debug("CC handling event {}; state={}, prev state={}, suspended={}", event, state, prevState,
+ suspended);
ActiveEvent.Kind eventKind = event.getEventKind();
switch (eventKind) {
case JOB_CREATED:
@@ -194,26 +194,21 @@
@SuppressWarnings("unchecked")
protected void finish(ActiveEvent event) throws HyracksDataException {
- if (LOGGER.isEnabled(level)) {
- LOGGER.log(level, "Active job {} finished", jobId);
- }
JobId lastJobId = jobId;
+ Pair<JobStatus, List<Exception>> status = (Pair<JobStatus, List<Exception>>) event.getEventObject();
if (numRegistered != numDeRegistered) {
LOGGER.log(Level.WARN,
- "Active job {} finished with reported runtime registrations = {} and deregistrations = {}", jobId,
- numRegistered, numDeRegistered);
+ "ingestion job {} finished with status={}, reported runtime registrations={}, deregistrations={}",
+ jobId, status, numRegistered, numDeRegistered);
}
jobId = null;
- Pair<JobStatus, List<Exception>> status = (Pair<JobStatus, List<Exception>>) event.getEventObject();
JobStatus jobStatus = status.getLeft();
List<Exception> exceptions = status.getRight();
- if (LOGGER.isEnabled(level)) {
- LOGGER.log(level, "Active job {} finished with status {}", lastJobId, jobStatus);
- }
+ LOGGER.debug("ingestion job {} finished with status {}", lastJobId, jobStatus);
if (!jobSuccessfullyTerminated(jobStatus)) {
jobFailure = exceptions.isEmpty() ? new RuntimeDataException(ErrorCode.UNREPORTED_TASK_FAILURE_EXCEPTION)
: exceptions.get(0);
- LOGGER.error("Active Job {} failed", lastJobId, jobFailure);
+ LOGGER.error("ingestion job {} failed", lastJobId, jobFailure);
setState((state == ActivityState.STOPPING || state == ActivityState.CANCELLING) ? ActivityState.STOPPED
: ActivityState.TEMPORARILY_FAILED);
if (prevState == ActivityState.RUNNING) {
@@ -271,6 +266,10 @@
return jobId;
}
+ public long getSuspendCount() {
+ return suspendCount;
+ }
+
@Override
public String getStats() {
return stats;
@@ -371,16 +370,14 @@
@Override
public synchronized void recover() {
- if (LOGGER.isEnabled(level)) {
- LOGGER.log(level, "Recover is called on {}", entityId);
- }
if (retryPolicyFactory == NoRetryPolicyFactory.INSTANCE) {
- LOGGER.log(level, "But it has no recovery policy, so it is set to permanent failure");
+ LOGGER.debug("recover is called on {} w/o recovery policy; setting to permanent failure", entityId);
setState(ActivityState.STOPPED);
} else {
+ LOGGER.debug("recover is called on {}", entityId);
ExecutorService executor = appCtx.getServiceContext().getControllerService().getExecutor();
setState(ActivityState.TEMPORARILY_FAILED);
- LOGGER.log(level, "Recovery task has been submitted");
+ LOGGER.debug("recovery task has been submitted");
rt = createRecoveryTask();
executor.submit(rt.recover());
}
@@ -479,15 +476,11 @@
// Note: once we start sending stop messages, we can't go back until the entity is stopped
final String nameBefore = Thread.currentThread().getName();
try {
- Thread.currentThread().setName(nameBefore + " : WaitForCompletionForJobId: " + jobId);
+ Thread.currentThread().setName(nameBefore + " : wait-for-ingestion-completion: " + jobId);
sendStopMessages(metadataProvider, timeout, unit);
- if (LOGGER.isDebugEnabled()) {
- LOGGER.debug("Waiting for its state to become " + waitFor);
- }
+ LOGGER.debug("waiting for {} to become {}", jobId, waitFor);
subscriber.sync();
- if (LOGGER.isDebugEnabled()) {
- LOGGER.debug("Disconnect has been completed " + waitFor);
- }
+ LOGGER.debug("disconnect has been completed {}", waitFor);
} catch (InterruptedException ie) {
forceStop(subscriber, ie);
Thread.currentThread().interrupt();
@@ -513,13 +506,8 @@
ICCMessageBroker messageBroker = (ICCMessageBroker) applicationCtx.getServiceContext().getMessageBroker();
AlgebricksAbsolutePartitionConstraint runtimeLocations = getLocations();
int partition = 0;
- if (LOGGER.isInfoEnabled()) {
- LOGGER.log(Level.INFO, "Sending stop messages to " + runtimeLocations);
- }
+ LOGGER.log(Level.INFO, "sending stop messages to {}", runtimeLocations);
for (String location : runtimeLocations.getLocations()) {
- if (LOGGER.isInfoEnabled()) {
- LOGGER.log(Level.INFO, "Sending to " + location);
- }
ActiveRuntimeId runtimeId = getActiveRuntimeId(partition++);
messageBroker.sendApplicationMessageToNC(new ActiveManagerMessage(ActiveManagerMessage.Kind.STOP_ACTIVITY,
runtimeId, new StopRuntimeParameters(timeout, unit)), location);
@@ -581,14 +569,11 @@
WaitForStateSubscriber subscriber;
Future<Void> suspendTask;
synchronized (this) {
- if (LOGGER.isEnabled(level)) {
- LOGGER.log(level, "suspending entity " + entityId);
- LOGGER.log(level, "Waiting for ongoing activities");
- }
+ LOGGER.log(level, "{} suspending entity {}", jobId, entityId);
+ LOGGER.log(level, "{} waiting for ongoing activities", jobId);
waitForNonTransitionState();
- if (LOGGER.isEnabled(level)) {
- LOGGER.log(level, "Proceeding with suspension. Current state is " + state);
- }
+ LOGGER.log(level, "{} proceeding with suspension. current state is {}", jobId, state);
+ suspendCount++;
if (state == ActivityState.STOPPED) {
suspended = true;
return;
@@ -609,12 +594,12 @@
doSuspend(metadataProvider);
return null;
});
- LOGGER.log(level, "Suspension task has been submitted");
+ LOGGER.log(level, "{} suspension task has been submitted", jobId);
}
try {
- LOGGER.log(level, "Waiting for suspension task to complete");
+ LOGGER.log(level, "{} waiting for suspension task to complete", jobId);
suspendTask.get();
- LOGGER.log(level, "waiting for state to become SUSPENDED or TEMPORARILY_FAILED");
+ LOGGER.log(level, "{} waiting for state to become SUSPENDED or TEMPORARILY_FAILED", jobId);
subscriber.sync();
suspended = true;
} catch (Exception e) {
@@ -691,7 +676,7 @@
@Override
public synchronized void replace(Dataset dataset) {
- if (getDatasets().contains(dataset)) {
+ if (getDatasets().remove(dataset)) {
getDatasets().remove(dataset);
getDatasets().add(dataset);
}
@@ -736,7 +721,7 @@
@Override
public String toString() {
- return "{\"class\":\"" + getClass().getSimpleName() + "\"," + "\"entityId\":\"" + entityId + "\","
- + "\"state\":\"" + state + "\"" + "}";
+ return "{\"class\":\"" + getClass().getSimpleName() + "\", \"entityId\":\"" + entityId + "\", \"state\":\""
+ + state + "\", \"prev state\":\"" + prevState + "\", \"suspended\":" + suspended + "}";
}
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveNotificationHandler.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveNotificationHandler.java
index 6b3581e..b619cfe 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveNotificationHandler.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveNotificationHandler.java
@@ -19,9 +19,11 @@
package org.apache.asterix.app.active;
import java.util.Collection;
+import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
+import java.util.Optional;
import org.apache.asterix.active.ActiveEvent;
import org.apache.asterix.active.ActiveEvent.Kind;
@@ -40,6 +42,7 @@
import org.apache.hyracks.api.job.JobId;
import org.apache.hyracks.api.job.JobSpecification;
import org.apache.hyracks.api.job.JobStatus;
+import org.apache.hyracks.api.job.resource.IJobCapacityController;
import org.apache.hyracks.api.util.SingleThreadEventProcessor;
import org.apache.hyracks.util.ExitUtil;
import org.apache.logging.log4j.Level;
@@ -66,96 +69,79 @@
@Override
protected void handle(ActiveEvent event) {
+ resolveListenerForEvent(event).ifPresent(listener -> listener.notify(event));
+ }
+
+ private synchronized Optional<IActiveEntityEventsListener> resolveListenerForEvent(ActiveEvent event) {
JobId jobId = event.getJobId();
Kind eventKind = event.getEventKind();
EntityId entityId = jobId2EntityId.get(jobId);
+ IActiveEntityEventsListener listener = null;
if (entityId != null) {
- IActiveEntityEventsListener listener = entityEventListeners.get(entityId);
- if (LOGGER.isEnabled(level)) {
- LOGGER.log(level, "Next event is {} for job {}", eventKind, jobId);
- }
+ listener = entityEventListeners.get(entityId);
if (eventKind == Kind.JOB_FINISHED) {
- LOGGER.log(level, "Removing job {}", jobId);
+ LOGGER.debug("removing ingestion job {}", jobId);
jobId2EntityId.remove(jobId);
}
- if (listener != null) {
- LOGGER.log(level, "Notifying the listener");
- listener.notify(event);
+ if (listener == null) {
+ LOGGER.debug("listener not found for entity {} on event {} for job {}", entityId, event, jobId);
}
} else {
- LOGGER.log(Level.ERROR, "Entity not found for event {} for job {}", eventKind, jobId);
+ LOGGER.log(Level.ERROR, "entity not found for event {} for job {}", eventKind, jobId);
}
+ return Optional.ofNullable(listener);
}
// *** IJobLifecycleListener
@Override
- public void notifyJobCreation(JobId jobId, JobSpecification jobSpecification) throws HyracksDataException {
- if (LOGGER.isEnabled(level)) {
- LOGGER.log(level, "notifyJobCreation was called for job {}", jobId);
- }
+ public void notifyJobCreation(JobId jobId, JobSpecification jobSpecification,
+ IJobCapacityController.JobSubmissionStatus status) throws HyracksDataException {
Object property = jobSpecification.getProperty(ACTIVE_ENTITY_PROPERTY_NAME);
if (!(property instanceof EntityId)) {
- if (LOGGER.isEnabled(level)) {
- LOGGER.log(level, "Job {} is not of type active job. property found to be {}", jobId, property);
+ if (property != null) {
+ LOGGER.debug("{} is not an ingestion job. job property={}", jobId, property);
}
return;
}
+ LOGGER.debug("notified of ingestion job creation {}", jobId);
EntityId entityId = (EntityId) property;
monitorJob(jobId, entityId);
- boolean found = jobId2EntityId.get(jobId) != null;
- LOGGER.log(level, "Job {} was found to be {}", jobId, (found ? "Active" : "Inactive"));
add(new ActiveEvent(jobId, Kind.JOB_CREATED, entityId, jobSpecification));
}
private synchronized void monitorJob(JobId jobId, EntityId entityId) {
- if (LOGGER.isEnabled(level)) {
- LOGGER.log(level, "monitorJob was called for job {}", jobId);
- }
- boolean found = jobId2EntityId.get(jobId) != null;
- if (LOGGER.isEnabled(level)) {
- LOGGER.log(level, "Job {} was found to be {}", jobId, (found ? "Active" : "Inactive"));
- }
+ boolean found = jobId2EntityId.containsKey(jobId);
+ LOGGER.debug("{} was {}", jobId, (found ? "active" : "inactive"));
if (entityEventListeners.containsKey(entityId)) {
- if (jobId2EntityId.containsKey(jobId)) {
- if (LOGGER.isErrorEnabled()) {
- LOGGER.error("Job {} is already being monitored", jobId);
- }
+ if (found) {
+ LOGGER.error("{} is already being monitored", jobId);
return;
}
- if (LOGGER.isEnabled(level)) {
- LOGGER.log(level, "Monitoring started for job {}", jobId);
- }
} else {
- if (LOGGER.isEnabled(level)) {
- LOGGER.info("No listener was found for the entity {} for job {}", entityId, jobId);
- }
+ LOGGER.debug("no listener found for entity {}; {}", entityId, jobId);
}
jobId2EntityId.put(jobId, entityId);
}
@Override
- public synchronized void notifyJobStart(JobId jobId) throws HyracksException {
+ public synchronized void notifyJobStart(JobId jobId, JobSpecification spec) throws HyracksException {
EntityId entityId = jobId2EntityId.get(jobId);
if (entityId != null) {
add(new ActiveEvent(jobId, Kind.JOB_STARTED, entityId, null));
}
+ // else must be non-active job, e.g. a job for a query
}
@Override
- public synchronized void notifyJobFinish(JobId jobId, JobStatus jobStatus, List<Exception> exceptions)
- throws HyracksException {
- if (LOGGER.isEnabled(level)) {
- LOGGER.log(level, "Getting notified of job finish for job {}", jobId);
- }
+ public synchronized void notifyJobFinish(JobId jobId, JobSpecification spec, JobStatus jobStatus,
+ List<Exception> exceptions) throws HyracksException {
EntityId entityId = jobId2EntityId.get(jobId);
if (entityId != null) {
+ LOGGER.debug("notified of ingestion job finish {}", jobId);
add(new ActiveEvent(jobId, Kind.JOB_FINISHED, entityId, Pair.of(jobStatus, exceptions)));
- } else {
- if (LOGGER.isEnabled(level)) {
- LOGGER.log(level, "No need to notify JOB_FINISHED for job {}", jobId);
- }
}
+ // else must be non-active job, e.g. a job for a query
}
// *** IActiveNotificationHandler
@@ -168,13 +154,6 @@
@Override
public IActiveEntityEventsListener getListener(EntityId entityId) {
- if (LOGGER.isEnabled(level)) {
- LOGGER.log(level, "getActiveEntityListener was called with entity {}", entityId);
- }
- IActiveEntityEventsListener listener = entityEventListeners.get(entityId);
- if (LOGGER.isEnabled(level)) {
- LOGGER.log(level, "Listener found: {}", listener);
- }
return entityEventListeners.get(entityId);
}
@@ -187,13 +166,16 @@
}
@Override
+ public synchronized Collection<IActiveEntityEventsListener> getEventListenersAsList() {
+ return Collections.unmodifiableCollection(entityEventListeners.values());
+ }
+
+ @Override
public synchronized void registerListener(IActiveEntityEventsListener listener) throws HyracksDataException {
if (suspended) {
throw new RuntimeDataException(ErrorCode.ACTIVE_NOTIFICATION_HANDLER_IS_SUSPENDED);
}
- if (LOGGER.isEnabled(level)) {
- LOGGER.log(level, "registerListener was called for the entity {}", listener.getEntityId());
- }
+ LOGGER.debug("register listener for entity {}, state={}", listener.getEntityId(), listener.getState());
if (entityEventListeners.containsKey(listener.getEntityId())) {
throw new RuntimeDataException(ErrorCode.ACTIVE_ENTITY_IS_ALREADY_REGISTERED, listener.getEntityId());
}
@@ -205,9 +187,7 @@
if (suspended) {
throw new RuntimeDataException(ErrorCode.ACTIVE_NOTIFICATION_HANDLER_IS_SUSPENDED);
}
- if (LOGGER.isEnabled(level)) {
- LOGGER.log(level, "unregisterListener was called for the entity {}", listener.getEntityId());
- }
+ LOGGER.debug("unregister listener for entity {}, state={}", listener.getEntityId(), listener.getState());
IActiveEntityEventsListener registeredListener = entityEventListeners.remove(listener.getEntityId());
if (registeredListener == null) {
throw new RuntimeDataException(ErrorCode.ACTIVE_ENTITY_LISTENER_IS_NOT_REGISTERED, listener.getEntityId());
@@ -223,9 +203,8 @@
LOGGER.info("Starting active recovery");
for (IActiveEntityEventsListener listener : getEventListeners()) {
synchronized (listener) {
- if (LOGGER.isEnabled(level)) {
- LOGGER.log(level, "Entity {} is {}", listener.getEntityId(), listener.getState());
- }
+ LOGGER.debug("entity {} is {}, active={}, suspended={}", listener.getEntityId(), listener.getState(),
+ listener.isActive(), listener.isSuspended());
listener.notifyAll();
}
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/RecoveryTask.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/RecoveryTask.java
index 34a54d1..b2e6204 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/RecoveryTask.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/RecoveryTask.java
@@ -121,6 +121,7 @@
protected Void doRecover(IRetryPolicy policy) throws AlgebricksException, InterruptedException {
LOGGER.log(level, "Actual Recovery task has started");
Exception failure;
+ long prevSuspendCount;
do {
synchronized (listener) {
while (!cancelRecovery && !canStartRecovery()) {
@@ -140,6 +141,7 @@
if (!cancelRecovery && listener.getState() == ActivityState.TEMPORARILY_FAILED) {
listener.setState(ActivityState.RECOVERING);
listener.doRecover(metadataProvider);
+ listener.setRunning(metadataProvider, true);
}
LOGGER.log(level, "Recovery completed successfully");
return null;
@@ -152,6 +154,7 @@
listener.setState(ActivityState.TEMPORARILY_FAILED);
failure = e;
} finally {
+ prevSuspendCount = listener.getSuspendCount();
releaseRecoveryLocks(metadataProvider);
}
} while (policy.retry(failure));
@@ -172,20 +175,28 @@
}
IMetadataLockManager lockManager = metadataProvider.getApplicationContext().getMetadataLockManager();
IMetadataLockUtil lockUtil = metadataProvider.getApplicationContext().getMetadataLockUtil();
+ boolean retryRecovery = false;
try {
acquirePostRecoveryLocks(lockManager, lockUtil);
synchronized (listener) {
if (!cancelRecovery && listener.getState() == ActivityState.TEMPORARILY_FAILED) {
- LOGGER.warn("Recovery for {} permanently failed", listener.getEntityId());
- listener.setState(ActivityState.STOPPED);
- listener.setRunning(metadataProvider, false);
+ if (prevSuspendCount == listener.getSuspendCount()) {
+ LOGGER.warn("Recovery for {} permanently failed", listener.getEntityId());
+ listener.setState(ActivityState.STOPPED);
+ listener.setRunning(metadataProvider, false);
+ } else {
+ LOGGER.log(level,
+ "Retrying recovery on non-retryable failure due to interleaving suspend/resume for {}",
+ listener.getEntityId());
+ retryRecovery = true;
+ }
}
listener.notifyAll();
}
} finally {
releasePostRecoveryLocks();
}
- return null;
+ return retryRecovery ? doRecover(policy) : null;
}
protected void acquireRecoveryLocks(IMetadataLockManager lockManager, IMetadataLockUtil lockUtil)
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CcApplicationContext.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CcApplicationContext.java
new file mode 100644
index 0000000..880880e
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CcApplicationContext.java
@@ -0,0 +1,360 @@
+/*
+ * 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.app.cc;
+
+import java.io.IOException;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Supplier;
+
+import org.apache.asterix.app.result.ResultReader;
+import org.apache.asterix.common.api.IConfigValidator;
+import org.apache.asterix.common.api.IConfigValidatorFactory;
+import org.apache.asterix.common.api.ICoordinationService;
+import org.apache.asterix.common.api.IMetadataLockManager;
+import org.apache.asterix.common.api.INodeJobTracker;
+import org.apache.asterix.common.api.IReceptionist;
+import org.apache.asterix.common.api.IReceptionistFactory;
+import org.apache.asterix.common.api.IRequestTracker;
+import org.apache.asterix.common.cluster.IClusterStateManager;
+import org.apache.asterix.common.cluster.IGlobalRecoveryManager;
+import org.apache.asterix.common.config.ActiveProperties;
+import org.apache.asterix.common.config.BuildProperties;
+import org.apache.asterix.common.config.CompilerProperties;
+import org.apache.asterix.common.config.ExtensionProperties;
+import org.apache.asterix.common.config.ExternalProperties;
+import org.apache.asterix.common.config.MessagingProperties;
+import org.apache.asterix.common.config.MetadataProperties;
+import org.apache.asterix.common.config.NodeProperties;
+import org.apache.asterix.common.config.PropertiesAccessor;
+import org.apache.asterix.common.config.ReplicationProperties;
+import org.apache.asterix.common.config.StorageProperties;
+import org.apache.asterix.common.config.TransactionProperties;
+import org.apache.asterix.common.context.IStorageComponentProvider;
+import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.asterix.common.external.IAdapterFactoryService;
+import org.apache.asterix.common.metadata.IMetadataBootstrap;
+import org.apache.asterix.common.metadata.IMetadataLockUtil;
+import org.apache.asterix.common.replication.INcLifecycleCoordinator;
+import org.apache.asterix.common.storage.ICompressionManager;
+import org.apache.asterix.common.transactions.IResourceIdManager;
+import org.apache.asterix.common.transactions.ITxnIdFactory;
+import org.apache.asterix.runtime.compression.CompressionManager;
+import org.apache.asterix.runtime.job.listener.NodeJobTracker;
+import org.apache.asterix.runtime.transaction.ResourceIdManager;
+import org.apache.asterix.runtime.utils.BulkTxnIdFactory;
+import org.apache.asterix.runtime.utils.ClusterStateManager;
+import org.apache.asterix.runtime.utils.NoOpCoordinationService;
+import org.apache.asterix.runtime.utils.RequestTracker;
+import org.apache.asterix.runtime.utils.RuntimeComponentsProvider;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.application.ICCServiceContext;
+import org.apache.hyracks.api.client.IHyracksClientConnection;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.job.IJobLifecycleListener;
+import org.apache.hyracks.api.result.IResultSet;
+import org.apache.hyracks.client.result.ResultSet;
+import org.apache.hyracks.ipc.impl.HyracksConnection;
+import org.apache.hyracks.storage.common.IStorageManager;
+import org.apache.hyracks.util.NetworkUtil;
+
+/*
+ * Acts as an holder class for IndexRegistryProvider, AsterixStorageManager
+ * instances that are accessed from the NCs. In addition an instance of ICCApplicationContext
+ * is stored for access by the CC.
+ */
+public class CcApplicationContext implements ICcApplicationContext {
+
+ private ICCServiceContext ccServiceCtx;
+ private IStorageComponentProvider storageComponentProvider;
+ private IGlobalRecoveryManager globalRecoveryManager;
+ private IResourceIdManager resourceIdManager;
+ private CompilerProperties compilerProperties;
+ private ExternalProperties externalProperties;
+ private MetadataProperties metadataProperties;
+ private StorageProperties storageProperties;
+ private TransactionProperties txnProperties;
+ private ActiveProperties activeProperties;
+ private BuildProperties buildProperties;
+ private ReplicationProperties replicationProperties;
+ private ExtensionProperties extensionProperties;
+ private MessagingProperties messagingProperties;
+ private NodeProperties nodeProperties;
+ private Supplier<IMetadataBootstrap> metadataBootstrapSupplier;
+ private volatile HyracksConnection hcc;
+ private volatile ResultSet resultSet;
+ private Object extensionManager;
+ private INcLifecycleCoordinator ftStrategy;
+ private IJobLifecycleListener activeLifeCycleListener;
+ private IMetadataLockManager mdLockManager;
+ private IMetadataLockUtil mdLockUtil;
+ private IClusterStateManager clusterStateManager;
+ private final INodeJobTracker nodeJobTracker;
+ private final ITxnIdFactory txnIdFactory;
+ private final ICompressionManager compressionManager;
+ private final IReceptionist receptionist;
+ private final IRequestTracker requestTracker;
+ private final IConfigValidator configValidator;
+ private final IAdapterFactoryService adapterFactoryService;
+ private final ReentrantReadWriteLock compilationLock = new ReentrantReadWriteLock(true);
+
+ public CcApplicationContext(ICCServiceContext ccServiceCtx, HyracksConnection hcc,
+ Supplier<IMetadataBootstrap> metadataBootstrapSupplier, IGlobalRecoveryManager globalRecoveryManager,
+ INcLifecycleCoordinator ftStrategy, IJobLifecycleListener activeLifeCycleListener,
+ IStorageComponentProvider storageComponentProvider, IMetadataLockManager mdLockManager,
+ IMetadataLockUtil mdLockUtil, IReceptionistFactory receptionistFactory,
+ IConfigValidatorFactory configValidatorFactory, Object extensionManager,
+ IAdapterFactoryService adapterFactoryService) throws AlgebricksException, IOException {
+ this.ccServiceCtx = ccServiceCtx;
+ this.hcc = hcc;
+ this.activeLifeCycleListener = activeLifeCycleListener;
+ this.extensionManager = extensionManager;
+ // Determine whether to use old-style asterix-configuration.xml or new-style configuration.
+ // QQQ strip this out eventually
+ PropertiesAccessor propertiesAccessor = PropertiesAccessor.getInstance(ccServiceCtx.getAppConfig());
+ compilerProperties = new CompilerProperties(propertiesAccessor);
+ externalProperties = new ExternalProperties(propertiesAccessor);
+ metadataProperties = new MetadataProperties(propertiesAccessor);
+ storageProperties = new StorageProperties(propertiesAccessor);
+ txnProperties = new TransactionProperties(propertiesAccessor);
+ activeProperties = new ActiveProperties(propertiesAccessor);
+ extensionProperties = new ExtensionProperties(propertiesAccessor);
+ replicationProperties = new ReplicationProperties(propertiesAccessor);
+ this.ftStrategy = ftStrategy;
+ this.buildProperties = new BuildProperties(propertiesAccessor);
+ this.messagingProperties = new MessagingProperties(propertiesAccessor);
+ this.nodeProperties = new NodeProperties(propertiesAccessor);
+ this.metadataBootstrapSupplier = metadataBootstrapSupplier;
+ this.globalRecoveryManager = globalRecoveryManager;
+ this.storageComponentProvider = storageComponentProvider;
+ this.mdLockManager = mdLockManager;
+ this.mdLockUtil = mdLockUtil;
+ clusterStateManager = new ClusterStateManager();
+ clusterStateManager.setCcAppCtx(this);
+ this.resourceIdManager = new ResourceIdManager(clusterStateManager);
+ nodeJobTracker = new NodeJobTracker();
+ txnIdFactory = new BulkTxnIdFactory();
+ compressionManager = new CompressionManager(storageProperties);
+ receptionist = receptionistFactory.create();
+ requestTracker = new RequestTracker(this);
+ configValidator = configValidatorFactory.create();
+ this.adapterFactoryService = adapterFactoryService;
+ }
+
+ @Override
+ public ICCServiceContext getServiceContext() {
+ return ccServiceCtx;
+ }
+
+ @Override
+ public StorageProperties getStorageProperties() {
+ return storageProperties;
+ }
+
+ @Override
+ public TransactionProperties getTransactionProperties() {
+ return txnProperties;
+ }
+
+ @Override
+ public CompilerProperties getCompilerProperties() {
+ return compilerProperties;
+ }
+
+ @Override
+ public MetadataProperties getMetadataProperties() {
+ return metadataProperties;
+ }
+
+ @Override
+ public ExternalProperties getExternalProperties() {
+ return externalProperties;
+ }
+
+ @Override
+ public ActiveProperties getActiveProperties() {
+ return activeProperties;
+ }
+
+ @Override
+ public BuildProperties getBuildProperties() {
+ return buildProperties;
+ }
+
+ @Override
+ public IHyracksClientConnection getHcc() throws HyracksDataException {
+ HyracksConnection hc = hcc;
+ if (!hc.isConnected()) {
+ synchronized (this) {
+ hc = hcc;
+ if (!hc.isConnected()) {
+ try {
+ ResultSet rs = resultSet;
+ resultSet = null;
+ NetworkUtil.closeQuietly(rs);
+
+ NetworkUtil.closeQuietly(hc);
+ hcc = hc = new HyracksConnection(hcc.getHost(), hcc.getPort());
+ } catch (Exception e) {
+ throw HyracksDataException.create(e);
+ }
+ }
+ }
+ }
+ return hc;
+ }
+
+ @Override
+ public IResultSet getResultSet() throws HyracksDataException {
+ ResultSet rs = resultSet;
+ if (rs == null) {
+ synchronized (this) {
+ rs = resultSet;
+ if (rs == null) {
+ try {
+ resultSet = rs = ResultReader.createResultSet(getHcc(), ccServiceCtx.getControllerService(),
+ compilerProperties);
+ } catch (Exception e) {
+ throw HyracksDataException.create(e);
+ }
+ }
+ }
+ }
+ return rs;
+ }
+
+ @Override
+ public IStorageManager getStorageManager() {
+ return RuntimeComponentsProvider.RUNTIME_PROVIDER;
+ }
+
+ @Override
+ public ReplicationProperties getReplicationProperties() {
+ return replicationProperties;
+ }
+
+ @Override
+ public IGlobalRecoveryManager getGlobalRecoveryManager() {
+ return globalRecoveryManager;
+ }
+
+ @Override
+ public Object getExtensionManager() {
+ return extensionManager;
+ }
+
+ @Override
+ public ExtensionProperties getExtensionProperties() {
+ return extensionProperties;
+ }
+
+ @Override
+ public MessagingProperties getMessagingProperties() {
+ return messagingProperties;
+ }
+
+ @Override
+ public NodeProperties getNodeProperties() {
+ return nodeProperties;
+ }
+
+ @Override
+ public IResourceIdManager getResourceIdManager() {
+ return resourceIdManager;
+ }
+
+ @Override
+ public IMetadataBootstrap getMetadataBootstrap() {
+ return metadataBootstrapSupplier.get();
+ }
+
+ @Override
+ public INcLifecycleCoordinator getNcLifecycleCoordinator() {
+ return ftStrategy;
+ }
+
+ @Override
+ public IJobLifecycleListener getActiveNotificationHandler() {
+ return activeLifeCycleListener;
+ }
+
+ @Override
+ public IStorageComponentProvider getStorageComponentProvider() {
+ return storageComponentProvider;
+ }
+
+ @Override
+ public IMetadataLockManager getMetadataLockManager() {
+ return mdLockManager;
+ }
+
+ @Override
+ public IMetadataLockUtil getMetadataLockUtil() {
+ return mdLockUtil;
+ }
+
+ @Override
+ public IClusterStateManager getClusterStateManager() {
+ return clusterStateManager;
+ }
+
+ @Override
+ public INodeJobTracker getNodeJobTracker() {
+ return nodeJobTracker;
+ }
+
+ @Override
+ public ICoordinationService getCoordinationService() {
+ return NoOpCoordinationService.INSTANCE;
+ }
+
+ @Override
+ public ITxnIdFactory getTxnIdFactory() {
+ return txnIdFactory;
+ }
+
+ @Override
+ public ICompressionManager getCompressionManager() {
+ return compressionManager;
+ }
+
+ @Override
+ public IReceptionist getReceptionist() {
+ return receptionist;
+ }
+
+ @Override
+ public IConfigValidator getConfigValidator() {
+ return configValidator;
+ }
+
+ @Override
+ public IRequestTracker getRequestTracker() {
+ return requestTracker;
+ }
+
+ @Override
+ public IAdapterFactoryService getAdapterFactoryService() {
+ return adapterFactoryService;
+ }
+
+ @Override
+ public ReentrantReadWriteLock getCompilationLock() {
+ return compilationLock;
+ }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/ActiveRequestsDatasource.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/ActiveRequestsDatasource.java
index 24a127a..acb2ca5 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/ActiveRequestsDatasource.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/ActiveRequestsDatasource.java
@@ -18,6 +18,8 @@
*/
package org.apache.asterix.app.function;
+import java.util.Objects;
+
import org.apache.asterix.app.message.ClientRequestsRequest;
import org.apache.asterix.metadata.api.IDatasourceFunction;
import org.apache.asterix.metadata.declared.DataSourceId;
@@ -43,4 +45,14 @@
AlgebricksAbsolutePartitionConstraint.randomLocation(locations.getLocations());
return new ClientRequestsFunction(randomLocation, ClientRequestsRequest.RequestType.RUNNING);
}
+
+ @Override
+ public boolean skipJobCapacityAssignment() {
+ return true;
+ }
+
+ @Override
+ protected boolean sameFunctionDatasource(FunctionDataSource other) {
+ return Objects.equals(this.functionId, other.getFunctionId());
+ }
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/CompletedRequestsDatasource.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/CompletedRequestsDatasource.java
index 0bfbff1..4d48eef 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/CompletedRequestsDatasource.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/CompletedRequestsDatasource.java
@@ -18,6 +18,8 @@
*/
package org.apache.asterix.app.function;
+import java.util.Objects;
+
import org.apache.asterix.app.message.ClientRequestsRequest;
import org.apache.asterix.metadata.api.IDatasourceFunction;
import org.apache.asterix.metadata.declared.DataSourceId;
@@ -43,4 +45,14 @@
AlgebricksAbsolutePartitionConstraint.randomLocation(locations.getLocations());
return new ClientRequestsFunction(randomLocation, ClientRequestsRequest.RequestType.COMPLETED);
}
+
+ @Override
+ public boolean skipJobCapacityAssignment() {
+ return true;
+ }
+
+ @Override
+ protected boolean sameFunctionDatasource(FunctionDataSource other) {
+ return Objects.equals(this.functionId, other.getFunctionId());
+ }
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DatasetResourcesDatasource.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DatasetResourcesDatasource.java
index 3e5033b..d1c3c01 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DatasetResourcesDatasource.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DatasetResourcesDatasource.java
@@ -18,6 +18,8 @@
*/
package org.apache.asterix.app.function;
+import java.util.Objects;
+
import org.apache.asterix.metadata.api.IDatasourceFunction;
import org.apache.asterix.metadata.declared.DataSourceId;
import org.apache.asterix.metadata.declared.FunctionDataSource;
@@ -43,4 +45,17 @@
AlgebricksAbsolutePartitionConstraint locations) {
return new DatasetResourcesFunction(locations, datasetId);
}
+
+ public int getDatasetId() {
+ return datasetId;
+ }
+
+ @Override
+ public boolean sameFunctionDatasource(FunctionDataSource other) {
+ if (!Objects.equals(this.functionId, other.getFunctionId())) {
+ return false;
+ }
+ DatasetResourcesDatasource that = (DatasetResourcesDatasource) other;
+ return Objects.equals(this.datasetId, that.getDatasetId());
+ }
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DatasetRewriter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DatasetRewriter.java
index b1c2f0a..68edc0d 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DatasetRewriter.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DatasetRewriter.java
@@ -21,10 +21,14 @@
import static org.apache.asterix.common.api.IIdentifierMapper.Modifier.PLURAL;
import static org.apache.asterix.common.api.IIdentifierMapper.Modifier.SINGULAR;
import static org.apache.asterix.common.utils.IdentifierUtil.dataset;
+import static org.apache.asterix.external.util.ExternalDataConstants.SUBPATH;
+import java.io.Serializable;
import java.util.ArrayList;
import java.util.List;
+import java.util.Map;
+import org.apache.asterix.common.config.DatasetConfig;
import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.common.metadata.DatasetFullyQualifiedName;
@@ -102,6 +106,15 @@
}
DataSourceScanOperator scan = new DataSourceScanOperator(variables, dataSource);
scan.setSourceLocation(unnest.getSourceLocation());
+ if (dataset.getDatasetType() == DatasetConfig.DatasetType.EXTERNAL) {
+ Map<String, Object> unnestAnnotations = unnest.getAnnotations();
+ scan.getAnnotations().putAll(unnestAnnotations);
+ Map<String, Serializable> dataSourceProperties = dataSource.getProperties();
+ Object externalSubpath = unnestAnnotations.get(SUBPATH);
+ if (externalSubpath instanceof String) {
+ dataSourceProperties.put(SUBPATH, (String) externalSubpath);
+ }
+ }
List<Mutable<ILogicalOperator>> scanInpList = scan.getInputs();
scanInpList.addAll(unnest.getInputs());
opRef.setValue(scan);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexDatasource.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexDatasource.java
index 3351075..e5d78a4 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexDatasource.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexDatasource.java
@@ -18,11 +18,16 @@
*/
package org.apache.asterix.app.function;
+import java.util.Objects;
+
import org.apache.asterix.common.cluster.IClusterStateManager;
+import org.apache.asterix.external.api.IDataParserFactory;
+import org.apache.asterix.external.parser.factory.JSONDataParserFactory;
import org.apache.asterix.metadata.api.IDatasourceFunction;
import org.apache.asterix.metadata.declared.DataSourceId;
import org.apache.asterix.metadata.declared.FunctionDataSource;
import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.metadata.entities.Index;
import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
@@ -38,15 +43,21 @@
private final RecordDescriptor recDesc;
private final IBinaryComparatorFactory[] comparatorFactories;
private final AlgebricksAbsolutePartitionConstraint storageLocations;
+ private final Index index;
public DumpIndexDatasource(INodeDomain domain, IndexDataflowHelperFactory indexDataflowHelperFactory,
RecordDescriptor recDesc, IBinaryComparatorFactory[] comparatorFactories,
- AlgebricksAbsolutePartitionConstraint storageLocations) throws AlgebricksException {
+ AlgebricksAbsolutePartitionConstraint storageLocations, Index index) throws AlgebricksException {
super(DUMP_INDEX_DATASOURCE_ID, DumpIndexRewriter.DUMP_INDEX, domain);
this.indexDataflowHelperFactory = indexDataflowHelperFactory;
this.recDesc = recDesc;
this.comparatorFactories = comparatorFactories;
this.storageLocations = storageLocations;
+ this.index = index;
+ }
+
+ public Index getIndex() {
+ return index;
}
@Override
@@ -59,4 +70,18 @@
AlgebricksAbsolutePartitionConstraint locations) {
return new DumpIndexFunction(locations, indexDataflowHelperFactory, recDesc, comparatorFactories);
}
+
+ @Override
+ protected IDataParserFactory createDataParserFactory() {
+ return new JSONDataParserFactory();
+ }
+
+ @Override
+ public boolean sameFunctionDatasource(FunctionDataSource other) {
+ if (!Objects.equals(this.functionId, other.getFunctionId())) {
+ return false;
+ }
+ DumpIndexDatasource that = (DumpIndexDatasource) other;
+ return Objects.equals(this.index, that.getIndex());
+ }
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexReader.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexReader.java
index aed15c6..8ef094e 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexReader.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexReader.java
@@ -24,7 +24,12 @@
import org.apache.asterix.external.api.IRawRecord;
import org.apache.asterix.external.input.record.CharArrayRecord;
+import org.apache.asterix.om.base.ARecord;
+import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.base.IACollection;
+import org.apache.asterix.om.base.IACursor;
import org.apache.asterix.om.base.IAObject;
+import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
@@ -106,25 +111,67 @@
if (tag == ATypeTag.MISSING) {
continue;
}
- if (isTemporal(tag)) {
- JSONUtil.quoteAndEscape(recordBuilder, field.toString());
- } else {
- recordBuilder.append(field);
- }
+ printField(recordBuilder, field);
recordBuilder.append(",");
}
recordBuilder.deleteCharAt(recordBuilder.length() - 1);
recordBuilder.append("]}");
}
- private static boolean isTemporal(ATypeTag typeTag) {
+ private void printField(StringBuilder sb, IAObject field) {
+ ATypeTag typeTag = field.getType().getTypeTag();
switch (typeTag) {
+ case OBJECT:
+ printObject(sb, ((ARecord) field));
+ break;
+ case ARRAY:
+ case MULTISET:
+ printCollection(sb, ((IACollection) field));
+ break;
case DATE:
case TIME:
case DATETIME:
- return true;
+ JSONUtil.quoteAndEscape(recordBuilder, field.toString());
+ break;
+ case STRING:
+ JSONUtil.quoteAndEscape(recordBuilder, ((AString) field).getStringValue());
+ break;
+ case MISSING:
+ break;
default:
- return false;
+ sb.append(field);
}
}
+
+ private void printObject(StringBuilder sb, ARecord record) {
+ sb.append("{ ");
+ int num = record.numberOfFields();
+ ARecordType type = record.getType();
+ for (int i = 0; i < num; i++) {
+ if (i > 0) {
+ sb.append(", ");
+ }
+ IAObject value = record.getValueByPos(i);
+ JSONUtil.quoteAndEscape(sb, type.getFieldNames()[i]);
+ sb.append(": ");
+ printField(sb, value);
+ }
+ sb.append(" }");
+ }
+
+ private void printCollection(StringBuilder sb, IACollection collection) {
+ IACursor cursor = collection.getCursor();
+ sb.append("[ ");
+ boolean first = true;
+ while (cursor.next()) {
+ IAObject element = cursor.get();
+ if (first) {
+ first = false;
+ } else {
+ sb.append(", ");
+ }
+ printField(sb, element);
+ }
+ sb.append(" ]");
+ }
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexRewriter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexRewriter.java
index 30eaf93..b5ea73f 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexRewriter.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexRewriter.java
@@ -25,6 +25,7 @@
import org.apache.asterix.metadata.declared.MetadataProvider;
import org.apache.asterix.metadata.entities.Dataset;
import org.apache.asterix.metadata.entities.Index;
+import org.apache.asterix.metadata.utils.ISecondaryIndexOperationsHelper;
import org.apache.asterix.metadata.utils.SecondaryIndexOperationsHelper;
import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -63,7 +64,7 @@
if (index.isPrimaryIndex()) {
throw new CompilationException(ErrorCode.OPERATION_NOT_SUPPORTED_ON_PRIMARY_INDEX, loc, indexName);
}
- SecondaryIndexOperationsHelper secondaryIndexHelper =
+ ISecondaryIndexOperationsHelper secondaryIndexHelper =
SecondaryIndexOperationsHelper.createIndexOperationsHelper(dataset, index, metadataProvider, loc);
IndexDataflowHelperFactory indexDataflowHelperFactory =
new IndexDataflowHelperFactory(metadataProvider.getStorageComponentProvider().getStorageManager(),
@@ -72,6 +73,6 @@
(AlgebricksAbsolutePartitionConstraint) secondaryIndexHelper.getSecondaryPartitionConstraint();
return new DumpIndexDatasource(context.getComputationNodeDomain(), indexDataflowHelperFactory,
secondaryIndexHelper.getSecondaryRecDesc(), secondaryIndexHelper.getSecondaryComparatorFactories(),
- secondaryPartitionConstraint);
+ secondaryPartitionConstraint, index);
}
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/FunctionRewriter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/FunctionRewriter.java
index dfe29c2..0b2975f 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/FunctionRewriter.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/FunctionRewriter.java
@@ -80,6 +80,12 @@
throw new CompilationException(ErrorCode.COMPILATION_ERROR, unnest.getSourceLocation(),
"No positional variables are allowed over datasource functions");
}
+ createDataScanOp(opRef, unnest, context, f);
+ return true;
+ }
+
+ protected void createDataScanOp(Mutable<ILogicalOperator> opRef, UnnestOperator unnest,
+ IOptimizationContext context, AbstractFunctionCallExpression f) throws AlgebricksException {
FunctionDataSource datasource = toDatasource(context, f);
List<LogicalVariable> variables = new ArrayList<>();
variables.add(unnest.getVariable());
@@ -89,7 +95,6 @@
scanInpList.addAll(unnest.getInputs());
opRef.setValue(scan);
context.computeAndSetTypeEnvironmentForOperator(scan);
- return true;
}
protected abstract FunctionDataSource toDatasource(IOptimizationContext context, AbstractFunctionCallExpression f)
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/JobSummariesDatasource.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/JobSummariesDatasource.java
index 6bab0cd..1a23db3 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/JobSummariesDatasource.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/JobSummariesDatasource.java
@@ -18,6 +18,8 @@
*/
package org.apache.asterix.app.function;
+import java.util.Objects;
+
import org.apache.asterix.metadata.api.IDatasourceFunction;
import org.apache.asterix.metadata.declared.DataSourceId;
import org.apache.asterix.metadata.declared.FunctionDataSource;
@@ -40,4 +42,9 @@
AlgebricksAbsolutePartitionConstraint locations) {
return new JobSummariesFunction(AlgebricksAbsolutePartitionConstraint.randomLocation(locations.getLocations()));
}
+
+ @Override
+ protected boolean sameFunctionDatasource(FunctionDataSource other) {
+ return Objects.equals(this.functionId, other.getFunctionId());
+ }
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/PingDatasource.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/PingDatasource.java
index 1b6d807..6eabda9 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/PingDatasource.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/PingDatasource.java
@@ -18,6 +18,8 @@
*/
package org.apache.asterix.app.function;
+import java.util.Objects;
+
import org.apache.asterix.metadata.api.IDatasourceFunction;
import org.apache.asterix.metadata.declared.DataSourceId;
import org.apache.asterix.metadata.declared.FunctionDataSource;
@@ -40,4 +42,8 @@
return new PingFunction(locations);
}
+ @Override
+ protected boolean sameFunctionDatasource(FunctionDataSource other) {
+ return Objects.equals(this.functionId, other.getFunctionId());
+ }
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/QueryIndexDatasource.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/QueryIndexDatasource.java
new file mode 100644
index 0000000..d7c0e6d
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/QueryIndexDatasource.java
@@ -0,0 +1,147 @@
+/*
+ * 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.app.function;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.asterix.common.cluster.IClusterStateManager;
+import org.apache.asterix.metadata.api.IDatasourceFunction;
+import org.apache.asterix.metadata.declared.DataSourceId;
+import org.apache.asterix.metadata.declared.FunctionDataSource;
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.metadata.entities.Dataset;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSource;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSourcePropertiesProvider;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IProjectionInfo;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
+import org.apache.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
+import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
+import org.apache.hyracks.algebricks.core.algebra.properties.LocalOrderProperty;
+import org.apache.hyracks.algebricks.core.algebra.properties.OrderColumn;
+import org.apache.hyracks.algebricks.core.algebra.properties.RandomPartitioningProperty;
+import org.apache.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
+import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
+
+public class QueryIndexDatasource extends FunctionDataSource {
+
+ private final Dataset ds;
+ private final String indexName;
+ private final AlgebricksAbsolutePartitionConstraint storageLocations;
+ private final int numSecKeys;
+
+ public QueryIndexDatasource(Dataset ds, String indexName, INodeDomain domain,
+ AlgebricksAbsolutePartitionConstraint storageLocations, ARecordType recType, int numSecKeys)
+ throws AlgebricksException {
+ super(createQueryIndexDataSourceId(ds, indexName), QueryIndexRewriter.QUERY_INDEX, domain, recType);
+ this.ds = ds;
+ this.indexName = indexName;
+ this.storageLocations = storageLocations;
+ this.numSecKeys = numSecKeys;
+ }
+
+ public Dataset getDataset() {
+ return ds;
+ }
+
+ public String getIndexName() {
+ return indexName;
+ }
+
+ @Override
+ protected void initSchemaType(IAType iType) {
+ ARecordType type = (ARecordType) iType;
+ IAType[] fieldTypes = type.getFieldTypes();
+ schemaTypes = new IAType[fieldTypes.length];
+ for (int i = 0; i < schemaTypes.length; i++) {
+ schemaTypes[i] = fieldTypes[i];
+ }
+ }
+
+ @Override
+ protected AlgebricksAbsolutePartitionConstraint getLocations(IClusterStateManager csm) {
+ return storageLocations;
+ }
+
+ @Override
+ public boolean isScanAccessPathALeaf() {
+ // the index scan op is not a leaf op. the ETS op will start the scan of the index. we need the ETS op below
+ // the index scan to be still generated
+ return false;
+ }
+
+ @Override
+ protected IDatasourceFunction createFunction(MetadataProvider metadataProvider,
+ AlgebricksAbsolutePartitionConstraint locations) {
+ throw new UnsupportedOperationException("query-index() does not use record reader adapter");
+ }
+
+ @Override
+ public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildDatasourceScanRuntime(
+ MetadataProvider metadataProvider, IDataSource<DataSourceId> dataSource,
+ List<LogicalVariable> scanVariables, List<LogicalVariable> projectVariables, boolean projectPushed,
+ List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars,
+ ITupleFilterFactory tupleFilterFactory, long outputLimit, IOperatorSchema opSchema,
+ IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig,
+ IProjectionInfo<?> projectionInfo) throws AlgebricksException {
+ return metadataProvider.buildBtreeRuntime(jobSpec, opSchema, typeEnv, context, true, false, null, ds, indexName,
+ null, null, true, true, false, null, null, null, tupleFilterFactory, outputLimit, false, false);
+ }
+
+ @Override
+ public IDataSourcePropertiesProvider getPropertiesProvider() {
+ return scanVariables -> {
+ List<ILocalStructuralProperty> propsLocal = new ArrayList<>(1);
+ //TODO(ali): consider primary keys?
+ List<OrderColumn> secKeys = new ArrayList<>(numSecKeys);
+ for (int i = 0; i < numSecKeys; i++) {
+ secKeys.add(new OrderColumn(scanVariables.get(i), OrderOperator.IOrder.OrderKind.ASC));
+ }
+ propsLocal.add(new LocalOrderProperty(secKeys));
+ return new StructuralPropertiesVector(new RandomPartitioningProperty(domain), propsLocal);
+ };
+ }
+
+ private static DataSourceId createQueryIndexDataSourceId(Dataset dataset, String indexName) {
+ return new DataSourceId(dataset.getDataverseName(), dataset.getDatasetName(),
+ new String[] { indexName, QueryIndexRewriter.QUERY_INDEX.getName() });
+ }
+
+ @Override
+ public boolean sameFunctionDatasource(FunctionDataSource other) {
+ if (!Objects.equals(this.functionId, other.getFunctionId())) {
+ return false;
+ }
+ QueryIndexDatasource that = (QueryIndexDatasource) other;
+ return Objects.equals(this.ds, that.getDataset()) && Objects.equals(this.indexName, that.getIndexName());
+ }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/QueryIndexRewriter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/QueryIndexRewriter.java
new file mode 100644
index 0000000..e0ff9c2
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/QueryIndexRewriter.java
@@ -0,0 +1,211 @@
+/*
+ * 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.app.function;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.asterix.common.config.DatasetConfig;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.functions.FunctionConstants;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.lang.common.util.FunctionUtil;
+import org.apache.asterix.metadata.declared.FunctionDataSource;
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.metadata.entities.Dataset;
+import org.apache.asterix.metadata.entities.Index;
+import org.apache.asterix.metadata.utils.DatasetUtil;
+import org.apache.asterix.metadata.utils.ISecondaryIndexOperationsHelper;
+import org.apache.asterix.metadata.utils.KeyFieldTypeUtil;
+import org.apache.asterix.metadata.utils.SecondaryIndexOperationsHelper;
+import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.constants.AsterixConstantValue;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.typecomputer.base.IResultTypeComputer;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.storage.am.common.dataflow.IndexDataflowHelperFactory;
+import org.apache.hyracks.util.LogRedactionUtil;
+
+public class QueryIndexRewriter extends FunctionRewriter implements IResultTypeComputer {
+
+ public static final FunctionIdentifier QUERY_INDEX =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "query-index", 3);
+ public static final QueryIndexRewriter INSTANCE = new QueryIndexRewriter(QUERY_INDEX);
+
+ private QueryIndexRewriter(FunctionIdentifier functionId) {
+ super(functionId);
+ }
+
+ @Override
+ public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env, IMetadataProvider<?, ?> mp)
+ throws AlgebricksException {
+ return computeRecType((AbstractFunctionCallExpression) expression, (MetadataProvider) mp, null, null, null);
+ }
+
+ @Override
+ public FunctionDataSource toDatasource(IOptimizationContext ctx, AbstractFunctionCallExpression f)
+ throws AlgebricksException {
+ final SourceLocation loc = f.getSourceLocation();
+ DataverseName dvName = getDataverseName(loc, f.getArguments(), 0);
+ String dsName = getString(loc, f.getArguments(), 1);
+ String idName = getString(loc, f.getArguments(), 2);
+ MetadataProvider mp = (MetadataProvider) ctx.getMetadataProvider();
+ final Dataset dataset = validateDataset(mp, dvName, dsName, loc);
+ Index index = validateIndex(f, mp, loc, dvName, dsName, idName);
+ return createQueryIndexDatasource(mp, dataset, index, loc, f);
+ }
+
+ @Override
+ protected void createDataScanOp(Mutable<ILogicalOperator> opRef, UnnestOperator unnest, IOptimizationContext ctx,
+ AbstractFunctionCallExpression f) throws AlgebricksException {
+ FunctionDataSource datasource = toDatasource(ctx, f);
+ List<LogicalVariable> variables = new ArrayList<>();
+ List<Mutable<ILogicalExpression>> closedRecArgs = new ArrayList<>();
+ MetadataProvider mp = (MetadataProvider) ctx.getMetadataProvider();
+ computeRecType(f, mp, variables, closedRecArgs, ctx);
+ DataSourceScanOperator scan = new DataSourceScanOperator(variables, datasource);
+ scan.setSourceLocation(unnest.getSourceLocation());
+ List<Mutable<ILogicalOperator>> scanInpList = scan.getInputs();
+ scanInpList.addAll(unnest.getInputs());
+ ScalarFunctionCallExpression recordCreationFunc = new ScalarFunctionCallExpression(
+ FunctionUtil.getFunctionInfo(BuiltinFunctions.CLOSED_RECORD_CONSTRUCTOR), closedRecArgs);
+ recordCreationFunc.setSourceLocation(unnest.getSourceLocation());
+ AssignOperator assignOp = new AssignOperator(unnest.getVariable(), new MutableObject<>(recordCreationFunc));
+ assignOp.getInputs().add(new MutableObject<>(scan));
+ assignOp.setSourceLocation(unnest.getSourceLocation());
+ ctx.computeAndSetTypeEnvironmentForOperator(scan);
+ ctx.computeAndSetTypeEnvironmentForOperator(assignOp);
+ opRef.setValue(assignOp);
+ }
+
+ private FunctionDataSource createQueryIndexDatasource(MetadataProvider mp, Dataset ds, Index idx,
+ SourceLocation loc, AbstractFunctionCallExpression f) throws AlgebricksException {
+ ISecondaryIndexOperationsHelper secIdxHelper =
+ SecondaryIndexOperationsHelper.createIndexOperationsHelper(ds, idx, mp, loc);
+ new IndexDataflowHelperFactory(mp.getStorageComponentProvider().getStorageManager(),
+ secIdxHelper.getSecondaryFileSplitProvider());
+ AlgebricksAbsolutePartitionConstraint secPartitionConstraint =
+ (AlgebricksAbsolutePartitionConstraint) secIdxHelper.getSecondaryPartitionConstraint();
+ INodeDomain domain = mp.findNodeDomain(ds.getNodeGroupName());
+ ARecordType recType = computeRecType(f, mp, null, null, null);
+ int numSecKeys = ((Index.ValueIndexDetails) idx.getIndexDetails()).getKeyFieldNames().size();
+ return new QueryIndexDatasource(ds, idx.getIndexName(), domain, secPartitionConstraint, recType, numSecKeys);
+ }
+
+ private ARecordType computeRecType(AbstractFunctionCallExpression f, MetadataProvider metadataProvider,
+ List<LogicalVariable> outVars, List<Mutable<ILogicalExpression>> closedRecArgs,
+ IOptimizationContext context) throws AlgebricksException {
+ final SourceLocation loc = f.getSourceLocation();
+ DataverseName dataverseName = getDataverseName(loc, f.getArguments(), 0);
+ String datasetName = getString(loc, f.getArguments(), 1);
+ String indexName = getString(loc, f.getArguments(), 2);
+ Dataset dataset = validateDataset(metadataProvider, dataverseName, datasetName, loc);
+ Index index = validateIndex(f, metadataProvider, loc, dataverseName, datasetName, indexName);
+ ARecordType dsType = (ARecordType) metadataProvider.findType(dataset);
+ ARecordType metaType = DatasetUtil.getMetaType(metadataProvider, dataset);
+ List<IAType> dsKeyTypes = KeyFieldTypeUtil.getPartitoningKeyTypes(dataset, dsType, metaType);
+ List<Pair<IAType, Boolean>> secKeyTypes = KeyFieldTypeUtil.getBTreeIndexKeyTypes(index, dsType, metaType);
+ int numPrimaryKeys = dsKeyTypes.size();
+ int numSecKeys = secKeyTypes.size();
+ String[] fieldNames = new String[numSecKeys + numPrimaryKeys];
+ IAType[] fieldTypes = new IAType[numSecKeys + numPrimaryKeys];
+ int keyIdx = 0;
+ boolean overridingKeyFieldTypes = index.getIndexDetails().isOverridingKeyFieldTypes();
+ for (int i = 0; i < numSecKeys; i++, keyIdx++) {
+ IAType secKeyType = secKeyTypes.get(i).first;
+ Boolean makeOptional = secKeyTypes.get(i).second;
+ fieldTypes[keyIdx] =
+ overridingKeyFieldTypes || makeOptional ? AUnionType.createUnknownableType(secKeyType) : secKeyType;
+ fieldNames[keyIdx] = "SK" + i;
+ setAssignVarsExprs(outVars, closedRecArgs, context, loc, fieldNames, keyIdx);
+ }
+ for (int k = 0; k < numPrimaryKeys; k++, keyIdx++) {
+ fieldTypes[keyIdx] = dsKeyTypes.get(k);
+ fieldNames[keyIdx] = "PK" + k;
+ setAssignVarsExprs(outVars, closedRecArgs, context, loc, fieldNames, keyIdx);
+ }
+ return new ARecordType("", fieldNames, fieldTypes, false);
+ }
+
+ private void setAssignVarsExprs(List<LogicalVariable> outVars, List<Mutable<ILogicalExpression>> closedRecArgs,
+ IOptimizationContext context, SourceLocation loc, String[] fieldNames, int n) {
+ if (context != null) {
+ LogicalVariable logicalVariable = context.newVar();
+ outVars.add(logicalVariable);
+ ConstantExpression nameExpr = new ConstantExpression(new AsterixConstantValue(new AString(fieldNames[n])));
+ VariableReferenceExpression varRefExpr = new VariableReferenceExpression(logicalVariable);
+ nameExpr.setSourceLocation(loc);
+ varRefExpr.setSourceLocation(loc);
+ closedRecArgs.add(new MutableObject<>(nameExpr));
+ closedRecArgs.add(new MutableObject<>(varRefExpr));
+ }
+ }
+
+ private static Dataset validateDataset(MetadataProvider mp, DataverseName dvName, String dsName, SourceLocation loc)
+ throws AlgebricksException {
+ Dataset dataset = mp.findDataset(dvName, dsName);
+ if (dataset == null) {
+ throw new CompilationException(ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE, loc, dsName, dvName);
+ }
+ return dataset;
+ }
+
+ private static Index validateIndex(AbstractFunctionCallExpression f, MetadataProvider mp, SourceLocation loc,
+ DataverseName dvName, String dsName, String idxName) throws AlgebricksException {
+ Index index = mp.getIndex(dvName, dsName, idxName);
+ if (index == null) {
+ throw new CompilationException(ErrorCode.UNKNOWN_INDEX, loc, idxName);
+ }
+ if (index.isPrimaryIndex()) {
+ throw new CompilationException(ErrorCode.OPERATION_NOT_SUPPORTED_ON_PRIMARY_INDEX, loc, idxName);
+ }
+ DatasetConfig.IndexType idxType = index.getIndexType();
+ // currently, only normal secondary indexes are supported
+ if (idxType != DatasetConfig.IndexType.BTREE || Index.IndexCategory.of(idxType) != Index.IndexCategory.VALUE
+ || index.isPrimaryKeyIndex()) {
+ throw new CompilationException(ErrorCode.COMPILATION_FUNC_EXPRESSION_CANNOT_UTILIZE_INDEX,
+ f.getSourceLocation(), LogRedactionUtil.userData(f.toString()));
+ }
+ return index;
+ }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/StorageComponentsDatasource.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/StorageComponentsDatasource.java
index 6157412..c470a20 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/StorageComponentsDatasource.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/StorageComponentsDatasource.java
@@ -18,6 +18,8 @@
*/
package org.apache.asterix.app.function;
+import java.util.Objects;
+
import org.apache.asterix.metadata.api.IDatasourceFunction;
import org.apache.asterix.metadata.declared.DataSourceId;
import org.apache.asterix.metadata.declared.FunctionDataSource;
@@ -38,9 +40,22 @@
this.datasetId = datasetId;
}
+ public int getDatasetId() {
+ return datasetId;
+ }
+
@Override
protected IDatasourceFunction createFunction(MetadataProvider metadataProvider,
AlgebricksAbsolutePartitionConstraint locations) {
return new StorageComponentsFunction(locations, datasetId);
}
+
+ @Override
+ public boolean sameFunctionDatasource(FunctionDataSource other) {
+ if (!Objects.equals(this.functionId, other.getFunctionId())) {
+ return false;
+ }
+ StorageComponentsDatasource that = (StorageComponentsDatasource) other;
+ return Objects.equals(this.datasetId, that.getDatasetId());
+ }
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/TPCDSAllTablesDataGeneratorDatasource.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/TPCDSAllTablesDataGeneratorDatasource.java
index 82303fc..489b052 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/TPCDSAllTablesDataGeneratorDatasource.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/TPCDSAllTablesDataGeneratorDatasource.java
@@ -18,6 +18,8 @@
*/
package org.apache.asterix.app.function;
+import java.util.Objects;
+
import org.apache.asterix.common.cluster.IClusterStateManager;
import org.apache.asterix.metadata.api.IDatasourceFunction;
import org.apache.asterix.metadata.declared.DataSourceId;
@@ -42,6 +44,10 @@
this.scalingFactor = scalingFactor;
}
+ public double getScalingFactor() {
+ return scalingFactor;
+ }
+
/**
* This ensures that each function will have a unique DataSourceId by passing the table name as part of the
* DataSourceId. This eliminates the issue of creating a single function even though multiple functions calls
@@ -64,6 +70,15 @@
@Override
protected AlgebricksAbsolutePartitionConstraint getLocations(IClusterStateManager csm) {
- return new AlgebricksAbsolutePartitionConstraint(csm.getClusterLocations().getLocations());
+ return csm.getSortedClusterLocations();
+ }
+
+ @Override
+ public boolean sameFunctionDatasource(FunctionDataSource other) {
+ if (!Objects.equals(this.functionId, other.getFunctionId())) {
+ return false;
+ }
+ TPCDSAllTablesDataGeneratorDatasource that = (TPCDSAllTablesDataGeneratorDatasource) other;
+ return Objects.equals(this.scalingFactor, that.getScalingFactor());
}
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/TPCDSSingleTableDataGeneratorDatasource.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/TPCDSSingleTableDataGeneratorDatasource.java
index 458bff6..c90dff1 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/TPCDSSingleTableDataGeneratorDatasource.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/TPCDSSingleTableDataGeneratorDatasource.java
@@ -18,6 +18,8 @@
*/
package org.apache.asterix.app.function;
+import java.util.Objects;
+
import org.apache.asterix.common.cluster.IClusterStateManager;
import org.apache.asterix.metadata.api.IDatasourceFunction;
import org.apache.asterix.metadata.declared.DataSourceId;
@@ -44,6 +46,14 @@
this.scalingFactor = scalingFactor;
}
+ public String getTableName() {
+ return tableName;
+ }
+
+ public double getScalingFactor() {
+ return scalingFactor;
+ }
+
/**
* This ensures that each function will have a unique DataSourceId by passing the table name as part of the
* DataSourceId. This eliminates the issue of creating a single function even though multiple functions calls
@@ -68,6 +78,16 @@
@Override
protected AlgebricksAbsolutePartitionConstraint getLocations(IClusterStateManager csm) {
- return new AlgebricksAbsolutePartitionConstraint(csm.getClusterLocations().getLocations());
+ return csm.getSortedClusterLocations();
+ }
+
+ @Override
+ public boolean sameFunctionDatasource(FunctionDataSource other) {
+ if (!Objects.equals(this.functionId, other.getFunctionId())) {
+ return false;
+ }
+ TPCDSSingleTableDataGeneratorDatasource that = (TPCDSSingleTableDataGeneratorDatasource) other;
+ return Objects.equals(this.tableName, that.getTableName())
+ && Objects.equals(this.scalingFactor, that.getScalingFactor());
}
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CancelQueryRequest.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CancelQueryRequest.java
index 65d1039..6570041 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CancelQueryRequest.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CancelQueryRequest.java
@@ -69,10 +69,8 @@
}
}
}
- if (LOGGER.isDebugEnabled()) {
- LOGGER.debug("Sending CancelQueryResponse to {}. requestId:{}, uuid:{}, contextId:{}, status:{}", nodeId,
- requestId, uuid, contextId, status);
- }
+ LOGGER.debug("sending CancelQueryResponse to {}. reqId:{}, uuid:{}, contextId:{}, status:{}", nodeId, requestId,
+ uuid, contextId, status);
CancelQueryResponse response = new CancelQueryResponse(reqId, status);
CCMessageBroker messageBroker = (CCMessageBroker) appCtx.getServiceContext().getMessageBroker();
try {
@@ -82,4 +80,9 @@
}
}
+ @Override
+ public String toString() {
+ return "CancelQueryRequest{from='" + nodeId + "', reqId=" + reqId + ", uuid='" + uuid + "', contextId='"
+ + contextId + "'}";
+ }
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CancelQueryResponse.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CancelQueryResponse.java
index d65ae31..68d3430 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CancelQueryResponse.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CancelQueryResponse.java
@@ -49,4 +49,8 @@
return status;
}
+ @Override
+ public String toString() {
+ return "CancelQueryResponse{reqId=" + reqId + ", status=" + status + '}';
+ }
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java
index 05bc87b..e314177 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
@@ -243,7 +243,8 @@
@Override
public String toString() {
- return String.format("%s(id=%s, from=%s, uuid=%s): %s", getClass().getSimpleName(), requestMessageId,
- requestNodeId, requestReference.getUuid(), LogRedactionUtil.statement(statementsText));
+ return String.format("%s(id=%s, from=%s, uuid=%s, clientContextID=%s): %s", getClass().getSimpleName(),
+ requestMessageId, requestNodeId, requestReference.getUuid(), clientContextID,
+ LogRedactionUtil.statement(statementsText));
}
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/HaltCallback.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/HaltCallback.java
index 9802001..bb689ac 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/HaltCallback.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/HaltCallback.java
@@ -41,6 +41,8 @@
@Override
public void operationFailed(ILSMIOOperation operation, Throwable t) {
LOGGER.error("Operation {} has failed", operation, t);
- ExitUtil.halt(ExitUtil.EC_IO_OPERATION_FAILED);
+ if (operation.getIOOpertionType() != ILSMIOOperation.LSMIOOperationType.REPLICATE) {
+ ExitUtil.halt(ExitUtil.EC_IO_OPERATION_FAILED);
+ }
}
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/IndexCheckpointManager.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/IndexCheckpointManager.java
index 290734f..34f0cb1 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/IndexCheckpointManager.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/IndexCheckpointManager.java
@@ -128,6 +128,11 @@
}
@Override
+ public synchronized boolean isValidIndex() throws HyracksDataException {
+ return getCheckpointCount() > 0;
+ }
+
+ @Override
public long getValidComponentSequence() throws HyracksDataException {
if (getCheckpointCount() > 0) {
return getLatest().getValidComponentSequence();
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
index d783f99..61656b7 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
@@ -21,12 +21,15 @@
import java.io.IOException;
import java.rmi.RemoteException;
import java.rmi.server.UnicastRemoteObject;
+import java.util.Collection;
+import java.util.OptionalInt;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ExecutorService;
import org.apache.asterix.active.ActiveManager;
+import org.apache.asterix.app.result.ResultReader;
import org.apache.asterix.common.api.IConfigValidator;
import org.apache.asterix.common.api.IConfigValidatorFactory;
import org.apache.asterix.common.api.ICoordinationService;
@@ -86,6 +89,8 @@
import org.apache.hyracks.api.lifecycle.ILifeCycleComponent;
import org.apache.hyracks.api.lifecycle.ILifeCycleComponentManager;
import org.apache.hyracks.api.network.INetworkSecurityManager;
+import org.apache.hyracks.api.result.IResultSet;
+import org.apache.hyracks.client.result.ResultSet;
import org.apache.hyracks.control.common.controllers.NCConfig;
import org.apache.hyracks.control.nc.NodeControllerService;
import org.apache.hyracks.ipc.impl.HyracksConnection;
@@ -108,6 +113,7 @@
import org.apache.hyracks.storage.common.file.ILocalResourceRepositoryFactory;
import org.apache.hyracks.storage.common.file.IResourceIdFactory;
import org.apache.hyracks.util.MaintainedThreadNameExecutorService;
+import org.apache.hyracks.util.NetworkUtil;
import org.apache.hyracks.util.cache.CacheManager;
import org.apache.hyracks.util.cache.ICacheManager;
import org.apache.logging.log4j.Level;
@@ -147,13 +153,15 @@
private final NCExtensionManager ncExtensionManager;
private final IStorageComponentProvider componentProvider;
private final IPersistedResourceRegistry persistedResourceRegistry;
- private IHyracksClientConnection hcc;
+ private volatile HyracksConnection hcc;
+ private volatile ResultSet resultSet;
private IIndexCheckpointManagerProvider indexCheckpointManagerProvider;
private IReplicaManager replicaManager;
private IReceptionist receptionist;
private ICacheManager cacheManager;
private IConfigValidator configValidator;
private IDiskWriteRateLimiterProvider diskWriteRateLimiterProvider;
+ private Integer metadataPartitionId;
public NCAppRuntimeContext(INCServiceContext ncServiceContext, NCExtensionManager extensionManager,
IPropertiesFactory propertiesFactory) {
@@ -223,6 +231,7 @@
datasetLifecycleManager =
new DatasetLifecycleManager(storageProperties, localResourceRepository, txnSubsystem.getLogManager(),
virtualBufferCache, indexCheckpointManagerProvider, ioManager.getIODevices().size());
+ localResourceRepository.setDatasetLifecycleManager(datasetLifecycleManager);
final String nodeId = getServiceContext().getNodeId();
final Set<Integer> nodePartitions = metadataProperties.getNodePartitions(nodeId);
replicaManager = new ReplicaManager(this, nodePartitions);
@@ -436,20 +445,14 @@
@Override
public void initializeMetadata(boolean newUniverse, int partitionId) throws Exception {
LOGGER.info("Bootstrapping metadata");
+ metadataPartitionId = partitionId;
MetadataNode.INSTANCE.initialize(this, ncExtensionManager.getMetadataTupleTranslatorProvider(),
ncExtensionManager.getMetadataExtensions(), partitionId);
- //noinspection unchecked
- ConcurrentHashMap<CcId, IAsterixStateProxy> proxyMap =
- (ConcurrentHashMap<CcId, IAsterixStateProxy>) getServiceContext().getDistributedState();
- if (proxyMap == null) {
- throw new IllegalStateException("Metadata node cannot access distributed state");
- }
-
// This is a special case, we just give the metadataNode directly.
// This way we can delay the registration of the metadataNode until
// it is completely initialized.
- MetadataManager.initialize(proxyMap.values(), MetadataNode.INSTANCE);
+ MetadataManager.initialize(getAsterixStateProxies(), MetadataNode.INSTANCE);
MetadataBootstrap.startUniverse(getServiceContext(), newUniverse);
MetadataBootstrap.startDDLRecovery();
ncExtensionManager.initializeMetadata(getServiceContext());
@@ -477,6 +480,17 @@
metadataNodeStub = null;
}
+ protected Collection<IAsterixStateProxy> getAsterixStateProxies() {
+ //noinspection unchecked
+ ConcurrentHashMap<CcId, IAsterixStateProxy> proxyMap =
+ (ConcurrentHashMap<CcId, IAsterixStateProxy>) getServiceContext().getDistributedState();
+ if (proxyMap == null) {
+ throw new IllegalStateException("Metadata node cannot access distributed state");
+ }
+
+ return proxyMap.values();
+ }
+
@Override
public synchronized void bindMetadataNodeStub(CcId ccId) throws RemoteException {
if (metadataNodeStub == null) {
@@ -505,15 +519,22 @@
@Override
public IHyracksClientConnection getHcc() throws HyracksDataException {
- if (hcc == null || !hcc.isConnected()) {
+ HyracksConnection hc = hcc;
+ if (hc == null || !hc.isConnected()) {
synchronized (this) {
- if (hcc == null || !hcc.isConnected()) {
+ hc = hcc;
+ if (hc == null || !hc.isConnected()) {
try {
+ ResultSet rs = resultSet;
+ resultSet = null;
+ NetworkUtil.closeQuietly(rs);
+
NodeControllerService ncSrv = (NodeControllerService) ncServiceContext.getControllerService();
// TODO(mblow): multicc
CcId primaryCcId = ncSrv.getPrimaryCcId();
ClusterControllerInfo ccInfo = ncSrv.getNodeParameters(primaryCcId).getClusterControllerInfo();
- hcc = new HyracksConnection(ccInfo.getClientNetAddress(), ccInfo.getClientNetPort(),
+ NetworkUtil.closeQuietly(hc);
+ hcc = hc = new HyracksConnection(ccInfo.getClientNetAddress(), ccInfo.getClientNetPort(),
ncSrv.getNetworkSecurityManager().getSocketChannelFactory());
} catch (Exception e) {
throw HyracksDataException.create(e);
@@ -521,7 +542,26 @@
}
}
}
- return hcc;
+ return hc;
+ }
+
+ @Override
+ public IResultSet getResultSet() throws HyracksDataException {
+ ResultSet rs = resultSet;
+ if (rs == null) {
+ synchronized (this) {
+ rs = resultSet;
+ if (rs == null) {
+ try {
+ resultSet = rs = ResultReader.createResultSet(getHcc(), ncServiceContext.getControllerService(),
+ compilerProperties);
+ } catch (Exception e) {
+ throw HyracksDataException.create(e);
+ }
+ }
+ }
+ }
+ return rs;
}
@Override
@@ -599,4 +639,9 @@
public IDiskWriteRateLimiterProvider getDiskWriteRateLimiterProvider() {
return diskWriteRateLimiterProvider;
}
+
+ @Override
+ public OptionalInt getMetadataPartitionId() {
+ return metadataPartitionId == null ? OptionalInt.empty() : OptionalInt.of(metadataPartitionId);
+ }
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java
index 2ca3fbc..4bc4d62 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java
@@ -615,9 +615,6 @@
boolean infoEnabled = LOGGER.isInfoEnabled();
// check if the transaction actually wrote some logs.
if (firstLSN == TransactionManagementConstants.LogManagerConstants.TERMINAL_LSN || firstLSN > lastLSN) {
- if (infoEnabled) {
- LOGGER.info("no need to rollback as there were no operations by " + txnContext.getTxnId());
- }
return;
}
if (infoEnabled) {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/OperatorResourcesComputer.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/OperatorResourcesComputer.java
index a16ac84..9e77922 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/OperatorResourcesComputer.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/OperatorResourcesComputer.java
@@ -18,25 +18,42 @@
*/
package org.apache.asterix.app.resource;
+import org.apache.asterix.common.config.CompilerProperties;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.base.IPhysicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
import org.apache.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
import org.apache.hyracks.algebricks.core.algebra.properties.LocalMemoryRequirements;
+import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
public class OperatorResourcesComputer {
+ private static final Logger LOGGER = LogManager.getLogger();
public static final int MIN_OPERATOR_CORES = 1;
private static final long MAX_BUFFER_PER_CONNECTION = 1L;
private final int numComputationPartitions;
private final long frameSize;
+ private final ExpressionMemoryComputer exprMemoryComputer;
+ private final CompilerProperties compilerProperties;
- public OperatorResourcesComputer(int numComputationPartitions, long frameSize) {
+ public OperatorResourcesComputer(int numComputationPartitions, long frameSize,
+ CompilerProperties compilerProperties) {
this.numComputationPartitions = numComputationPartitions;
this.frameSize = frameSize;
+ this.exprMemoryComputer = new ExpressionMemoryComputer();
+ this.compilerProperties = compilerProperties;
}
public int getOperatorRequiredCores(ILogicalOperator operator) {
@@ -52,10 +69,22 @@
return getExchangeRequiredMemory((ExchangeOperator) operator);
} else {
IPhysicalOperator physOp = ((AbstractLogicalOperator) operator).getPhysicalOperator();
- return getOperatorRequiredMemory(operator.getExecutionMode(), physOp.getLocalMemoryRequirements());
+ return getOperatorRequiredMemory(operator.getExecutionMode(), physOp.getLocalMemoryRequirements())
+ + getOperatorExpressionsRequiredMemory(operator);
}
}
+ private long getOperatorExpressionsRequiredMemory(ILogicalOperator operator) {
+ exprMemoryComputer.reset(operator);
+ try {
+ operator.acceptExpressionTransform(exprMemoryComputer);
+ } catch (Throwable e) {
+ // ignore
+ LOGGER.warn("encountered error while computing operator expressions required memory", e);
+ }
+ return exprMemoryComputer.requiredMemory;
+ }
+
private long getOperatorRequiredMemory(AbstractLogicalOperator.ExecutionMode opExecMode, long memorySize) {
if (opExecMode == AbstractLogicalOperator.ExecutionMode.PARTITIONED
|| opExecMode == AbstractLogicalOperator.ExecutionMode.LOCAL) {
@@ -78,4 +107,42 @@
}
return 2L * MAX_BUFFER_PER_CONNECTION * numComputationPartitions * numComputationPartitions * frameSize;
}
+
+ class ExpressionMemoryComputer implements ILogicalExpressionReferenceTransform {
+
+ private long requiredMemory;
+ private ILogicalOperator operator;
+
+ public ExpressionMemoryComputer() {
+ }
+
+ @Override
+ public boolean transform(Mutable<ILogicalExpression> expression) throws AlgebricksException {
+ ILogicalExpression expr = expression.getValue();
+ if (expr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+ AbstractFunctionCallExpression funExpr = (AbstractFunctionCallExpression) expr;
+ if (funExpr.getKind() == AbstractFunctionCallExpression.FunctionKind.AGGREGATE) {
+ if (isMedian(funExpr.getFunctionIdentifier())) {
+ requiredMemory +=
+ (compilerProperties.getSortMemorySize() * numCompute(operator.getExecutionMode()));
+ }
+ }
+ }
+ return false;
+ }
+
+ private int numCompute(AbstractLogicalOperator.ExecutionMode executionMode) {
+ return (executionMode == AbstractLogicalOperator.ExecutionMode.PARTITIONED
+ || executionMode == AbstractLogicalOperator.ExecutionMode.LOCAL) ? numComputationPartitions : 1;
+ }
+
+ private boolean isMedian(FunctionIdentifier funId) {
+ return BuiltinFunctions.LOCAL_SQL_MEDIAN.equals(funId) || BuiltinFunctions.SQL_MEDIAN.equals(funId);
+ }
+
+ public void reset(ILogicalOperator op) {
+ requiredMemory = 0;
+ operator = op;
+ }
+ }
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/JobResultCallback.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/JobResultCallback.java
index 05073f9..201a470 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/JobResultCallback.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/JobResultCallback.java
@@ -23,8 +23,8 @@
import java.util.Iterator;
import java.util.Set;
-import org.apache.asterix.api.common.ResultMetadata;
import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.asterix.translator.ResultMetadata;
import org.apache.hyracks.api.exceptions.Warning;
import org.apache.hyracks.api.job.JobFlag;
import org.apache.hyracks.api.job.JobId;
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultReader.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultReader.java
index 1acae87..56dfd5e 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultReader.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultReader.java
@@ -18,6 +18,8 @@
*/
package org.apache.asterix.app.result;
+import org.apache.asterix.common.config.CompilerProperties;
+import org.apache.hyracks.api.client.IHyracksClientConnection;
import org.apache.hyracks.api.comm.IFrame;
import org.apache.hyracks.api.comm.IFrameTupleAccessor;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -27,12 +29,14 @@
import org.apache.hyracks.api.result.IResultSetReader;
import org.apache.hyracks.api.result.ResultJobRecord.Status;
import org.apache.hyracks.api.result.ResultSetId;
+import org.apache.hyracks.api.service.IControllerService;
+import org.apache.hyracks.client.result.ResultSet;
import org.apache.hyracks.dataflow.common.comm.io.ResultFrameTupleAccessor;
public class ResultReader {
- private IResultSetReader reader;
+ private final IResultSetReader reader;
- private IFrameTupleAccessor frameTupleAccessor;
+ private final IFrameTupleAccessor frameTupleAccessor;
// Number of parallel result reader buffers
public static final int NUM_READERS = 1;
@@ -57,4 +61,10 @@
public IResultMetadata getMetadata() {
return reader.getResultMetadata();
}
+
+ public static ResultSet createResultSet(IHyracksClientConnection hcc, IControllerService srv,
+ CompilerProperties compilerProperties) throws Exception {
+ return new ResultSet(hcc, srv.getNetworkSecurityManager().getSocketChannelFactory(),
+ compilerProperties.getFrameSize(), ResultReader.NUM_READERS);
+ }
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/SignaturePrinter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/SignaturePrinter.java
index 048584c..3f1e632 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/SignaturePrinter.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/SignaturePrinter.java
@@ -22,7 +22,6 @@
import java.util.LinkedHashSet;
import java.util.List;
-import org.apache.asterix.api.common.ResultMetadata;
import org.apache.asterix.api.http.server.ResultUtil;
import org.apache.asterix.common.annotations.IRecordTypeAnnotation;
import org.apache.asterix.common.annotations.RecordFieldOrderAnnotation;
@@ -34,6 +33,7 @@
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.IAType;
import org.apache.asterix.translator.ExecutionPlans;
+import org.apache.asterix.translator.ResultMetadata;
import org.apache.hyracks.algebricks.common.utils.Pair;
import org.apache.hyracks.util.JSONUtil;
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
index df7d8d6..31b903b 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
@@ -111,6 +111,8 @@
import org.apache.asterix.lang.common.expression.TypeReferenceExpression;
import org.apache.asterix.lang.common.expression.VariableExpr;
import org.apache.asterix.lang.common.statement.AdapterDropStatement;
+import org.apache.asterix.lang.common.statement.AnalyzeDropStatement;
+import org.apache.asterix.lang.common.statement.AnalyzeStatement;
import org.apache.asterix.lang.common.statement.CompactStatement;
import org.apache.asterix.lang.common.statement.ConnectFeedStatement;
import org.apache.asterix.lang.common.statement.CreateAdapterStatement;
@@ -195,6 +197,7 @@
import org.apache.asterix.metadata.utils.KeyFieldTypeUtil;
import org.apache.asterix.metadata.utils.MetadataConstants;
import org.apache.asterix.metadata.utils.MetadataUtil;
+import org.apache.asterix.metadata.utils.SampleOperationsHelper;
import org.apache.asterix.metadata.utils.TypeUtil;
import org.apache.asterix.om.base.ANull;
import org.apache.asterix.om.base.IAObject;
@@ -208,6 +211,7 @@
import org.apache.asterix.runtime.fulltext.FullTextConfigDescriptor;
import org.apache.asterix.runtime.fulltext.IFullTextFilterDescriptor;
import org.apache.asterix.runtime.fulltext.StopwordsFullTextFilterDescriptor;
+import org.apache.asterix.runtime.operators.DatasetStreamStats;
import org.apache.asterix.transaction.management.service.transaction.DatasetIdFactory;
import org.apache.asterix.translator.AbstractLangTranslator;
import org.apache.asterix.translator.ClientRequest;
@@ -235,13 +239,11 @@
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.common.utils.Pair;
import org.apache.hyracks.algebricks.common.utils.Triple;
+import org.apache.hyracks.algebricks.core.algebra.base.Counter;
import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression.FunctionKind;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
import org.apache.hyracks.algebricks.data.IAWriterFactory;
-import org.apache.hyracks.algebricks.data.IResultSerializerFactoryProvider;
-import org.apache.hyracks.algebricks.runtime.serializer.ResultSerializerFactoryProvider;
-import org.apache.hyracks.algebricks.runtime.writers.PrinterBasedWriterFactory;
import org.apache.hyracks.api.client.IClusterInfoCollector;
import org.apache.hyracks.api.client.IHyracksClientConnection;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -253,6 +255,7 @@
import org.apache.hyracks.api.job.JobFlag;
import org.apache.hyracks.api.job.JobId;
import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
import org.apache.hyracks.api.result.IResultSet;
import org.apache.hyracks.api.result.ResultSetId;
import org.apache.hyracks.control.cc.ClusterControllerService;
@@ -330,10 +333,8 @@
public void compileAndExecute(IHyracksClientConnection hcc, IRequestParameters requestParameters) throws Exception {
validateStatements(requestParameters);
trackRequest(requestParameters);
- int resultSetIdCounter = 0;
+ Counter resultSetIdCounter = new Counter(0);
FileSplit outputFile = null;
- IAWriterFactory writerFactory = PrinterBasedWriterFactory.INSTANCE;
- IResultSerializerFactoryProvider resultSerializerFactoryProvider = ResultSerializerFactoryProvider.INSTANCE;
String threadName = Thread.currentThread().getName();
Thread.currentThread().setName(
QueryTranslator.class.getSimpleName() + ":" + requestParameters.getRequestReference().getUuid());
@@ -353,8 +354,8 @@
}
validateOperation(appCtx, activeDataverse, stmt);
MetadataProvider metadataProvider = MetadataProvider.create(appCtx, activeDataverse);
- configureMetadataProvider(metadataProvider, config, resultSerializerFactoryProvider, writerFactory,
- outputFile, requestParameters, stmt);
+ configureMetadataProvider(metadataProvider, config, resultSetIdCounter, outputFile, requestParameters,
+ stmt);
IStatementRewriter stmtRewriter = rewriterFactory.createStatementRewriter();
rewriteStatement(stmt, stmtRewriter, metadataProvider); // Rewrite the statement's AST.
Statement.Kind kind = stmt.getKind();
@@ -444,7 +445,7 @@
case INSERT:
case UPSERT:
if (((InsertStatement) stmt).getReturnExpression() != null) {
- metadataProvider.setResultSetId(new ResultSetId(resultSetIdCounter++));
+ metadataProvider.setResultSetId(new ResultSetId(resultSetIdCounter.getAndInc()));
metadataProvider.setResultAsyncMode(resultDelivery == ResultDelivery.ASYNC
|| resultDelivery == ResultDelivery.DEFERRED);
metadataProvider.setMaxResultReads(maxResultReads);
@@ -480,7 +481,7 @@
handleCreateFeedPolicyStatement(metadataProvider, stmt);
break;
case QUERY:
- metadataProvider.setResultSetId(new ResultSetId(resultSetIdCounter++));
+ metadataProvider.setResultSetId(new ResultSetId(resultSetIdCounter.getAndInc()));
metadataProvider.setResultAsyncMode(
resultDelivery == ResultDelivery.ASYNC || resultDelivery == ResultDelivery.DEFERRED);
metadataProvider.setMaxResultReads(maxResultReads);
@@ -490,6 +491,12 @@
handleQuery(metadataProvider, (Query) stmt, hcc, resultSet, resultDelivery, outMetadata, stats,
requestParameters, stmtParams, stmtRewriter);
break;
+ case ANALYZE:
+ handleAnalyzeStatement(metadataProvider, stmt, hcc, requestParameters);
+ break;
+ case ANALYZE_DROP:
+ handleAnalyzeDropStatement(metadataProvider, stmt, hcc, requestParameters);
+ break;
case COMPACT:
handleCompactStatement(metadataProvider, stmt, hcc);
break;
@@ -505,7 +512,10 @@
case EXTENSION:
final ExtensionStatement extStmt = (ExtensionStatement) stmt;
statementProperties.setName(extStmt.getName());
- extStmt.handle(hcc, this, requestParameters, metadataProvider, resultSetIdCounter);
+ if (!isCompileOnly()) {
+ extStmt.handle(hcc, this, requestParameters, metadataProvider,
+ resultSetIdCounter.getAndInc());
+ }
break;
default:
throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, stmt.getSourceLocation(),
@@ -522,14 +532,13 @@
}
protected void configureMetadataProvider(MetadataProvider metadataProvider, Map<String, String> config,
- IResultSerializerFactoryProvider resultSerializerFactoryProvider, IAWriterFactory writerFactory,
- FileSplit outputFile, IRequestParameters requestParameters, Statement statement) {
+ Counter resultSetIdCounter, FileSplit outputFile, IRequestParameters requestParameters,
+ Statement statement) {
if (statement.getKind() == Statement.Kind.QUERY && requestParameters.isSQLCompatMode()) {
metadataProvider.getConfig().put(SqlppQueryRewriter.SQL_COMPAT_OPTION, Boolean.TRUE.toString());
}
metadataProvider.getConfig().putAll(config);
- metadataProvider.setWriterFactory(writerFactory);
- metadataProvider.setResultSerializerFactoryProvider(resultSerializerFactoryProvider);
+ metadataProvider.setResultSetIdCounter(resultSetIdCounter);
metadataProvider.setOutputFile(outputFile);
}
@@ -610,6 +619,9 @@
CreateDataverseStatement stmtCreateDataverse = (CreateDataverseStatement) stmt;
DataverseName dvName = stmtCreateDataverse.getDataverseName();
metadataProvider.validateDataverseName(dvName, stmtCreateDataverse.getSourceLocation());
+ if (isCompileOnly()) {
+ return;
+ }
lockUtil.createDataverseBegin(lockManager, metadataProvider.getLocks(), dvName);
try {
doCreateDataverseStatement(metadataProvider, stmtCreateDataverse, requestParameters);
@@ -713,6 +725,9 @@
String compactionPolicy = dd.getCompactionPolicy();
boolean defaultCompactionPolicy = compactionPolicy == null;
+ if (isCompileOnly()) {
+ return;
+ }
lockUtil.createDatasetBegin(lockManager, metadataProvider.getLocks(), dataverseName, datasetName,
itemTypeDataverseName, itemTypeName, itemTypeAnonymous, metaItemTypeDataverseName, metaItemTypeName,
metaItemTypeAnonymous, nodegroupName, compactionPolicy, defaultCompactionPolicy, dd.getDatasetType(),
@@ -1061,6 +1076,9 @@
metadataProvider.validateDatabaseObjectName(stmtCreateIndex.getDataverseName(), indexName,
stmt.getSourceLocation());
DataverseName dataverseName = getActiveDataverseName(stmtCreateIndex.getDataverseName());
+ if (isCompileOnly()) {
+ return;
+ }
lockUtil.createIndexBegin(lockManager, metadataProvider.getLocks(), dataverseName, datasetName,
fullTextConfigName);
try {
@@ -1409,6 +1427,9 @@
stmt.getSourceLocation());
DataverseName dataverseName = getActiveDataverseName(stmtCreateFilter.getDataverseName());
+ if (isCompileOnly()) {
+ return;
+ }
lockUtil.createFullTextFilterBegin(lockManager, metadataProvider.getLocks(), dataverseName, fullTextFilterName);
try {
doCreateFullTextFilter(metadataProvider, stmtCreateFilter, dataverseName);
@@ -1477,6 +1498,9 @@
DataverseName dataverseName = getActiveDataverseName(stmtCreateConfig.getDataverseName());
ImmutableList<String> filterNames = stmtCreateConfig.getFilterNames();
+ if (isCompileOnly()) {
+ return;
+ }
lockUtil.createFullTextConfigBegin(lockManager, metadataProvider.getLocks(), dataverseName, configName,
filterNames);
try {
@@ -1842,6 +1866,9 @@
String typeName = stmtCreateType.getIdent().getValue();
metadataProvider.validateDatabaseObjectName(stmtCreateType.getDataverseName(), typeName, sourceLoc);
DataverseName dataverseName = getActiveDataverseName(stmtCreateType.getDataverseName());
+ if (isCompileOnly()) {
+ return;
+ }
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
metadataProvider.setMetadataTxnContext(mdTxnCtx);
lockUtil.createTypeBegin(lockManager, metadataProvider.getLocks(), dataverseName, typeName);
@@ -1895,6 +1922,9 @@
throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
dataverseName + " " + dataverse() + " can't be dropped");
}
+ if (isCompileOnly()) {
+ return;
+ }
lockUtil.dropDataverseBegin(lockManager, metadataProvider.getLocks(), dataverseName);
try {
doDropDataverse(stmtDropDataverse, metadataProvider, hcc, requestParameters);
@@ -2098,6 +2128,9 @@
String datasetName = stmtDelete.getDatasetName().getValue();
metadataProvider.validateDatabaseObjectName(stmtDelete.getDataverseName(), datasetName, sourceLoc);
DataverseName dataverseName = getActiveDataverseName(stmtDelete.getDataverseName());
+ if (isCompileOnly()) {
+ return;
+ }
lockUtil.dropDatasetBegin(lockManager, metadataProvider.getLocks(), dataverseName, datasetName);
try {
doDropDataset(dataverseName, datasetName, metadataProvider, stmtDelete.getIfExists(), hcc,
@@ -2205,6 +2238,9 @@
stmtIndexDrop.getIndexName().getValue(), stmtIndexDrop.getSourceLocation());
DataverseName dataverseName = getActiveDataverseName(stmtIndexDrop.getDataverseName());
String datasetName = stmtIndexDrop.getDatasetName().getValue();
+ if (isCompileOnly()) {
+ return;
+ }
lockUtil.dropIndexBegin(lockManager, metadataProvider.getLocks(), dataverseName, datasetName);
try {
doDropIndex(metadataProvider, stmtIndexDrop, dataverseName, datasetName, hcc, requestParameters);
@@ -2245,14 +2281,8 @@
}
ensureNonPrimaryIndexDrop(index, sourceLoc);
validateDatasetState(metadataProvider, ds, sourceLoc);
- // #. prepare a job to drop the index in NC.
- jobsToExecute.add(IndexUtil.buildDropIndexJobSpec(index, metadataProvider, ds, sourceLoc));
-
- // #. mark PendingDropOp on the existing index
- MetadataManager.INSTANCE.dropIndex(mdTxnCtx, dataverseName, datasetName, indexName);
- MetadataManager.INSTANCE.addIndex(mdTxnCtx,
- new Index(dataverseName, datasetName, indexName, index.getIndexType(), index.getIndexDetails(),
- index.isEnforced(), index.isPrimaryIndex(), MetadataUtil.PENDING_DROP_OP));
+ prepareIndexDrop(metadataProvider, dataverseName, datasetName, sourceLoc, indexName, jobsToExecute,
+ mdTxnCtx, ds, index);
// #. commit the existing transaction before calling runJob.
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
@@ -2286,8 +2316,9 @@
"Dropping " + dataset() + " files index is not allowed.");
}
ensureNonPrimaryIndexDrop(index, sourceLoc);
- // #. prepare a job to drop the index in NC.
- jobsToExecute.add(IndexUtil.buildDropIndexJobSpec(index, metadataProvider, ds, sourceLoc));
+ prepareIndexDrop(metadataProvider, dataverseName, datasetName, sourceLoc, indexName, jobsToExecute,
+ mdTxnCtx, ds, index);
+
List<Index> datasetIndexes =
MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
if (datasetIndexes.size() == 2) {
@@ -2309,12 +2340,6 @@
}
}
- // #. mark PendingDropOp on the existing index
- MetadataManager.INSTANCE.dropIndex(mdTxnCtx, dataverseName, datasetName, indexName);
- MetadataManager.INSTANCE.addIndex(mdTxnCtx,
- new Index(dataverseName, datasetName, indexName, index.getIndexType(), index.getIndexDetails(),
- index.isEnforced(), index.isPrimaryIndex(), MetadataUtil.PENDING_DROP_OP));
-
// #. commit the existing transaction before calling runJob.
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
bActiveTxn = false;
@@ -2387,6 +2412,9 @@
DataverseName dataverseName = getActiveDataverseName(stmtFilterDrop.getDataverseName());
String fullTextFilterName = stmtFilterDrop.getFilterName();
+ if (isCompileOnly()) {
+ return;
+ }
lockUtil.dropFullTextFilterBegin(lockManager, metadataProvider.getLocks(), dataverseName, fullTextFilterName);
try {
doDropFullTextFilter(metadataProvider, stmtFilterDrop, dataverseName, fullTextFilterName);
@@ -2427,6 +2455,9 @@
DataverseName dataverseName = getActiveDataverseName(stmtConfigDrop.getDataverseName());
String configName = stmtConfigDrop.getConfigName();
+ if (isCompileOnly()) {
+ return;
+ }
lockUtil.dropFullTextConfigBegin(lockManager, metadataProvider.getLocks(), dataverseName, configName);
try {
doDropFullTextConfig(metadataProvider, stmtConfigDrop, hcc, requestParameters);
@@ -2476,6 +2507,9 @@
String typeName = stmtTypeDrop.getTypeName().getValue();
metadataProvider.validateDatabaseObjectName(stmtTypeDrop.getDataverseName(), typeName, sourceLoc);
DataverseName dataverseName = getActiveDataverseName(stmtTypeDrop.getDataverseName());
+ if (isCompileOnly()) {
+ return;
+ }
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
metadataProvider.setMetadataTxnContext(mdTxnCtx);
lockUtil.dropTypeBegin(lockManager, metadataProvider.getLocks(), dataverseName, typeName);
@@ -2516,6 +2550,9 @@
SourceLocation sourceLoc = stmtDelete.getSourceLocation();
String nodegroupName = stmtDelete.getNodeGroupName().getValue();
metadataProvider.validateDatabaseObjectName(null, nodegroupName, sourceLoc);
+ if (isCompileOnly()) {
+ return;
+ }
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
metadataProvider.setMetadataTxnContext(mdTxnCtx);
lockManager.acquireNodeGroupWriteLock(metadataProvider.getLocks(), nodegroupName);
@@ -2560,6 +2597,9 @@
viewItemTypeAnonymous = false;
}
+ if (isCompileOnly()) {
+ return;
+ }
lockUtil.createDatasetBegin(lockManager, metadataProvider.getLocks(), dataverseName, viewName,
viewItemTypeDataverseName, viewItemTypeName, viewItemTypeAnonymous, null, null, false, null, null, true,
DatasetType.VIEW, null);
@@ -2751,6 +2791,9 @@
String viewName = stmtDrop.getViewName().getValue();
metadataProvider.validateDatabaseObjectName(stmtDrop.getDataverseName(), viewName, sourceLoc);
DataverseName dataverseName = getActiveDataverseName(stmtDrop.getDataverseName());
+ if (isCompileOnly()) {
+ return;
+ }
lockUtil.dropDatasetBegin(lockManager, metadataProvider.getLocks(), dataverseName, viewName);
try {
doDropView(metadataProvider, stmtDrop, dataverseName, viewName);
@@ -2833,6 +2876,9 @@
}
}
+ if (isCompileOnly()) {
+ return;
+ }
lockUtil.createFunctionBegin(lockManager, metadataProvider.getLocks(), dataverseName, signature.getName(),
libraryDataverseName, libraryName);
try {
@@ -3082,6 +3128,9 @@
stmtDropFunction.getSourceLocation());
DataverseName dataverseName = getActiveDataverseName(signature.getDataverseName());
signature.setDataverseName(dataverseName);
+ if (isCompileOnly()) {
+ return;
+ }
lockUtil.dropFunctionBegin(lockManager, metadataProvider.getLocks(), dataverseName, signature.getName());
try {
doDropFunction(metadataProvider, stmtDropFunction, signature, requestParameters);
@@ -3140,6 +3189,9 @@
libraryDataverseName = dataverseName;
}
String libraryName = cas.getLibraryName();
+ if (isCompileOnly()) {
+ return;
+ }
lockUtil.createAdapterBegin(lockManager, metadataProvider.getLocks(), dataverseName, adapterName,
libraryDataverseName, libraryName);
try {
@@ -3210,6 +3262,9 @@
String adapterName = stmtDropAdapter.getAdapterName();
metadataProvider.validateDatabaseObjectName(stmtDropAdapter.getDataverseName(), adapterName, sourceLoc);
DataverseName dataverseName = getActiveDataverseName(stmtDropAdapter.getDataverseName());
+ if (isCompileOnly()) {
+ return;
+ }
lockUtil.dropAdapterBegin(lockManager, metadataProvider.getLocks(), dataverseName, adapterName);
try {
doDropAdapter(metadataProvider, stmtDropAdapter, dataverseName, adapterName);
@@ -3260,6 +3315,9 @@
DataverseName dataverseName = getActiveDataverseName(cls.getDataverseName());
String libraryName = cls.getLibraryName();
String libraryHash = cls.getHash();
+ if (isCompileOnly()) {
+ return;
+ }
lockUtil.createLibraryBegin(lockManager, metadataProvider.getLocks(), dataverseName, libraryName);
try {
doCreateLibrary(metadataProvider, dataverseName, libraryName, libraryHash, cls, hcc, requestParameters);
@@ -3388,6 +3446,9 @@
metadataProvider.validateDatabaseObjectName(stmtDropLibrary.getDataverseName(), libraryName,
stmtDropLibrary.getSourceLocation());
DataverseName dataverseName = getActiveDataverseName(stmtDropLibrary.getDataverseName());
+ if (isCompileOnly()) {
+ return;
+ }
lockUtil.dropLibraryBegin(lockManager, metadataProvider.getLocks(), dataverseName, libraryName);
try {
doDropLibrary(metadataProvider, stmtDropLibrary, dataverseName, libraryName, hcc, requestParameters);
@@ -3482,6 +3543,9 @@
DataverseName objectDataverseName =
css.getObjectDataverseName() != null ? css.getObjectDataverseName() : dataverseName;
String objectName = css.getObjectName();
+ if (isCompileOnly()) {
+ return;
+ }
lockUtil.createSynonymBegin(lockManager, metadataProvider.getLocks(), dataverseName, synonymName);
try {
doCreateSynonym(metadataProvider, css, dataverseName, synonymName, objectDataverseName, objectName);
@@ -3529,6 +3593,9 @@
metadataProvider.validateDatabaseObjectName(stmtSynDrop.getDataverseName(), synonymName,
stmtSynDrop.getSourceLocation());
DataverseName dataverseName = getActiveDataverseName(stmtSynDrop.getDataverseName());
+ if (isCompileOnly()) {
+ return;
+ }
lockUtil.dropSynonymBegin(lockManager, metadataProvider.getLocks(), dataverseName, synonymName);
try {
doDropSynonym(metadataProvider, stmtSynDrop, dataverseName, synonymName);
@@ -3582,7 +3649,7 @@
afterCompile();
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
bActiveTxn = false;
- if (spec != null && sessionConfig.isExecuteQuery()) {
+ if (spec != null && !isCompileOnly()) {
runJob(hcc, spec);
}
} catch (Exception e) {
@@ -3597,8 +3664,8 @@
public JobSpecification handleInsertUpsertStatement(MetadataProvider metadataProvider, Statement stmt,
IHyracksClientConnection hcc, IResultSet resultSet, ResultDelivery resultDelivery,
- ResultMetadata outMetadata, Stats stats, IRequestParameters requestParameters,
- Map<String, IAObject> stmtParams, IStatementRewriter stmtRewriter) throws Exception {
+ ResultMetadata outMetadata, Stats stats, IRequestParameters reqParams, Map<String, IAObject> stmtParams,
+ IStatementRewriter stmtRewriter) throws Exception {
InsertStatement stmtInsertUpsert = (InsertStatement) stmt;
String datasetName = stmtInsertUpsert.getDatasetName();
metadataProvider.validateDatabaseObjectName(stmtInsertUpsert.getDataverseName(), datasetName,
@@ -3627,7 +3694,7 @@
rewriteCompileInsertUpsert(hcc, metadataProvider, stmtInsertUpsert, stmtParams);
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
bActiveTxn = false;
- return !sessionConfig.isExecuteQuery() ? null : jobSpec;
+ return isCompileOnly() ? null : jobSpec;
} catch (Exception e) {
if (bActiveTxn) {
abort(e, e, mdTxnCtx);
@@ -3635,10 +3702,11 @@
throw e;
}
};
-
+ IRequestTracker requestTracker = appCtx.getRequestTracker();
+ ClientRequest clientRequest = (ClientRequest) requestTracker.get(reqParams.getRequestReference().getUuid());
if (stmtInsertUpsert.getReturnExpression() != null) {
deliverResult(hcc, resultSet, compiler, metadataProvider, locker, resultDelivery, outMetadata, stats,
- requestParameters, false);
+ reqParams, false, clientRequest);
} else {
locker.lock();
try {
@@ -3678,7 +3746,7 @@
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
bActiveTxn = false;
- if (jobSpec != null && sessionConfig.isExecuteQuery()) {
+ if (jobSpec != null && !isCompileOnly()) {
runJob(hcc, jobSpec);
}
return jobSpec;
@@ -3755,6 +3823,9 @@
String feedName = cfs.getFeedName().getValue();
metadataProvider.validateDatabaseObjectName(cfs.getDataverseName(), feedName, sourceLoc);
DataverseName dataverseName = getActiveDataverseName(cfs.getDataverseName());
+ if (isCompileOnly()) {
+ return;
+ }
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
metadataProvider.setMetadataTxnContext(mdTxnCtx);
lockUtil.createFeedBegin(lockManager, metadataProvider.getLocks(), dataverseName, feedName);
@@ -3794,6 +3865,9 @@
String policyName = cfps.getPolicyName();
metadataProvider.validateDatabaseObjectName(null, policyName, sourceLoc);
DataverseName dataverseName = getActiveDataverseName(null);
+ if (isCompileOnly()) {
+ return;
+ }
lockUtil.createFeedPolicyBegin(lockManager, metadataProvider.getLocks(), dataverseName, policyName);
try {
mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
@@ -3855,6 +3929,9 @@
String feedName = stmtFeedDrop.getFeedName().getValue();
metadataProvider.validateDatabaseObjectName(stmtFeedDrop.getDataverseName(), feedName, sourceLoc);
DataverseName dataverseName = getActiveDataverseName(stmtFeedDrop.getDataverseName());
+ if (isCompileOnly()) {
+ return;
+ }
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
metadataProvider.setMetadataTxnContext(mdTxnCtx);
lockUtil.dropFeedBegin(lockManager, metadataProvider.getLocks(), dataverseName, feedName);
@@ -3908,6 +3985,9 @@
String policyName = stmtFeedPolicyDrop.getPolicyName().getValue();
metadataProvider.validateDatabaseObjectName(stmtFeedPolicyDrop.getDataverseName(), policyName, sourceLoc);
DataverseName dataverseName = getActiveDataverseName(stmtFeedPolicyDrop.getDataverseName());
+ if (isCompileOnly()) {
+ return;
+ }
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
metadataProvider.setMetadataTxnContext(mdTxnCtx);
lockUtil.dropFeedPolicyBegin(lockManager, metadataProvider.getLocks(), dataverseName, policyName);
@@ -3937,6 +4017,9 @@
SourceLocation sourceLoc = sfs.getSourceLocation();
DataverseName dataverseName = getActiveDataverseName(sfs.getDataverseName());
String feedName = sfs.getFeedName().getValue();
+ if (isCompileOnly()) {
+ return;
+ }
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
boolean committed = false;
lockUtil.startFeedBegin(lockManager, metadataProvider.getLocks(), dataverseName, feedName);
@@ -3999,6 +4082,9 @@
throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
"Feed " + feedName + " is not started.");
}
+ if (isCompileOnly()) {
+ return;
+ }
lockUtil.stopFeedBegin(lockManager, metadataProvider.getLocks(), dataverseName, feedName);
try {
listener.stop(metadataProvider);
@@ -4016,6 +4102,9 @@
String datasetName = cfs.getDatasetName().getValue();
String policyName = cfs.getPolicy();
String whereClauseBody = cfs.getWhereClauseBody();
+ if (isCompileOnly()) {
+ return;
+ }
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
metadataProvider.setMetadataTxnContext(mdTxnCtx);
// TODO: Check whether we are connecting a change feed to a non-meta dataset
@@ -4071,6 +4160,9 @@
DataverseName dataverseName = getActiveDataverseName(cfs.getDataverseName());
String datasetName = cfs.getDatasetName().getValue();
String feedName = cfs.getFeedName().getValue();
+ if (isCompileOnly()) {
+ return;
+ }
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
metadataProvider.setMetadataTxnContext(mdTxnCtx);
lockUtil.disconnectFeedBegin(lockManager, metadataProvider.getLocks(), dataverseName, datasetName, feedName);
@@ -4110,12 +4202,385 @@
}
}
+ protected void handleAnalyzeStatement(MetadataProvider metadataProvider, Statement stmt,
+ IHyracksClientConnection hcc, IRequestParameters requestParameters) throws Exception {
+ AnalyzeStatement analyzeStatement = (AnalyzeStatement) stmt;
+ metadataProvider.validateDatabaseObjectName(analyzeStatement.getDataverseName(),
+ analyzeStatement.getDatasetName(), analyzeStatement.getSourceLocation());
+ DataverseName dataverseName = getActiveDataverseName(analyzeStatement.getDataverseName());
+ String datasetName = analyzeStatement.getDatasetName();
+ if (isCompileOnly()) {
+ return;
+ }
+ lockUtil.analyzeDatasetBegin(lockManager, metadataProvider.getLocks(), dataverseName, datasetName);
+ try {
+ doAnalyzeDataset(metadataProvider, analyzeStatement, dataverseName, datasetName, hcc, requestParameters);
+ } finally {
+ metadataProvider.getLocks().unlock();
+ }
+ }
+
+ protected void doAnalyzeDataset(MetadataProvider metadataProvider, AnalyzeStatement stmtAnalyze,
+ DataverseName dataverseName, String datasetName, IHyracksClientConnection hcc,
+ IRequestParameters requestParameters) throws Exception {
+ SourceLocation sourceLoc = stmtAnalyze.getSourceLocation();
+ ProgressState progressNewIndexCreate = ProgressState.NO_PROGRESS;
+ ProgressState progressExistingIndexDrop = ProgressState.NO_PROGRESS;
+ Dataset ds = null;
+ Index existingIndex = null, newIndexPendingAdd = null;
+ JobSpecification existingIndexDropSpec = null;
+ MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+ boolean bActiveTxn = true;
+ metadataProvider.setMetadataTxnContext(mdTxnCtx);
+ try {
+ // Check if the dataverse exists
+ Dataverse dv = MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dataverseName);
+ if (dv == null) {
+ throw new CompilationException(ErrorCode.UNKNOWN_DATAVERSE, sourceLoc, dataverseName);
+ }
+ // Check if the dataset exists
+ ds = metadataProvider.findDataset(dataverseName, datasetName);
+ if (ds == null) {
+ throw new CompilationException(ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE, sourceLoc, datasetName,
+ dataverseName);
+ }
+ if (ds.getDatasetType() == DatasetType.INTERNAL) {
+ validateDatasetState(metadataProvider, ds, sourceLoc);
+ } else {
+ throw new CompilationException(ErrorCode.OPERATION_NOT_SUPPORTED, sourceLoc);
+ }
+
+ IndexType sampleIndexType = IndexType.SAMPLE;
+ Pair<String, String> sampleIndexNames = IndexUtil.getSampleIndexNames(datasetName);
+ String newIndexName;
+ existingIndex = MetadataManager.INSTANCE.getIndex(metadataProvider.getMetadataTxnContext(), dataverseName,
+ datasetName, sampleIndexNames.first);
+ if (existingIndex != null) {
+ newIndexName = sampleIndexNames.second;
+ } else {
+ existingIndex = MetadataManager.INSTANCE.getIndex(metadataProvider.getMetadataTxnContext(),
+ dataverseName, datasetName, sampleIndexNames.second);
+ newIndexName = sampleIndexNames.first;
+ }
+
+ InternalDatasetDetails dsDetails = (InternalDatasetDetails) ds.getDatasetDetails();
+ int sampleCardinalityTarget = stmtAnalyze.getSampleSize();
+ long sampleSeed = stmtAnalyze.getOrCreateSampleSeed();
+
+ Index.SampleIndexDetails newIndexDetailsPendingAdd = new Index.SampleIndexDetails(dsDetails.getPrimaryKey(),
+ dsDetails.getKeySourceIndicator(), dsDetails.getPrimaryKeyType(), sampleCardinalityTarget, 0, 0,
+ sampleSeed, Collections.emptyMap());
+ newIndexPendingAdd = new Index(dataverseName, datasetName, newIndexName, sampleIndexType,
+ newIndexDetailsPendingAdd, false, false, MetadataUtil.PENDING_ADD_OP);
+
+ // #. add a new index with PendingAddOp
+ MetadataManager.INSTANCE.addIndex(metadataProvider.getMetadataTxnContext(), newIndexPendingAdd);
+ // #. prepare to create the index artifact in NC.
+ JobSpecification spec =
+ IndexUtil.buildSecondaryIndexCreationJobSpec(ds, newIndexPendingAdd, metadataProvider, sourceLoc);
+ if (spec == null) {
+ throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+ "Failed to create job spec for creating index '" + ds.getDatasetName() + "."
+ + newIndexPendingAdd.getIndexName() + "'");
+ }
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ bActiveTxn = false;
+ progressNewIndexCreate = ProgressState.ADDED_PENDINGOP_RECORD_TO_METADATA;
+
+ // #. create the index artifact in NC.
+ runJob(hcc, spec);
+
+ // #. flush dataset
+ FlushDatasetUtil.flushDataset(hcc, metadataProvider, dataverseName, datasetName);
+
+ mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+ bActiveTxn = true;
+ metadataProvider.setMetadataTxnContext(mdTxnCtx);
+
+ // #. load data into the index in NC.
+ spec = IndexUtil.buildSecondaryIndexLoadingJobSpec(ds, newIndexPendingAdd, metadataProvider, sourceLoc);
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ bActiveTxn = false;
+
+ List<IOperatorStats> opStats = runJob(hcc, spec, jobFlags,
+ Collections.singletonList(SampleOperationsHelper.DATASET_STATS_OPERATOR_NAME));
+ if (opStats == null || opStats.size() == 0) {
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, "", sourceLoc);
+ }
+ DatasetStreamStats stats = new DatasetStreamStats(opStats.get(0));
+
+ Index.SampleIndexDetails newIndexDetailsFinal = new Index.SampleIndexDetails(dsDetails.getPrimaryKey(),
+ dsDetails.getKeySourceIndicator(), dsDetails.getPrimaryKeyType(), sampleCardinalityTarget,
+ stats.getCardinality(), stats.getAvgTupleSize(), sampleSeed, stats.getIndexesStats());
+ Index newIndexFinal = new Index(dataverseName, datasetName, newIndexName, sampleIndexType,
+ newIndexDetailsFinal, false, false, MetadataUtil.PENDING_NO_OP);
+
+ // #. begin new metadataTxn
+ mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+ bActiveTxn = true;
+ metadataProvider.setMetadataTxnContext(mdTxnCtx);
+ // #. add same new index with PendingNoOp after deleting its entry with PendingAddOp
+ MetadataManager.INSTANCE.dropIndex(metadataProvider.getMetadataTxnContext(),
+ newIndexPendingAdd.getDataverseName(), newIndexPendingAdd.getDatasetName(),
+ newIndexPendingAdd.getIndexName());
+ MetadataManager.INSTANCE.addIndex(metadataProvider.getMetadataTxnContext(), newIndexFinal);
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ bActiveTxn = false;
+ progressNewIndexCreate = ProgressState.NO_PROGRESS;
+
+ if (existingIndex != null) {
+ // #. set existing index to PendingDropOp because we'll be dropping it next
+ mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+ bActiveTxn = true;
+ metadataProvider.setMetadataTxnContext(mdTxnCtx);
+ MetadataManager.INSTANCE.dropIndex(metadataProvider.getMetadataTxnContext(),
+ existingIndex.getDataverseName(), existingIndex.getDatasetName(), existingIndex.getIndexName());
+ existingIndex.setPendingOp(MetadataUtil.PENDING_DROP_OP);
+ MetadataManager.INSTANCE.addIndex(metadataProvider.getMetadataTxnContext(), existingIndex);
+ existingIndexDropSpec = IndexUtil.buildDropIndexJobSpec(existingIndex, metadataProvider, ds, sourceLoc);
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ progressExistingIndexDrop = ProgressState.ADDED_PENDINGOP_RECORD_TO_METADATA;
+ bActiveTxn = false;
+
+ // #. drop existing index on NCs
+ runJob(hcc, existingIndexDropSpec);
+
+ // #. drop existing index metadata
+ mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+ bActiveTxn = true;
+ metadataProvider.setMetadataTxnContext(mdTxnCtx);
+ MetadataManager.INSTANCE.dropIndex(metadataProvider.getMetadataTxnContext(),
+ existingIndex.getDataverseName(), existingIndex.getDatasetName(), existingIndex.getIndexName());
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ bActiveTxn = false;
+ progressExistingIndexDrop = ProgressState.NO_PROGRESS;
+ }
+
+ } catch (Exception e) {
+ LOGGER.error("failed to analyze dataset; executing compensating operations", e);
+ if (bActiveTxn) {
+ abort(e, e, mdTxnCtx);
+ }
+
+ if (progressExistingIndexDrop == ProgressState.ADDED_PENDINGOP_RECORD_TO_METADATA) {
+ // #. execute compensation operations remove the index in NCs
+ try {
+ runJob(hcc, existingIndexDropSpec);
+ } catch (Exception e2) {
+ // do no throw exception since still the metadata needs to be compensated.
+ e.addSuppressed(e2);
+ }
+ // #. remove the record from the metadata.
+ mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+ metadataProvider.setMetadataTxnContext(mdTxnCtx);
+ try {
+ MetadataManager.INSTANCE.dropIndex(metadataProvider.getMetadataTxnContext(),
+ existingIndex.getDataverseName(), existingIndex.getDatasetName(),
+ existingIndex.getIndexName());
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ } catch (Exception e2) {
+ e.addSuppressed(e2);
+ abort(e, e2, mdTxnCtx);
+ throw new IllegalStateException("System is inconsistent state: pending index("
+ + existingIndex.getDataverseName() + "." + existingIndex.getDatasetName() + "."
+ + existingIndex.getIndexName() + ") couldn't be removed from the metadata", e);
+ }
+ } else if (progressNewIndexCreate == ProgressState.ADDED_PENDINGOP_RECORD_TO_METADATA) {
+ // #. execute compensation operations remove the index in NCs
+ mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+ bActiveTxn = true;
+ metadataProvider.setMetadataTxnContext(mdTxnCtx);
+ try {
+ JobSpecification jobSpec =
+ IndexUtil.buildDropIndexJobSpec(newIndexPendingAdd, metadataProvider, ds, sourceLoc);
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ bActiveTxn = false;
+ runJob(hcc, jobSpec);
+ } catch (Exception e2) {
+ e.addSuppressed(e2);
+ if (bActiveTxn) {
+ abort(e, e2, mdTxnCtx);
+ }
+ }
+ // #. remove the record from the metadata.
+ mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+ metadataProvider.setMetadataTxnContext(mdTxnCtx);
+ try {
+ MetadataManager.INSTANCE.dropIndex(metadataProvider.getMetadataTxnContext(),
+ newIndexPendingAdd.getDataverseName(), newIndexPendingAdd.getDatasetName(),
+ newIndexPendingAdd.getIndexName());
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ } catch (Exception e2) {
+ e.addSuppressed(e2);
+ abort(e, e2, mdTxnCtx);
+ throw new IllegalStateException("System is in inconsistent state: pending index("
+ + newIndexPendingAdd.getDataverseName() + "." + newIndexPendingAdd.getDatasetName() + "."
+ + newIndexPendingAdd.getIndexName() + ") couldn't be removed from the metadata", e);
+ }
+ }
+
+ throw e;
+ }
+ }
+
+ protected void handleAnalyzeDropStatement(MetadataProvider metadataProvider, Statement stmt,
+ IHyracksClientConnection hcc, IRequestParameters requestParams) throws Exception {
+ AnalyzeDropStatement analyzeDropStmt = (AnalyzeDropStatement) stmt;
+ metadataProvider.validateDatabaseObjectName(analyzeDropStmt.getDataverseName(),
+ analyzeDropStmt.getDatasetName(), analyzeDropStmt.getSourceLocation());
+ DataverseName dataverseName = getActiveDataverseName(analyzeDropStmt.getDataverseName());
+ String datasetName = analyzeDropStmt.getDatasetName();
+ if (isCompileOnly()) {
+ return;
+ }
+ lockUtil.analyzeDatasetDropBegin(lockManager, metadataProvider.getLocks(), dataverseName, datasetName);
+ try {
+ doAnalyzeDatasetDrop(metadataProvider, analyzeDropStmt, dataverseName, datasetName, hcc, requestParams);
+ } finally {
+ metadataProvider.getLocks().unlock();
+ }
+ }
+
+ protected boolean doAnalyzeDatasetDrop(MetadataProvider metadataProvider, AnalyzeDropStatement stmtIndexDrop,
+ DataverseName dataverseName, String datasetName, IHyracksClientConnection hcc,
+ IRequestParameters requestParams) throws Exception {
+ SourceLocation sourceLoc = stmtIndexDrop.getSourceLocation();
+ Pair<String, String> sampleIndexNames = IndexUtil.getSampleIndexNames(datasetName);
+ String indexName1 = sampleIndexNames.first;
+ String indexName2 = sampleIndexNames.second;
+ ProgressState progress = ProgressState.NO_PROGRESS;
+ List<JobSpecification> jobsToExecute = new ArrayList<>();
+
+ MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+ boolean bActiveTxn = true;
+ metadataProvider.setMetadataTxnContext(mdTxnCtx);
+ boolean index1Exists = false, index2Exists = false;
+ try {
+ Dataset ds = metadataProvider.findDataset(dataverseName, datasetName);
+ if (ds == null) {
+ throw new CompilationException(ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE, sourceLoc, datasetName,
+ dataverseName);
+ }
+ if (ds.getDatasetType() != DatasetType.INTERNAL) {
+ throw new CompilationException(ErrorCode.OPERATION_NOT_SUPPORTED, sourceLoc);
+ }
+ Index index1 = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataverseName, datasetName, indexName1);
+ Index index2 = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataverseName, datasetName, indexName2);
+ index1Exists = index1 != null;
+ index2Exists = index2 != null;
+ if (!index1Exists && !index2Exists) {
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ return false;
+ }
+ ensureNonPrimaryIndexesDrop(index1, index2, sourceLoc);
+ validateDatasetState(metadataProvider, ds, sourceLoc);
+ prepareIndexDrop(metadataProvider, dataverseName, datasetName, sourceLoc, indexName1, jobsToExecute,
+ mdTxnCtx, ds, index1);
+ prepareIndexDrop(metadataProvider, dataverseName, datasetName, sourceLoc, indexName2, jobsToExecute,
+ mdTxnCtx, ds, index2);
+
+ // #. commit the existing transaction before calling runJob.
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ bActiveTxn = false;
+ progress = ProgressState.ADDED_PENDINGOP_RECORD_TO_METADATA;
+
+ for (JobSpecification jobSpec : jobsToExecute) {
+ runJob(hcc, jobSpec);
+ }
+
+ // #. begin a new transaction
+ mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+ bActiveTxn = true;
+ metadataProvider.setMetadataTxnContext(mdTxnCtx);
+
+ // #. finally, delete the existing indexes
+ if (index1Exists) {
+ MetadataManager.INSTANCE.dropIndex(mdTxnCtx, dataverseName, datasetName, indexName1);
+ }
+ if (index2Exists) {
+ MetadataManager.INSTANCE.dropIndex(mdTxnCtx, dataverseName, datasetName, indexName2);
+ }
+
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ return true;
+ } catch (Exception e) {
+ if (bActiveTxn) {
+ abort(e, e, mdTxnCtx);
+ }
+
+ if (progress == ProgressState.ADDED_PENDINGOP_RECORD_TO_METADATA) {
+ // #. execute compensation operations remove the all indexes in NC
+ try {
+ for (JobSpecification jobSpec : jobsToExecute) {
+ runJob(hcc, jobSpec);
+ }
+ } catch (Exception e2) {
+ // do no throw exception since still the metadata needs to be compensated.
+ e.addSuppressed(e2);
+ }
+
+ // remove the record from the metadata.
+ mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+ metadataProvider.setMetadataTxnContext(mdTxnCtx);
+ try {
+ if (index1Exists) {
+ MetadataManager.INSTANCE.dropIndex(metadataProvider.getMetadataTxnContext(), dataverseName,
+ datasetName, indexName1);
+ }
+ if (index2Exists) {
+ MetadataManager.INSTANCE.dropIndex(metadataProvider.getMetadataTxnContext(), dataverseName,
+ datasetName, indexName2);
+ }
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ } catch (Exception e2) {
+ e.addSuppressed(e2);
+ abort(e, e2, mdTxnCtx);
+ String msg = String.format(
+ "System is in inconsistent state: pending index %1$s.%2$s.%3$s and/or %1$s.%2$s.%4$s "
+ + "couldn't be removed from the metadata",
+ dataverseName, datasetName, indexName1, indexName2);
+ throw new IllegalStateException(msg, e);
+ }
+ }
+
+ throw e;
+ }
+ }
+
+ private void ensureNonPrimaryIndexesDrop(Index index1, Index index2, SourceLocation sourceLoc)
+ throws AlgebricksException {
+ if (index1 != null) {
+ ensureNonPrimaryIndexDrop(index1, sourceLoc);
+ }
+ if (index2 != null) {
+ ensureNonPrimaryIndexDrop(index2, sourceLoc);
+ }
+ }
+
+ private void prepareIndexDrop(MetadataProvider metadataProvider, DataverseName dataverseName, String datasetName,
+ SourceLocation sourceLoc, String indexName, List<JobSpecification> jobsToExecute,
+ MetadataTransactionContext mdTxnCtx, Dataset ds, Index index) throws AlgebricksException {
+ if (index != null) {
+ // #. prepare a job to drop the index in NC.
+ jobsToExecute.add(IndexUtil.buildDropIndexJobSpec(index, metadataProvider, ds, sourceLoc));
+
+ // #. mark PendingDropOp on the existing index
+ MetadataManager.INSTANCE.dropIndex(mdTxnCtx, dataverseName, datasetName, indexName);
+ MetadataManager.INSTANCE.addIndex(mdTxnCtx,
+ new Index(dataverseName, datasetName, indexName, index.getIndexType(), index.getIndexDetails(),
+ index.isEnforced(), index.isPrimaryIndex(), MetadataUtil.PENDING_DROP_OP));
+ }
+ }
+
protected void handleCompactStatement(MetadataProvider metadataProvider, Statement stmt,
IHyracksClientConnection hcc) throws Exception {
CompactStatement compactStatement = (CompactStatement) stmt;
SourceLocation sourceLoc = compactStatement.getSourceLocation();
DataverseName dataverseName = getActiveDataverseName(compactStatement.getDataverseName());
String datasetName = compactStatement.getDatasetName().getValue();
+ if (isCompileOnly()) {
+ return;
+ }
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
boolean bActiveTxn = true;
metadataProvider.setMetadataTxnContext(mdTxnCtx);
@@ -4228,7 +4693,7 @@
afterCompile();
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
bActiveTxn = false;
- return query.isExplain() || !sessionConfig.isExecuteQuery() ? null : jobSpec;
+ return query.isExplain() || isCompileOnly() ? null : jobSpec;
} catch (Exception e) {
LOGGER.log(Level.INFO, e.getMessage(), e);
if (bActiveTxn) {
@@ -4238,13 +4703,13 @@
}
};
deliverResult(hcc, resultSet, compiler, metadataProvider, locker, resultDelivery, outMetadata, stats,
- requestParameters, true);
+ requestParameters, true, clientRequest);
}
private void deliverResult(IHyracksClientConnection hcc, IResultSet resultSet, IStatementCompiler compiler,
MetadataProvider metadataProvider, IMetadataLocker locker, ResultDelivery resultDelivery,
- ResultMetadata outMetadata, Stats stats, IRequestParameters requestParameters, boolean cancellable)
- throws Exception {
+ ResultMetadata outMetadata, Stats stats, IRequestParameters requestParameters, boolean cancellable,
+ ClientRequest clientRequest) throws Exception {
final ResultSetId resultSetId = metadataProvider.getResultSetId();
switch (resultDelivery) {
case ASYNC:
@@ -4260,7 +4725,7 @@
case IMMEDIATE:
createAndRunJob(hcc, jobFlags, null, compiler, locker, resultDelivery, id -> {
final ResultReader resultReader = new ResultReader(resultSet, id, resultSetId);
- updateJobStats(id, stats, metadataProvider.getResultSetId());
+ updateJobStats(id, stats, metadataProvider.getResultSetId(), clientRequest);
responsePrinter.addResultPrinter(new ResultsPrinter(appCtx, resultReader,
metadataProvider.findOutputRecordType(), stats, sessionOutput));
responsePrinter.printResults();
@@ -4268,7 +4733,7 @@
break;
case DEFERRED:
createAndRunJob(hcc, jobFlags, null, compiler, locker, resultDelivery, id -> {
- updateJobStats(id, stats, metadataProvider.getResultSetId());
+ updateJobStats(id, stats, metadataProvider.getResultSetId(), clientRequest);
responsePrinter.addResultPrinter(
new ResultHandlePrinter(sessionOutput, new ResultHandle(id, resultSetId)));
responsePrinter.printResults();
@@ -4283,16 +4748,18 @@
}
}
- private void updateJobStats(JobId jobId, Stats stats, ResultSetId rsId) throws HyracksDataException {
+ private void updateJobStats(JobId jobId, Stats stats, ResultSetId rsId, ClientRequest clientRequest)
+ throws HyracksDataException {
final ClusterControllerService controllerService =
(ClusterControllerService) appCtx.getServiceContext().getControllerService();
- org.apache.asterix.api.common.ResultMetadata resultMetadata =
- (org.apache.asterix.api.common.ResultMetadata) controllerService.getResultDirectoryService()
+ org.apache.asterix.translator.ResultMetadata resultMetadata =
+ (org.apache.asterix.translator.ResultMetadata) controllerService.getResultDirectoryService()
.getResultMetadata(jobId, rsId);
stats.setProcessedObjects(resultMetadata.getProcessedObjects());
if (jobFlags.contains(JobFlag.PROFILE_RUNTIME)) {
stats.setJobProfile(resultMetadata.getJobProfile());
}
+ clientRequest.setPlan(apiFramework.getExecutionPlans().getOptimizedLogicalPlan());
stats.updateTotalWarningsCount(resultMetadata.getTotalWarningsCount());
WarningUtil.mergeWarnings(resultMetadata.getWarnings(), warningCollector);
}
@@ -4345,6 +4812,12 @@
JobUtils.runJob(hcc, jobSpec, jobFlags, true);
}
+ private static List<IOperatorStats> runJob(IHyracksClientConnection hcc, JobSpecification jobSpec,
+ EnumSet<JobFlag> jobFlags, List<String> statOperatorNames) throws Exception {
+ Pair<JobId, List<IOperatorStats>> p = JobUtils.runJob(hcc, jobSpec, jobFlags, true, statOperatorNames);
+ return p.second;
+ }
+
private static void createAndRunJob(IHyracksClientConnection hcc, EnumSet<JobFlag> jobFlags, Mutable<JobId> jId,
IStatementCompiler compiler, IMetadataLocker locker, ResultDelivery resultDelivery, IResultPrinter printer,
IRequestParameters requestParameters, boolean cancellable, ICcApplicationContext appCtx,
@@ -4366,10 +4839,11 @@
appCtx.getReceptionist().ensureSchedulable(schedulableRequest);
// ensure request not cancelled before running job
ensureNotCancelled(clientRequest);
+ jobSpec.setRequestId(clientRequest.getId());
final JobId jobId = JobUtils.runJob(hcc, jobSpec, jobFlags, false);
- if (LOGGER.isDebugEnabled()) {
- LOGGER.debug("createAndRunJob jobId:{}, uuid:{}", jobId,
- requestParameters.getRequestReference().getUuid());
+ if (LOGGER.isInfoEnabled()) {
+ LOGGER.info("Created job {} for query uuid:{}, clientContextID:{}", jobId,
+ requestParameters.getRequestReference().getUuid(), requestParameters.getClientContextId());
}
clientRequest.setJobId(jobId);
if (jId != null) {
@@ -4404,6 +4878,9 @@
String ngName = stmtCreateNodegroup.getNodegroupName().getValue();
metadataProvider.validateDatabaseObjectName(null, ngName, sourceLoc);
+ if (isCompileOnly()) {
+ return;
+ }
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
metadataProvider.setMetadataTxnContext(mdTxnCtx);
lockManager.acquireNodeGroupWriteLock(metadataProvider.getLocks(), ngName);
@@ -4438,9 +4915,6 @@
DataverseName dataverseName = getActiveDataverseName(stmtRefresh.getDataverseName());
String datasetName = stmtRefresh.getDatasetName().getValue();
TransactionState transactionState = TransactionState.COMMIT;
- MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
- boolean bActiveTxn = true;
- metadataProvider.setMetadataTxnContext(mdTxnCtx);
JobSpecification spec = null;
Dataset ds = null;
List<ExternalFile> metadataFiles = null;
@@ -4451,6 +4925,12 @@
Dataset transactionDataset = null;
boolean lockAquired = false;
boolean success = false;
+ if (isCompileOnly()) {
+ return;
+ }
+ MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+ metadataProvider.setMetadataTxnContext(mdTxnCtx);
+ boolean bActiveTxn = true;
lockUtil.refreshDatasetBegin(lockManager, metadataProvider.getLocks(), dataverseName, datasetName);
try {
ds = metadataProvider.findDataset(dataverseName, datasetName);
@@ -4732,7 +5212,7 @@
protected void trackRequest(IRequestParameters requestParameters) throws HyracksDataException {
final IClientRequest clientRequest = appCtx.getReceptionist().requestReceived(requestParameters);
- appCtx.getRequestTracker().track(clientRequest);
+ this.appCtx.getRequestTracker().track(clientRequest);
}
protected void validateStatements(IRequestParameters requestParameters)
@@ -4785,7 +5265,7 @@
private Map<VarIdentifier, IAObject> createExternalVariables(IReturningStatement stmt,
Map<String, IAObject> stmtParams) throws CompilationException {
- if (sessionConfig.isExecuteQuery()) {
+ if (!isCompileOnly()) {
if (stmtParams == null || stmtParams.isEmpty()) {
return Collections.emptyMap();
}
@@ -4810,6 +5290,10 @@
}
}
+ protected boolean isCompileOnly() {
+ return !sessionConfig.isExecuteQuery();
+ }
+
protected void validateDatasetState(MetadataProvider metadataProvider, Dataset dataset, SourceLocation sourceLoc)
throws Exception {
validateIfResourceIsActiveInFeed(metadataProvider.getApplicationContext(), dataset, sourceLoc);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
index 8f3deb1..31ec44c 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
@@ -27,6 +27,7 @@
import java.io.File;
import java.io.IOException;
import java.nio.charset.Charset;
+import java.rmi.RemoteException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
@@ -36,9 +37,9 @@
import java.util.concurrent.ConcurrentMap;
import org.apache.asterix.api.http.IQueryWebServerRegistrant;
+import org.apache.asterix.api.http.server.ActiveRequestsServlet;
import org.apache.asterix.api.http.server.ActiveStatsApiServlet;
import org.apache.asterix.api.http.server.ApiServlet;
-import org.apache.asterix.api.http.server.CcQueryCancellationServlet;
import org.apache.asterix.api.http.server.ClusterApiServlet;
import org.apache.asterix.api.http.server.ClusterControllerDetailsApiServlet;
import org.apache.asterix.api.http.server.ConnectorApiServlet;
@@ -53,6 +54,7 @@
import org.apache.asterix.api.http.server.VersionApiServlet;
import org.apache.asterix.app.active.ActiveNotificationHandler;
import org.apache.asterix.app.cc.CCExtensionManager;
+import org.apache.asterix.app.cc.CcApplicationContext;
import org.apache.asterix.app.config.ConfigValidator;
import org.apache.asterix.app.io.PersistedResourceRegistry;
import org.apache.asterix.app.replication.NcLifecycleCoordinator;
@@ -85,7 +87,6 @@
import org.apache.asterix.metadata.lock.MetadataLockManager;
import org.apache.asterix.metadata.utils.MetadataLockUtil;
import org.apache.asterix.runtime.job.resource.JobCapacityController;
-import org.apache.asterix.runtime.utils.CcApplicationContext;
import org.apache.asterix.translator.IStatementExecutorFactory;
import org.apache.asterix.translator.Receptionist;
import org.apache.asterix.util.MetadataBuiltinFunctions;
@@ -120,13 +121,13 @@
public class CCApplication extends BaseCCApplication {
private static final Logger LOGGER = LogManager.getLogger();
- private static IAsterixStateProxy proxy;
+ private IAsterixStateProxy proxy;
protected CCExtensionManager ccExtensionManager;
protected IStorageComponentProvider componentProvider;
protected WebManager webManager;
protected ICcApplicationContext appCtx;
private IJobCapacityController jobCapacityController;
- private IHyracksClientConnection hcc;
+ private HyracksConnection hcc;
@Override
public void init(IServiceContext serviceCtx) throws Exception {
@@ -166,11 +167,11 @@
}
MetadataProperties metadataProperties = appCtx.getMetadataProperties();
- setAsterixStateProxy(AsterixStateProxy.registerRemoteObject(controllerService.getNetworkSecurityManager(),
- metadataProperties.getMetadataCallbackPort()));
+ proxy = getAsterixStateProxy(controllerService, metadataProperties);
ccServiceCtx.setDistributedState(proxy);
MetadataManager.initialize(proxy, metadataProperties, appCtx);
ccServiceCtx.addJobLifecycleListener(appCtx.getActiveNotificationHandler());
+ ccServiceCtx.addJobLifecycleListener(appCtx.getRequestTracker());
// create event loop groups
webManager = new WebManager();
@@ -209,7 +210,7 @@
IReceptionistFactory receptionistFactory, IConfigValidatorFactory configValidatorFactory,
CCExtensionManager ccExtensionManager, IAdapterFactoryService adapterFactoryService)
throws AlgebricksException, IOException {
- return new CcApplicationContext(ccServiceCtx, getHcc(), () -> MetadataManager.INSTANCE, globalRecoveryManager,
+ return new CcApplicationContext(ccServiceCtx, hcc, () -> MetadataManager.INSTANCE, globalRecoveryManager,
lifecycleCoordinator, new ActiveNotificationHandler(), componentProvider, new MetadataLockManager(),
createMetadataLockUtil(), receptionistFactory, configValidatorFactory, ccExtensionManager,
adapterFactoryService);
@@ -315,7 +316,7 @@
ConcurrentMap<String, Object> ctx = server.ctx();
switch (key) {
case Servlets.RUNNING_REQUESTS:
- return new CcQueryCancellationServlet(ctx, appCtx, paths);
+ return new ActiveRequestsServlet(ctx, appCtx, paths);
case Servlets.QUERY_STATUS:
return new QueryStatusApiServlet(ctx, appCtx, paths);
case Servlets.QUERY_RESULT:
@@ -362,8 +363,10 @@
ApplicationConfigurator.registerConfigOptions(configManager);
}
- public static synchronized void setAsterixStateProxy(IAsterixStateProxy proxy) {
- CCApplication.proxy = proxy;
+ protected IAsterixStateProxy getAsterixStateProxy(ClusterControllerService controllerService,
+ MetadataProperties metadataProperties) throws RemoteException {
+ return AsterixStateProxy.registerRemoteObject(controllerService.getNetworkSecurityManager(),
+ metadataProperties.getMetadataCallbackPort());
}
@Override
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java
index f60349f..a223e0d 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java
@@ -88,6 +88,7 @@
import org.apache.commons.csv.CSVParser;
import org.apache.commons.csv.CSVRecord;
import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.api.application.INCServiceContext;
import org.apache.hyracks.api.application.IServiceContext;
import org.apache.hyracks.api.client.NodeStatus;
import org.apache.hyracks.api.config.IConfigManager;
@@ -154,7 +155,7 @@
MetadataBuiltinFunctions.init();
ncExtensionManager = new NCExtensionManager(new ArrayList<>(getExtensions()));
- runtimeContext = new NCAppRuntimeContext(ncServiceCtx, ncExtensionManager, getPropertiesFactory());
+ runtimeContext = createNCApplicationContext(ncServiceCtx, ncExtensionManager, getPropertiesFactory());
MetadataProperties metadataProperties = runtimeContext.getMetadataProperties();
if (!metadataProperties.getNodeNames().contains(this.ncServiceCtx.getNodeId())) {
if (LOGGER.isInfoEnabled()) {
@@ -186,6 +187,12 @@
performLocalCleanUp();
}
+ protected INcApplicationContext createNCApplicationContext(INCServiceContext ncServiceCtx,
+ NCExtensionManager ncExtensionManager, IPropertiesFactory propertiesFactory)
+ throws IOException, AsterixException {
+ return new NCAppRuntimeContext(ncServiceCtx, ncExtensionManager, propertiesFactory);
+ }
+
protected IRecoveryManagerFactory getRecoveryManagerFactory() {
return RecoveryManager::new;
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/rebalance/IDatasetRebalanceCallback.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/rebalance/IDatasetRebalanceCallback.java
index e8683c9..c366977 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/rebalance/IDatasetRebalanceCallback.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/rebalance/IDatasetRebalanceCallback.java
@@ -30,7 +30,7 @@
public interface IDatasetRebalanceCallback {
/**
- * The action to perform before the target dataset is populated.
+ * The check to perform before the target dataset is populated.
*
* @param metadataProvider,
* the metadata provider.
@@ -40,9 +40,13 @@
* the target dataset.
* @param hcc,
* the hyracks client connection.
+ *
+ * @return <code>true</code> if the rebalance of the dataset should proceed, otherwise <code>false</code> to skip.
+ * If the dataset is skipped, the active metadata transaction context, if any, can be expected to be aborted.
+ *
* @throws HyracksDataException
*/
- void beforeRebalance(MetadataProvider metadataProvider, Dataset source, Dataset target,
+ boolean canRebalance(MetadataProvider metadataProvider, Dataset source, Dataset target,
IHyracksClientConnection hcc) throws HyracksDataException;
/**
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/rebalance/NoOpDatasetRebalanceCallback.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/rebalance/NoOpDatasetRebalanceCallback.java
index 680adbf..7085567 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/rebalance/NoOpDatasetRebalanceCallback.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/rebalance/NoOpDatasetRebalanceCallback.java
@@ -33,9 +33,10 @@
}
@Override
- public void beforeRebalance(MetadataProvider metadataProvider, Dataset source, Dataset target,
+ public boolean canRebalance(MetadataProvider metadataProvider, Dataset source, Dataset target,
IHyracksClientConnection hcc) {
// Does nothing.
+ return true;
}
@Override
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/util/MetadataBuiltinFunctions.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/util/MetadataBuiltinFunctions.java
index d6dc67c..5a2ef3c 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/util/MetadataBuiltinFunctions.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/util/MetadataBuiltinFunctions.java
@@ -26,6 +26,7 @@
import org.apache.asterix.app.function.FeedRewriter;
import org.apache.asterix.app.function.JobSummariesRewriter;
import org.apache.asterix.app.function.PingRewriter;
+import org.apache.asterix.app.function.QueryIndexRewriter;
import org.apache.asterix.app.function.StorageComponentsRewriter;
import org.apache.asterix.app.function.TPCDSAllTablesDataGeneratorRewriter;
import org.apache.asterix.app.function.TPCDSSingleTableDataGeneratorRewriter;
@@ -95,6 +96,10 @@
(expression, env, mp) -> RecordUtil.FULLY_OPEN_RECORD_TYPE, true);
BuiltinFunctions.addUnnestFun(DumpIndexRewriter.DUMP_INDEX, false);
BuiltinFunctions.addDatasourceFunction(DumpIndexRewriter.DUMP_INDEX, DumpIndexRewriter.INSTANCE);
+ // Query index function
+ BuiltinFunctions.addFunction(QueryIndexRewriter.QUERY_INDEX, QueryIndexRewriter.INSTANCE, true);
+ BuiltinFunctions.addUnnestFun(QueryIndexRewriter.QUERY_INDEX, false);
+ BuiltinFunctions.addDatasourceFunction(QueryIndexRewriter.QUERY_INDEX, QueryIndexRewriter.INSTANCE);
}
private MetadataBuiltinFunctions() {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RebalanceUtil.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RebalanceUtil.java
index e2d8e01..601cd02 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RebalanceUtil.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RebalanceUtil.java
@@ -20,6 +20,8 @@
import static org.apache.asterix.app.translator.QueryTranslator.abort;
import static org.apache.asterix.common.config.DatasetConfig.DatasetType;
+import static org.apache.asterix.common.utils.IdentifierUtil.dataset;
+import static org.apache.asterix.metadata.utils.DatasetUtil.getFullyQualifiedDisplayName;
import static org.apache.hyracks.storage.am.common.dataflow.IndexDropOperatorDescriptor.DropOption;
import java.rmi.RemoteException;
@@ -28,6 +30,7 @@
import java.util.HashSet;
import java.util.List;
import java.util.Set;
+import java.util.stream.Collectors;
import java.util.stream.IntStream;
import org.apache.asterix.active.IActiveEntityEventsListener;
@@ -89,13 +92,15 @@
* the metadata provider.
* @param hcc,
* the reusable hyracks connection.
+ * @return <code>false</code> if the rebalance was safely skipped
* @throws Exception
*/
- public static void rebalance(DataverseName dataverseName, String datasetName, Set<String> targetNcNames,
+ public static boolean rebalance(DataverseName dataverseName, String datasetName, Set<String> targetNcNames,
MetadataProvider metadataProvider, IHyracksClientConnection hcc,
IDatasetRebalanceCallback datasetRebalanceCallback, boolean forceRebalance) throws Exception {
Dataset sourceDataset;
Dataset targetDataset;
+ boolean success = true;
// Executes the first Metadata transaction.
// Generates the rebalance target files. While doing that, hold read locks on the dataset so
// that no one can drop the rebalance source dataset.
@@ -107,13 +112,13 @@
// If the source dataset doesn't exist, then it's a no-op.
if (sourceDataset == null) {
- return;
+ return true;
}
Set<String> sourceNodes = new HashSet<>(metadataProvider.findNodes(sourceDataset.getNodeGroupName()));
if (!forceRebalance && sourceNodes.equals(targetNcNames)) {
- return;
+ return true;
}
if (!targetNcNames.isEmpty()) {
@@ -124,20 +129,25 @@
// The target dataset for rebalance.
targetDataset = sourceDataset.getTargetDatasetForRebalance(nodeGroupName);
- LOGGER.info("Rebalancing dataset {} from node group {} with nodes {} to node group {} with nodes {}",
- sourceDataset.getDatasetName(), sourceDataset.getNodeGroupName(), sourceNodes,
- targetDataset.getNodeGroupName(), targetNcNames);
+ LOGGER.info("Rebalancing {} {} from node group {} with nodes {} to node group {} with nodes {}",
+ dataset(), getFullyQualifiedDisplayName(sourceDataset), sourceDataset.getNodeGroupName(),
+ sourceNodes, targetDataset.getNodeGroupName(), targetNcNames);
// Rebalances the source dataset into the target dataset.
if (sourceDataset.getDatasetType() != DatasetType.EXTERNAL) {
- rebalance(sourceDataset, targetDataset, metadataProvider, hcc, datasetRebalanceCallback);
+ success = rebalance(sourceDataset, targetDataset, metadataProvider, hcc, datasetRebalanceCallback);
}
} else {
targetDataset = null;
// if this the last NC in the cluster, just drop the dataset
purgeDataset(sourceDataset, metadataProvider, hcc);
}
- // Complete the metadata transaction.
- MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ if (success) {
+ // Complete the metadata transaction.
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ } else {
+ // Abort the metadata transaction, since we failed to rebalance the dataset
+ MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
+ }
} catch (Exception e) {
abort(e, e, mdTxnCtx);
throw e;
@@ -145,7 +155,10 @@
if (targetNcNames.isEmpty()) {
// Nothing else to do since the dataset was dropped.
- return;
+ return true;
+ } else if (!success) {
+ LOGGER.info("Dataset {} rebalance was skipped, see above log for reason", datasetName);
+ return false;
}
// Up to this point, since the bulk part of a rebalance operation is done,
// the following two operations will retry after interrupt and finally rethrow InterruptedException,
@@ -164,6 +177,7 @@
runMetadataTransaction(metadataProvider, () -> dropSourceDataset(sourceDataset, metadataProvider, hcc));
});
LOGGER.info("Dataset {} rebalance completed successfully", datasetName);
+ return true;
}
@FunctionalInterface
@@ -213,13 +227,16 @@
}
// Rebalances from the source to the target.
- private static void rebalance(Dataset source, Dataset target, MetadataProvider metadataProvider,
+ private static boolean rebalance(Dataset source, Dataset target, MetadataProvider metadataProvider,
IHyracksClientConnection hcc, IDatasetRebalanceCallback datasetRebalanceCallback) throws Exception {
// Drops the target dataset files (if any) to make rebalance idempotent.
dropDatasetFiles(target, metadataProvider, hcc);
// Performs the specified operation before the target dataset is populated.
- datasetRebalanceCallback.beforeRebalance(metadataProvider, source, target, hcc);
+ if (!datasetRebalanceCallback.canRebalance(metadataProvider, source, target, hcc)) {
+ // the callback indicates that this rebalance should be skipped; short circuit the remaining steps
+ return false;
+ }
// Creates the rebalance target.
createRebalanceTarget(target, metadataProvider, hcc);
@@ -232,6 +249,8 @@
// Performs the specified operation after the target dataset is populated.
datasetRebalanceCallback.afterRebalance(metadataProvider, source, target, hcc);
+
+ return true;
}
// Switches the metadata entity from the source dataset to the target dataset.
@@ -373,10 +392,19 @@
// Creates and loads all secondary indexes for the rebalance target dataset.
private static void createAndLoadSecondaryIndexesForTarget(Dataset source, Dataset target,
MetadataProvider metadataProvider, IHyracksClientConnection hcc) throws Exception {
- for (Index index : metadataProvider.getDatasetIndexes(source.getDataverseName(), source.getDatasetName())) {
- if (!index.isSecondaryIndex()) {
- continue;
- }
+ List<Index> indexes = metadataProvider.getDatasetIndexes(source.getDataverseName(), source.getDatasetName());
+ List<Index> secondaryIndexes = indexes.stream().filter(Index::isSecondaryIndex).collect(Collectors.toList());
+ List<Index> nonSampleIndexes =
+ secondaryIndexes.stream().filter(idx -> !idx.isSampleIndex()).collect(Collectors.toList());
+ List<Index> sampleIndexes = secondaryIndexes.stream().filter(Index::isSampleIndex).collect(Collectors.toList());
+ // must create all non samples secondary indexes first since samples need the stats of secondary indexes
+ createAndLoadIndexes(target, metadataProvider, hcc, nonSampleIndexes);
+ createAndLoadIndexes(target, metadataProvider, hcc, sampleIndexes);
+ }
+
+ private static void createAndLoadIndexes(Dataset target, MetadataProvider metadataProvider,
+ IHyracksClientConnection hcc, List<Index> indexes) throws Exception {
+ for (Index index : indexes) {
// Creates the secondary index.
JobSpecification indexCreationJobSpec =
IndexUtil.buildSecondaryIndexCreationJobSpec(target, index, metadataProvider, null);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RedactionUtil.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RedactionUtil.java
index 156b78a..d835664 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RedactionUtil.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RedactionUtil.java
@@ -26,6 +26,8 @@
import org.apache.hyracks.util.ILogRedactor;
+import com.fasterxml.jackson.databind.node.TextNode;
+
public class RedactionUtil {
private RedactionUtil() {
throw new AssertionError("do not instantiate");
@@ -34,6 +36,7 @@
private static final Pattern STATEMENT_PATTERN =
Pattern.compile("(" + SECRET_ACCESS_KEY_FIELD_NAME + ").*", CASE_INSENSITIVE | DOTALL);
private static final String STATEMENT_REPLACEMENT = "$1...<redacted sensitive data>";
+ public static final TextNode REDACTED_SENSITIVE_ENTRY_VALUE = new TextNode("<redacted sensitive entry>");
public static final ILogRedactor LOG_REDACTOR = new ILogRedactor() {
@Override
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/ResourceUtils.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/ResourceUtils.java
index ba11956..353bebf 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/ResourceUtils.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/ResourceUtils.java
@@ -24,6 +24,7 @@
import org.apache.asterix.app.resource.OperatorResourcesComputer;
import org.apache.asterix.app.resource.PlanStage;
import org.apache.asterix.app.resource.PlanStagesGenerator;
+import org.apache.asterix.common.config.CompilerProperties;
import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
@@ -38,25 +39,28 @@
}
/**
- * Calculates the required cluster capacity from a given query plan, the computation locations,
- * the operator memory budgets, and frame size.
+ * Calculates the required cluster capacity from a given query plan, the computation locations, the operator memory
+ * budgets, and frame size.
*
* @param plan,
- * a given query plan.
+ * a given query plan.
* @param computationLocations,
- * the partitions for computation.
+ * the partitions for computation.
* @param physicalOptimizationConfig,
- * a PhysicalOptimizationConfig.
+ * a PhysicalOptimizationConfig.
+ * @param compilerProperties
* @return the required cluster capacity for executing the query.
* @throws AlgebricksException
- * if the query plan is malformed.
+ * if the query plan is malformed.
*/
public static IClusterCapacity getRequiredCapacity(ILogicalPlan plan,
AlgebricksAbsolutePartitionConstraint computationLocations,
- PhysicalOptimizationConfig physicalOptimizationConfig) throws AlgebricksException {
+ PhysicalOptimizationConfig physicalOptimizationConfig, CompilerProperties compilerProperties)
+ throws AlgebricksException {
final int frameSize = physicalOptimizationConfig.getFrameSize();
final List<PlanStage> planStages = getStages(plan);
- return getStageBasedRequiredCapacity(planStages, computationLocations.getLocations().length, frameSize);
+ return getStageBasedRequiredCapacity(planStages, computationLocations.getLocations().length, frameSize,
+ compilerProperties);
}
public static List<PlanStage> getStages(ILogicalPlan plan) throws AlgebricksException {
@@ -68,8 +72,9 @@
}
public static IClusterCapacity getStageBasedRequiredCapacity(List<PlanStage> stages, int computationLocations,
- int frameSize) {
- final OperatorResourcesComputer computer = new OperatorResourcesComputer(computationLocations, frameSize);
+ int frameSize, CompilerProperties compilerProperties) {
+ final OperatorResourcesComputer computer =
+ new OperatorResourcesComputer(computationLocations, frameSize, compilerProperties);
final IClusterCapacity clusterCapacity = new ClusterCapacity();
final long maxRequiredMemory = stages.stream().mapToLong(stage -> stage.getRequiredMemory(computer)).max()
.orElseThrow(IllegalStateException::new);
diff --git a/asterixdb/asterix-app/src/main/resources/cc-cloud-storage.conf b/asterixdb/asterix-app/src/main/resources/cc-cloud-storage.conf
new file mode 100644
index 0000000..a61ac87
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/resources/cc-cloud-storage.conf
@@ -0,0 +1,71 @@
+<<<<<<< HEAD (5f076a [ASTERIXDB-3221][REPL] Use IO scheduler for replication ops)
+=======
+; Licensed to the Apache Software Foundation (ASF) under one
+; or more contributor license agreements. See the NOTICE file
+; distributed with this work for additional information
+; regarding copyright ownership. The ASF licenses this file
+; to you under the Apache License, Version 2.0 (the
+; "License"); you may not use this file except in compliance
+; with the License. You may obtain a copy of the License at
+;
+; http://www.apache.org/licenses/LICENSE-2.0
+;
+; Unless required by applicable law or agreed to in writing,
+; software distributed under the License is distributed on an
+; "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+; KIND, either express or implied. See the License for the
+; specific language governing permissions and limitations
+; under the License.
+
+[nc/asterix_nc1]
+txn.log.dir=target/tmp/asterix_nc1/txnlog
+core.dump.dir=target/tmp/asterix_nc1/coredump
+iodevices=asterix_nc1/iodevice1
+iodevices=asterix_nc1/iodevice2
+nc.api.port=19004
+#jvm.args=-agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=5006
+
+[nc/asterix_nc2]
+ncservice.port=9091
+txn.log.dir=target/tmp/asterix_nc2/txnlog
+core.dump.dir=target/tmp/asterix_nc2/coredump
+iodevices=asterix_nc2/iodevice1
+iodevices=asterix_nc2/iodevice2
+nc.api.port=19005
+#jvm.args=-agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=5007
+
+[nc]
+address=127.0.0.1
+command=asterixnc
+app.class=org.apache.asterix.hyracks.bootstrap.NCApplication
+jvm.args=-Xmx4096m -Dnode.Resolver="org.apache.asterix.external.util.IdentitiyResolverFactory"
+storage.buffercache.size=128MB
+storage.memorycomponent.globalbudget=512MB
+storage.io.scheduler=greedy
+storage.filtered.memorycomponent.max.size=16MB
+
+[cc]
+address = 127.0.0.1
+app.class=org.apache.asterix.hyracks.bootstrap.CCApplication
+heartbeat.period=2000
+heartbeat.max.misses=25
+
+[common]
+log.dir = logs/
+log.level = INFO
+compiler.cbo=false
+compiler.cbotest=true
+compiler.queryplanshape=zigzag
+compiler.framesize=32KB
+compiler.sortmemory=320KB
+compiler.groupmemory=160KB
+compiler.joinmemory=256KB
+compiler.textsearchmemory=160KB
+compiler.windowmemory=192KB
+compiler.sort.parallel=false
+compiler.internal.sanitycheck=true
+messaging.frame.size=4096
+messaging.frame.count=512
+cloud.deployment=true
+storage.buffercache.pagesize=32KB
+>>>>>>> CHANGE (337517 [NO ISSUE][OTH] Move page size storage config to common conf)
diff --git a/asterixdb/asterix-app/src/main/resources/cc.conf b/asterixdb/asterix-app/src/main/resources/cc.conf
index 98ede89..b531940 100644
--- a/asterixdb/asterix-app/src/main/resources/cc.conf
+++ b/asterixdb/asterix-app/src/main/resources/cc.conf
@@ -37,7 +37,6 @@
command=asterixnc
app.class=org.apache.asterix.hyracks.bootstrap.NCApplication
jvm.args=-Xmx4096m -Dnode.Resolver="org.apache.asterix.external.util.IdentitiyResolverFactory"
-storage.buffercache.pagesize=32KB
storage.buffercache.size=128MB
storage.memorycomponent.globalbudget=512MB
storage.io.scheduler=greedy
@@ -52,6 +51,9 @@
[common]
log.dir = logs/
log.level = INFO
+compiler.cbo=false
+compiler.cbotest=true
+compiler.queryplanshape=zigzag
compiler.framesize=32KB
compiler.sortmemory=320KB
compiler.groupmemory=160KB
@@ -62,3 +64,4 @@
compiler.internal.sanitycheck=true
messaging.frame.size=4096
messaging.frame.count=512
+storage.buffercache.pagesize=32KB
diff --git a/asterixdb/asterix-app/src/main/resources/cc2.conf b/asterixdb/asterix-app/src/main/resources/cc2.conf
index 016c485..3f456cf 100644
--- a/asterixdb/asterix-app/src/main/resources/cc2.conf
+++ b/asterixdb/asterix-app/src/main/resources/cc2.conf
@@ -35,7 +35,6 @@
command=asterixnc
app.class=org.apache.asterix.hyracks.bootstrap.NCApplication
jvm.args=-Xmx4096m -Dnode.Resolver="org.apache.asterix.external.util.IdentitiyResolverFactory"
-storage.buffercache.pagesize=32KB
storage.buffercache.size=128MB
storage.memorycomponent.globalbudget=512MB
@@ -48,6 +47,9 @@
[common]
log.dir = logs/
log.level = WARN
+compiler.cbo=false
+compiler.cbotest=true
+compiler.queryplanshape=zigzag
compiler.framesize=32KB
compiler.sortmemory=320KB
compiler.groupmemory=160KB
@@ -57,3 +59,4 @@
compiler.parallelism=-1
messaging.frame.size=4096
messaging.frame.count=512
+storage.buffercache.pagesize=32KB
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/main/resources/cc3.conf b/asterixdb/asterix-app/src/main/resources/cc3.conf
index 88362aa..aac7cbe 100644
--- a/asterixdb/asterix-app/src/main/resources/cc3.conf
+++ b/asterixdb/asterix-app/src/main/resources/cc3.conf
@@ -35,7 +35,6 @@
command=asterixnc
app.class=org.apache.asterix.hyracks.bootstrap.NCApplication
jvm.args=-Xmx4096m -Dnode.Resolver="org.apache.asterix.external.util.IdentitiyResolverFactory"
-storage.buffercache.pagesize=32KB
storage.buffercache.size=128MB
storage.memorycomponent.globalbudget=512MB
@@ -48,6 +47,9 @@
[common]
log.dir = logs/
log.level = WARN
+compiler.cbo=false
+compiler.cbotest=true
+compiler.queryplanshape=zigzag
compiler.framesize=32KB
compiler.sortmemory=320KB
compiler.groupmemory=160KB
@@ -57,3 +59,4 @@
compiler.parallelism=3
messaging.frame.size=4096
messaging.frame.count=512
+storage.buffercache.pagesize=32KB
diff --git a/asterixdb/asterix-app/src/main/resources/cc4.conf b/asterixdb/asterix-app/src/main/resources/cc4.conf
index 6a66d25..bf37499 100644
--- a/asterixdb/asterix-app/src/main/resources/cc4.conf
+++ b/asterixdb/asterix-app/src/main/resources/cc4.conf
@@ -35,7 +35,6 @@
command=asterixnc
app.class=org.apache.asterix.hyracks.bootstrap.NCApplication
jvm.args=-Xmx4096m -Dnode.Resolver="org.apache.asterix.external.util.IdentitiyResolverFactory"
-storage.buffercache.pagesize=32KB
[cc]
address = 127.0.0.1
@@ -54,3 +53,4 @@
messaging.frame.size=4096
messaging.frame.count=512
compiler.parallelism=-1
+storage.buffercache.pagesize=32KB
diff --git a/asterixdb/asterix-app/src/main/resources/cc_no_cbo.conf b/asterixdb/asterix-app/src/main/resources/cc_no_cbo.conf
new file mode 100644
index 0000000..ab028c8
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/resources/cc_no_cbo.conf
@@ -0,0 +1,66 @@
+; Licensed to the Apache Software Foundation (ASF) under one
+; or more contributor license agreements. See the NOTICE file
+; distributed with this work for additional information
+; regarding copyright ownership. The ASF licenses this file
+; to you under the Apache License, Version 2.0 (the
+; "License"); you may not use this file except in compliance
+; with the License. You may obtain a copy of the License at
+;
+; http://www.apache.org/licenses/LICENSE-2.0
+;
+; Unless required by applicable law or agreed to in writing,
+; software distributed under the License is distributed on an
+; "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+; KIND, either express or implied. See the License for the
+; specific language governing permissions and limitations
+; under the License.
+
+[nc/asterix_nc1]
+txn.log.dir=target/tmp/asterix_nc1/txnlog
+core.dump.dir=target/tmp/asterix_nc1/coredump
+iodevices=asterix_nc1/iodevice1
+iodevices=asterix_nc1/iodevice2
+nc.api.port=19004
+#jvm.args=-agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=5006
+
+[nc/asterix_nc2]
+ncservice.port=9091
+txn.log.dir=target/tmp/asterix_nc2/txnlog
+core.dump.dir=target/tmp/asterix_nc2/coredump
+iodevices=asterix_nc2/iodevice1
+iodevices=asterix_nc2/iodevice2
+nc.api.port=19005
+#jvm.args=-agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=5007
+
+[nc]
+address=127.0.0.1
+command=asterixnc
+app.class=org.apache.asterix.hyracks.bootstrap.NCApplication
+jvm.args=-Xmx4096m -Dnode.Resolver="org.apache.asterix.external.util.IdentitiyResolverFactory"
+storage.buffercache.size=128MB
+storage.memorycomponent.globalbudget=512MB
+storage.io.scheduler=greedy
+storage.filtered.memorycomponent.max.size=16MB
+
+[cc]
+address = 127.0.0.1
+app.class=org.apache.asterix.hyracks.bootstrap.CCApplication
+heartbeat.period=2000
+heartbeat.max.misses=25
+
+[common]
+log.dir = logs/
+log.level = INFO
+compiler.cbo=false
+compiler.cbotest=false
+compiler.framesize=32KB
+compiler.sortmemory=320KB
+compiler.groupmemory=160KB
+compiler.joinmemory=256KB
+compiler.textsearchmemory=160KB
+compiler.windowmemory=192KB
+compiler.sort.parallel=false
+compiler.internal.sanitycheck=true
+messaging.frame.size=4096
+messaging.frame.count=512
+storage.buffercache.pagesize=32KB
diff --git a/asterixdb/asterix-app/src/main/resources/cc_static_partitioning.conf b/asterixdb/asterix-app/src/main/resources/cc_static_partitioning.conf
new file mode 100644
index 0000000..e3e779e
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/resources/cc_static_partitioning.conf
@@ -0,0 +1,71 @@
+<<<<<<< HEAD (5f076a [ASTERIXDB-3221][REPL] Use IO scheduler for replication ops)
+=======
+; Licensed to the Apache Software Foundation (ASF) under one
+; or more contributor license agreements. See the NOTICE file
+; distributed with this work for additional information
+; regarding copyright ownership. The ASF licenses this file
+; to you under the Apache License, Version 2.0 (the
+; "License"); you may not use this file except in compliance
+; with the License. You may obtain a copy of the License at
+;
+; http://www.apache.org/licenses/LICENSE-2.0
+;
+; Unless required by applicable law or agreed to in writing,
+; software distributed under the License is distributed on an
+; "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+; KIND, either express or implied. See the License for the
+; specific language governing permissions and limitations
+; under the License.
+
+[nc/asterix_nc1]
+txn.log.dir=target/tmp/asterix_nc1/txnlog
+core.dump.dir=target/tmp/asterix_nc1/coredump
+iodevices=asterix_nc1/iodevice1
+iodevices=asterix_nc1/iodevice2
+nc.api.port=19004
+#jvm.args=-agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=5006
+
+[nc/asterix_nc2]
+ncservice.port=9091
+txn.log.dir=target/tmp/asterix_nc2/txnlog
+core.dump.dir=target/tmp/asterix_nc2/coredump
+iodevices=asterix_nc2/iodevice1
+iodevices=asterix_nc2/iodevice2
+nc.api.port=19005
+#jvm.args=-agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=5007
+
+[nc]
+address=127.0.0.1
+command=asterixnc
+app.class=org.apache.asterix.hyracks.bootstrap.NCApplication
+jvm.args=-Xmx4096m -Dnode.Resolver="org.apache.asterix.external.util.IdentitiyResolverFactory"
+storage.buffercache.size=128MB
+storage.memorycomponent.globalbudget=512MB
+storage.io.scheduler=greedy
+storage.filtered.memorycomponent.max.size=16MB
+
+[cc]
+address = 127.0.0.1
+app.class=org.apache.asterix.hyracks.bootstrap.CCApplication
+heartbeat.period=2000
+heartbeat.max.misses=25
+
+[common]
+log.dir = logs/
+log.level = INFO
+compiler.cbo=false
+compiler.cbotest=true
+compiler.queryplanshape=zigzag
+compiler.framesize=32KB
+compiler.sortmemory=320KB
+compiler.groupmemory=160KB
+compiler.joinmemory=256KB
+compiler.textsearchmemory=160KB
+compiler.windowmemory=192KB
+compiler.sort.parallel=false
+compiler.internal.sanitycheck=true
+messaging.frame.size=4096
+messaging.frame.count=512
+storage.buffercache.pagesize=32KB
+storage.partitioning=static
+>>>>>>> CHANGE (337517 [NO ISSUE][OTH] Move page size storage config to common conf)
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java
index 86d8ed4..34f4e40 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java
@@ -63,6 +63,7 @@
import org.apache.logging.log4j.Level;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
+import org.jetbrains.annotations.NotNull;
import org.kohsuke.args4j.CmdLineException;
@SuppressWarnings({ "squid:ClassVariableVisibilityCheck", "squid:S00112" })
@@ -129,7 +130,7 @@
configManager.processConfig();
ccConfig.setKeyStorePath(joinPath(RESOURCES_PATH, ccConfig.getKeyStorePath()));
ccConfig.setTrustStorePath(joinPath(RESOURCES_PATH, ccConfig.getTrustStorePath()));
- cc = new ClusterControllerService(ccConfig, ccApplication);
+ cc = createCC(ccApplication, ccConfig);
nodeNames = ccConfig.getConfigManager().getNodeNames();
if (deleteOldInstanceData) {
@@ -150,8 +151,7 @@
}
ncApplication.registerConfig(ncConfigManager);
opts.forEach(opt -> ncConfigManager.set(nodeId, opt.getLeft(), opt.getRight()));
- nodeControllers
- .add(new NodeControllerService(fixupPaths(createNCConfig(nodeId, ncConfigManager)), ncApplication));
+ nodeControllers.add(createNC(fixupPaths(createNCConfig(nodeId, ncConfigManager)), ncApplication));
}
opts.forEach(opt -> configManager.set(opt.getLeft(), opt.getRight()));
@@ -185,6 +185,7 @@
this.ncs = nodeControllers.toArray(new NodeControllerService[nodeControllers.size()]);
}
+ @NotNull
private void configureExternalLibDir() {
// hack to ensure we have a unique location for external libraries in our tests (asterix cluster has a shared
// home directory)-- TODO: rework this once the external lib dir can be configured explicitly
@@ -223,10 +224,20 @@
return ccConfig;
}
+ protected ClusterControllerService createCC(ICCApplication ccApplication, CCConfig ccConfig) throws Exception {
+ return new ClusterControllerService(ccConfig, ccApplication);
+ }
+
protected ICCApplication createCCApplication() {
return new CCApplication();
}
+ @NotNull
+ protected NodeControllerService createNC(NCConfig config, INCApplication ncApplication)
+ throws IOException, CmdLineException, AsterixException {
+ return new NodeControllerService(config, ncApplication);
+ }
+
protected NCConfig createNCConfig(String ncName, ConfigManager configManager) {
NCConfig ncConfig = new NCConfig(ncName, configManager);
ncConfig.setClusterAddress("localhost");
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/QueryCancellationServletTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/QueryCancellationServletTest.java
index 68fb9a8..673611d 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/QueryCancellationServletTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/QueryCancellationServletTest.java
@@ -27,7 +27,7 @@
import java.util.concurrent.ConcurrentHashMap;
-import org.apache.asterix.api.http.server.CcQueryCancellationServlet;
+import org.apache.asterix.api.http.server.ActiveRequestsServlet;
import org.apache.asterix.api.http.server.ServletConstants;
import org.apache.asterix.app.translator.RequestParameters;
import org.apache.asterix.common.api.RequestReference;
@@ -35,8 +35,10 @@
import org.apache.asterix.common.dataflow.ICcApplicationContext;
import org.apache.asterix.runtime.utils.RequestTracker;
import org.apache.asterix.translator.ClientRequest;
+import org.apache.hyracks.api.application.ICCServiceContext;
import org.apache.hyracks.api.client.IHyracksClientConnection;
import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.control.cc.ClusterControllerService;
import org.apache.hyracks.http.api.IServletRequest;
import org.apache.hyracks.http.api.IServletResponse;
import org.junit.Test;
@@ -57,8 +59,8 @@
RequestTracker tracker = new RequestTracker(appCtx);
Mockito.when(appCtx.getRequestTracker()).thenReturn(tracker);
// Creates a query cancellation servlet.
- CcQueryCancellationServlet cancellationServlet =
- new CcQueryCancellationServlet(new ConcurrentHashMap<>(), appCtx, new String[] { "/" });
+ ActiveRequestsServlet cancellationServlet =
+ new ActiveRequestsServlet(new ConcurrentHashMap<>(), appCtx, new String[] { "/" });
// Adds mocked Hyracks client connection into the servlet context.
IHyracksClientConnection mockHcc = mock(IHyracksClientConnection.class);
cancellationServlet.ctx().put(ServletConstants.HYRACKS_CONNECTION_ATTR, mockHcc);
@@ -66,6 +68,11 @@
// Tests the case that query is not in the map.
IServletRequest mockRequest = mockRequest("1");
IServletResponse mockResponse = mock(IServletResponse.class);
+ ICCServiceContext mockCCServiceCtx = mock(ICCServiceContext.class);
+ ClusterControllerService mockCCService = mock(ClusterControllerService.class);
+ Mockito.when(appCtx.getServiceContext()).thenReturn(mockCCServiceCtx);
+ Mockito.when(appCtx.getServiceContext().getControllerService()).thenReturn(mockCCService);
+ Mockito.when(mockCCServiceCtx.getControllerService()).thenReturn(mockCCService);
cancellationServlet.handle(mockRequest, mockResponse);
verify(mockResponse, times(1)).setStatus(HttpResponseStatus.NOT_FOUND);
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/VersionApiServletTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/VersionApiServletTest.java
index 8dcfa26..eb63218 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/VersionApiServletTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/VersionApiServletTest.java
@@ -31,8 +31,8 @@
import java.util.concurrent.ConcurrentHashMap;
import org.apache.asterix.api.http.server.VersionApiServlet;
+import org.apache.asterix.app.cc.CcApplicationContext;
import org.apache.asterix.common.config.BuildProperties;
-import org.apache.asterix.runtime.utils.CcApplicationContext;
import org.apache.hyracks.api.client.IHyracksClientConnection;
import org.apache.hyracks.http.api.IServletRequest;
import org.apache.hyracks.http.api.IServletResponse;
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
index 0b80881..34696b1 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
@@ -26,6 +26,7 @@
import java.util.List;
import java.util.Map;
+import org.apache.asterix.app.cc.CcApplicationContext;
import org.apache.asterix.app.nc.NCAppRuntimeContext;
import org.apache.asterix.app.nc.TransactionSubsystem;
import org.apache.asterix.common.config.TransactionProperties;
@@ -61,7 +62,6 @@
import org.apache.asterix.runtime.operators.LSMIndexBulkLoadOperatorNodePushable;
import org.apache.asterix.runtime.operators.LSMPrimaryInsertOperatorNodePushable;
import org.apache.asterix.runtime.operators.LSMPrimaryUpsertOperatorNodePushable;
-import org.apache.asterix.runtime.utils.CcApplicationContext;
import org.apache.asterix.test.runtime.ExecutionTestUtil;
import org.apache.asterix.transaction.management.runtime.CommitRuntime;
import org.apache.asterix.transaction.management.service.logging.LogReader;
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/resource/PlanStagesGeneratorTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/resource/PlanStagesGeneratorTest.java
index b0de85e..77d0e60 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/resource/PlanStagesGeneratorTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/resource/PlanStagesGeneratorTest.java
@@ -340,7 +340,7 @@
}
}
final IClusterCapacity clusterCapacity =
- ResourceUtils.getStageBasedRequiredCapacity(stages, PARALLELISM, FRAME_SIZE);
+ ResourceUtils.getStageBasedRequiredCapacity(stages, PARALLELISM, FRAME_SIZE, null);
Assert.assertEquals(clusterCapacity.getAggregatedMemoryByteSize(), expectedMemory);
}
}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ClusterStateManagerTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ClusterStateManagerTest.java
index b80fa30..a2a3b48 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ClusterStateManagerTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ClusterStateManagerTest.java
@@ -26,6 +26,7 @@
import java.util.TreeMap;
import java.util.concurrent.ConcurrentHashMap;
+import org.apache.asterix.app.cc.CcApplicationContext;
import org.apache.asterix.app.replication.NcLifecycleCoordinator;
import org.apache.asterix.app.replication.message.NCLifecycleTaskReportMessage;
import org.apache.asterix.common.api.IClusterManagementWork.ClusterState;
@@ -37,7 +38,6 @@
import org.apache.asterix.hyracks.bootstrap.CCApplication;
import org.apache.asterix.runtime.transaction.ResourceIdManager;
import org.apache.asterix.runtime.utils.BulkTxnIdFactory;
-import org.apache.asterix.runtime.utils.CcApplicationContext;
import org.apache.asterix.runtime.utils.ClusterStateManager;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.api.application.ICCServiceContext;
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/PartitionManagerTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/PartitionManagerTest.java
index 218ef97..f8dbaad 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/PartitionManagerTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/PartitionManagerTest.java
@@ -60,7 +60,7 @@
public void failedJobPartitionRequestTest() throws Exception {
final NodeControllerService nc1 = integrationUtil.ncs[0];
final NodeControllerService nc2 = integrationUtil.ncs[1];
- final JobId failedJob = new JobId(-1);
+ final JobId failedJob = new JobId(10);
nc2.getPartitionManager().jobCompleted(failedJob, JobStatus.FAILURE);
final NetworkAddress localNetworkAddress = nc2.getNetworkManager().getPublicNetworkAddress();
final InetSocketAddress nc2Address =
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveEventsListenerTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveEventsListenerTest.java
index c71e602..e5e33d0 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveEventsListenerTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveEventsListenerTest.java
@@ -34,6 +34,7 @@
import org.apache.asterix.active.EntityId;
import org.apache.asterix.active.NoRetryPolicyFactory;
import org.apache.asterix.app.active.ActiveNotificationHandler;
+import org.apache.asterix.app.cc.CcApplicationContext;
import org.apache.asterix.common.api.IClusterManagementWork.ClusterState;
import org.apache.asterix.common.api.IMetadataLockManager;
import org.apache.asterix.common.cluster.IClusterStateManager;
@@ -53,7 +54,6 @@
import org.apache.asterix.metadata.utils.MetadataLockUtil;
import org.apache.asterix.runtime.functions.FunctionCollection;
import org.apache.asterix.runtime.functions.FunctionManager;
-import org.apache.asterix.runtime.utils.CcApplicationContext;
import org.apache.asterix.test.active.TestEventsListener.Behavior;
import org.apache.asterix.test.base.TestMethodTracer;
import org.apache.asterix.translator.IStatementExecutor;
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveStatsTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveStatsTest.java
index c51fd95..c9d862c 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveStatsTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveStatsTest.java
@@ -37,6 +37,7 @@
import org.apache.asterix.app.active.ActiveEntityEventsListener;
import org.apache.asterix.app.active.ActiveNotificationHandler;
import org.apache.asterix.app.cc.CCExtensionManager;
+import org.apache.asterix.app.cc.CcApplicationContext;
import org.apache.asterix.app.nc.NCAppRuntimeContext;
import org.apache.asterix.app.result.ResponsePrinter;
import org.apache.asterix.common.exceptions.ErrorCode;
@@ -45,7 +46,6 @@
import org.apache.asterix.external.operators.FeedIntakeOperatorNodePushable;
import org.apache.asterix.metadata.declared.MetadataProvider;
import org.apache.asterix.metadata.entities.Dataset;
-import org.apache.asterix.runtime.utils.CcApplicationContext;
import org.apache.asterix.test.runtime.ExecutionTestUtil;
import org.apache.asterix.translator.IStatementExecutor;
import org.apache.asterix.translator.SessionOutput;
@@ -53,6 +53,7 @@
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.job.JobId;
import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.api.job.resource.IJobCapacityController;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
@@ -135,8 +136,9 @@
TestUserActor user = new TestUserActor("Xikui", mdProvider, null);
Action start = user.startActivity(eventsListener);
startingSubscriber.sync();
- activeJobNotificationHandler.notifyJobCreation(jobId, jobSpec);
- activeJobNotificationHandler.notifyJobStart(jobId);
+ activeJobNotificationHandler.notifyJobCreation(jobId, jobSpec,
+ IJobCapacityController.JobSubmissionStatus.EXECUTE);
+ activeJobNotificationHandler.notifyJobStart(jobId, jobSpec);
try {
eventsListener.refreshStats(1000);
} catch (HyracksDataException e) {
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/TestClusterControllerActor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/TestClusterControllerActor.java
index 52d4225..883a0cb 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/TestClusterControllerActor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/TestClusterControllerActor.java
@@ -28,6 +28,7 @@
import org.apache.hyracks.api.job.JobId;
import org.apache.hyracks.api.job.JobSpecification;
import org.apache.hyracks.api.job.JobStatus;
+import org.apache.hyracks.api.job.resource.IJobCapacityController;
import org.mockito.Mockito;
public class TestClusterControllerActor extends Actor {
@@ -49,8 +50,8 @@
JobSpecification jobSpecification = Mockito.mock(JobSpecification.class);
Mockito.when(jobSpecification.getProperty(ActiveNotificationHandler.ACTIVE_ENTITY_PROPERTY_NAME))
.thenReturn(entityId);
- handler.notifyJobCreation(jobId, jobSpecification);
- handler.notifyJobStart(jobId);
+ handler.notifyJobCreation(jobId, jobSpecification, IJobCapacityController.JobSubmissionStatus.EXECUTE);
+ handler.notifyJobStart(jobId, null);
}
};
add(startJob);
@@ -72,7 +73,7 @@
Action delivery = new Action() {
@Override
protected void doExecute(MetadataProvider actorMdProvider) throws Exception {
- handler.notifyJobFinish(jobId, jobStatus, exceptions);
+ handler.notifyJobFinish(jobId, null, jobStatus, exceptions);
}
};
add(delivery);
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/ResultExtractor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/ResultExtractor.java
index 091fc20..05b8bbf 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/ResultExtractor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/ResultExtractor.java
@@ -214,8 +214,8 @@
result = OBJECT_READER.readValue(resultStr);
} catch (Exception e) {
// whoops, not JSON (e.g. 404) - just include the body
- GlobalConfig.ASTERIX_LOGGER.log(Level.ERROR, resultStr);
- throw new Exception(resultStr);
+ GlobalConfig.ASTERIX_LOGGER.log(Level.ERROR, "result: {} json parse exception: {}", resultStr, e);
+ throw new Exception(resultStr, e);
}
final boolean isJsonFormat = isJsonFormat(fmt);
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 14784c5..04da930 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
@@ -326,7 +326,7 @@
public void runScriptAndCompareWithResult(File scriptFile, File expectedFile, File actualFile,
ComparisonEnum compare, Charset actualEncoding, String statement) throws Exception {
- LOGGER.info("Expected results file: {} ", canonicalize(expectedFile));
+ LOGGER.info("Expected results file: {} ", canonicalize(expectedFile.getAbsolutePath()));
boolean regex = false;
if (expectedFile.getName().endsWith(".ignore")) {
return; //skip the comparison
@@ -354,6 +354,9 @@
} else if (actualFile.toString().endsWith(".unorderedtxt")) {
runScriptAndCompareWithResultUnorderedLinesText(scriptFile, readerExpected, readerActual);
return;
+ } else if (actualFile.toString().endsWith(".plan")) {
+ runScriptAndCompareWithResultPlan(scriptFile, readerExpected, readerActual);
+ return;
}
String lineExpected, lineActual;
@@ -404,24 +407,25 @@
}
} catch (Exception e) {
if (!actualEncoding.equals(UTF_8)) {
- LOGGER.info("Actual results file: {} encoding: {}", canonicalize(actualFile), actualEncoding);
+ LOGGER.info("Actual results file: {} encoding: {}", canonicalize(actualFile.getAbsolutePath()),
+ actualEncoding);
} else {
- LOGGER.info("Actual results file: {}", canonicalize(actualFile));
+ LOGGER.info("Actual results file: {}", canonicalize(actualFile.getAbsolutePath()));
}
throw e;
}
}
- private ComparisonException createLineChangedException(File scriptFile, String lineExpected, String lineActual,
- int num) {
+ public static ComparisonException createLineChangedException(File scriptFile, String lineExpected,
+ String lineActual, int num) {
return new ComparisonException("Result for " + canonicalize(scriptFile) + " changed at line " + num
+ ":\nexpected < " + truncateIfLong(lineExpected) + "\nactual > " + truncateIfLong(lineActual));
}
- private ComparisonException createLineNotFoundException(File scriptFile, String lineExpected) {
- return new ComparisonException(
- "Result for " + canonicalize(scriptFile) + " expected line not found: " + truncateIfLong(lineExpected));
+ public static ComparisonException createLineNotFoundException(File scriptFile, String lineExpected, int num) {
+ return new ComparisonException("Result for " + canonicalize(scriptFile) + " expected line at " + num
+ + " not found: " + truncateIfLong(lineExpected));
}
private ComparisonException createExpectedLinesNotReturnedException(File scriptFile, List<String> expectedLines) {
@@ -429,7 +433,7 @@
+ String.join("\n", expectedLines));
}
- private String truncateIfLong(String string) {
+ private static String truncateIfLong(String string) {
if (string.length() < TRUNCATE_THRESHOLD) {
return string;
}
@@ -636,10 +640,12 @@
List<String> expectedLines = readerExpected.lines().collect(Collectors.toList());
List<String> actualLines = readerActual.lines().collect(Collectors.toList());
+ int num = 1;
for (String line : actualLines) {
if (!expectedLines.remove(line)) {
- throw createLineNotFoundException(scriptFile, line);
+ throw createLineNotFoundException(scriptFile, line, num);
}
+ num++;
}
// number of expect > actual
@@ -648,6 +654,13 @@
}
}
+ public void runScriptAndCompareWithResultPlan(File scriptFile, BufferedReader readerExpected,
+ BufferedReader readerActual) throws Exception {
+ List<String> expectedLines = readerExpected.lines().collect(Collectors.toList());
+ List<String> actualLines = readerActual.lines().collect(Collectors.toList());
+ TestHelper.comparePlans(expectedLines, actualLines, scriptFile);
+ }
+
// For tests where you simply want the byte-for-byte output.
private static void writeOutputToFile(File actualFile, InputStream resultStream) throws Exception {
final File parentDir = actualFile.getParentFile();
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestHelper.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestHelper.java
index fcab213..8d7f188 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestHelper.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestHelper.java
@@ -27,12 +27,15 @@
import java.io.StringReader;
import java.io.StringWriter;
import java.util.BitSet;
+import java.util.Collection;
import java.util.Collections;
import java.util.Enumeration;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
import java.util.zip.ZipEntry;
import java.util.zip.ZipFile;
@@ -62,7 +65,8 @@
private static final Logger LOGGER = LogManager.getLogger();
private static final String TEST_DIR_BASE_PATH = System.getProperty("user.dir") + File.separator + "target";
private static final String[] TEST_DIRS = new String[] { "txnLogDir", "IODevice", "spill_area", "config" };
-
+ private static final String PATTERN_VAR_ID_PREFIX = "\\$\\$";
+ private static final Pattern PATTERN_VAR_ID = Pattern.compile(PATTERN_VAR_ID_PREFIX + "(\\d+)");
private static final ObjectMapper SORTED_MAPPER = new ObjectMapper();
private static final ObjectWriter PRETTY_SORTED_WRITER;
@@ -309,4 +313,68 @@
objectMapper.configure(SerializationFeature.ORDER_MAP_ENTRIES_BY_KEYS, true);
return objectMapper;
}
+
+ public static void comparePlans(List<String> linesExpected, List<String> linesActual, File queryFile)
+ throws Exception {
+ int varBaseExpected = findBaseVarId(linesExpected);
+ int varBaseActual = findBaseVarId(linesActual);
+
+ Iterator<String> readerExpected = linesExpected.iterator();
+ Iterator<String> readerActual = linesActual.iterator();
+ String lineExpected, lineActual;
+ int num = 1;
+ while (readerExpected.hasNext()) {
+ lineExpected = readerExpected.next();
+ if (!readerActual.hasNext()) {
+ throw TestExecutor.createLineNotFoundException(queryFile, lineExpected, num);
+ }
+ lineActual = readerActual.next();
+
+ if (!planLineEquals(lineExpected, varBaseExpected, lineActual, varBaseActual)) {
+ throw TestExecutor.createLineChangedException(queryFile, lineExpected, lineActual, num);
+ }
+ ++num;
+ }
+ if (readerActual.hasNext()) {
+ throw new Exception(
+ "Result for " + queryFile + " changed at line " + num + ":\n< \n> " + readerActual.next());
+ }
+ }
+
+ private static boolean planLineEquals(String lineExpected, int varIdBaseExpected, String lineActual,
+ int varIdBaseActual) {
+ String lineExpectedNorm = normalizePlanLine(lineExpected, varIdBaseExpected);
+ String lineActualNorm = normalizePlanLine(lineActual, varIdBaseActual);
+ return lineExpectedNorm.equals(lineActualNorm);
+ }
+
+ // rewrite variable ids in given plan line: $$varId -> $$(varId-varIdBase)
+ private static String normalizePlanLine(String line, int varIdBase) {
+ if (varIdBase == Integer.MAX_VALUE) {
+ // plan did not contain any variables -> no rewriting necessary
+ return line;
+ }
+ Matcher m = PATTERN_VAR_ID.matcher(line);
+ StringBuilder sb = new StringBuilder(line.length());
+ while (m.find()) {
+ int varId = Integer.parseInt(m.group(1));
+ int newVarId = varId - varIdBase;
+ m.appendReplacement(sb, PATTERN_VAR_ID_PREFIX + newVarId);
+ }
+ m.appendTail(sb);
+ return sb.toString();
+ }
+
+ private static int findBaseVarId(Collection<String> plan) {
+ int varIdBase = Integer.MAX_VALUE;
+ Matcher m = PATTERN_VAR_ID.matcher("");
+ for (String line : plan) {
+ m.reset(line);
+ while (m.find()) {
+ int varId = Integer.parseInt(m.group(1));
+ varIdBase = Math.min(varIdBase, varId);
+ }
+ }
+ return varIdBase;
+ }
}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmBTreeResourceFactoryProvider.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmBTreeResourceFactoryProvider.java
index 2ee8fcf..fcc8bc6 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmBTreeResourceFactoryProvider.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmBTreeResourceFactoryProvider.java
@@ -86,7 +86,7 @@
filterCmpFactories, filterFields, opTrackerFactory, ioOpCallbackFactory, pageWriteCallbackFactory,
metadataPageManagerFactory, vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties,
true, bloomFilterFields, bloomFilterFalsePositiveRate, index.isPrimaryIndex(), btreeFields,
- bloomFilterFields != null);
+ bloomFilterFields != null, false);
}
private static ITypeTraits[] getTypeTraits(MetadataProvider metadataProvider, Dataset dataset, Index index,
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestPrimaryIndexOperationTrackerFactory.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestPrimaryIndexOperationTrackerFactory.java
index c4390fa..38fdf56 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestPrimaryIndexOperationTrackerFactory.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestPrimaryIndexOperationTrackerFactory.java
@@ -19,7 +19,6 @@
package org.apache.asterix.test.dataflow;
import java.lang.reflect.Field;
-import java.lang.reflect.Modifier;
import java.util.Map;
import org.apache.asterix.common.api.INcApplicationContext;
@@ -70,20 +69,10 @@
}
}
- static void setFinal(Field field, Object obj, Object newValue) throws Exception {
- field.setAccessible(true);
- Field modifiersField = Field.class.getDeclaredField("modifiers");
- modifiersField.setAccessible(true);
- modifiersField.setInt(field, field.getModifiers() & ~Modifier.FINAL);
- field.set(obj, newValue);
- }
-
@SuppressWarnings({ "rawtypes", "unchecked" })
static void replaceMapEntry(Field field, Object obj, Object key, Object value)
throws Exception, IllegalAccessException {
field.setAccessible(true);
- Field modifiersField = Field.class.getDeclaredField("modifiers");
- modifiersField.setAccessible(true);
Map map = (Map) field.get(obj);
map.put(key, value);
}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/jsonplan/JsonLogicalPlanTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/jsonplan/JsonLogicalPlanTest.java
index 65cbb9e..d68586a 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/jsonplan/JsonLogicalPlanTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/jsonplan/JsonLogicalPlanTest.java
@@ -27,8 +27,6 @@
import java.io.StringReader;
import java.nio.charset.StandardCharsets;
import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
import java.util.Map;
import org.apache.asterix.api.java.AsterixJavaClient;
@@ -63,7 +61,7 @@
static {
EXTENSION_RESULT = "plan.json";
- PATH_ACTUAL = "target" + File.separator + "jplantest" + SEPARATOR;
+ PATH_ACTUAL = "target" + SEPARATOR + "jplantest" + SEPARATOR;
}
@Parameters(name = "JsonLogicalPlanTest {index}: {0}")
@@ -71,8 +69,8 @@
return AbstractOptimizerTest.tests();
}
- public JsonLogicalPlanTest(final File queryFile, final File expectedFile, final File actualFile) {
- super(queryFile, expectedFile, actualFile);
+ public JsonLogicalPlanTest(File queryFile, String expectedFilePath, File actualFile) {
+ super(queryFile, actualFile);
}
@Test
@@ -82,7 +80,7 @@
@Override
protected void runAndCompare(String query, ILangCompilationProvider provider, Map<String, IAObject> queryParams,
- IHyracksClientConnection hcc, List<String> linesExpected) throws Exception {
+ IHyracksClientConnection hcc) throws Exception {
FileUtils.writeStringToFile(actualFile, "", StandardCharsets.UTF_8);
String planStr;
try (PrintWriter plan = new PrintWriter(actualFile)) {
@@ -99,7 +97,7 @@
}
BufferedReader readerActual =
- new BufferedReader(new InputStreamReader(new FileInputStream(actualFile), "UTF-8"));
+ new BufferedReader(new InputStreamReader(new FileInputStream(actualFile), StandardCharsets.UTF_8));
String lineActual, objectActual = "";
boolean firstPlan = false;
while ((lineActual = readerActual.readLine()) != null) {
@@ -123,10 +121,4 @@
readerActual.close();
}
}
-
- @Override
- protected List<String> getExpectedLines() {
- // this test only checks the produced result is valid, so no expected results
- return Collections.emptyList();
- }
}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/jsonplan/JsonOptimizedLogicalPlanTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/jsonplan/JsonOptimizedLogicalPlanTest.java
index 7f78743..9bf03ec 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/jsonplan/JsonOptimizedLogicalPlanTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/jsonplan/JsonOptimizedLogicalPlanTest.java
@@ -32,11 +32,11 @@
static {
EXTENSION_RESULT = "plan.json";
optimized = true;
- PATH_ACTUAL = "target" + File.separator + "joptplantest" + SEPARATOR;
+ PATH_ACTUAL = "target" + SEPARATOR + "joptplantest" + SEPARATOR;
}
- public JsonOptimizedLogicalPlanTest(File queryFile, File expectedFile, File actualFile) {
- super(queryFile, expectedFile, actualFile);
+ public JsonOptimizedLogicalPlanTest(File queryFile, String expectedFilePath, File actualFile) {
+ super(queryFile, expectedFilePath, actualFile);
}
@Parameterized.Parameters(name = "JsonOptimizedLogicalPlanTest {index}: {0}")
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/AbstractOptimizerTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/AbstractOptimizerTest.java
index 0fba54b..8126d1e 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/AbstractOptimizerTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/AbstractOptimizerTest.java
@@ -19,12 +19,10 @@
package org.apache.asterix.test.optimizer;
import java.io.File;
-import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
-import java.util.List;
import java.util.Map;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
@@ -68,12 +66,11 @@
protected static final String PATH_BASE =
"src" + SEPARATOR + "test" + SEPARATOR + "resources" + SEPARATOR + "optimizerts" + SEPARATOR;
protected static final String PATH_QUERIES = PATH_BASE + "queries" + SEPARATOR;
- protected static final String PATH_EXPECTED = PATH_BASE + "results" + SEPARATOR;
protected static String PATH_ACTUAL;
protected static final ArrayList<String> ignore = AsterixTestHelper.readTestListFile(FILENAME_IGNORE, PATH_BASE);
protected static final ArrayList<String> only = AsterixTestHelper.readTestListFile(FILENAME_ONLY, PATH_BASE);
- protected static final String TEST_CONFIG_FILE_NAME = "src/main/resources/cc.conf";
+ protected static String TEST_CONFIG_FILE_NAME = "src/main/resources/cc.conf";
protected static final ILangCompilationProvider sqlppCompilationProvider = new SqlppCompilationProvider();
protected static ILangCompilationProvider extensionLangCompilationProvider = null;
protected static IStatementExecutorFactory statementExecutorFactory = new DefaultStatementExecutorFactory();
@@ -123,9 +120,8 @@
}
if (file.isFile() && file.getName().endsWith(EXTENSION_SQLPP)) {
String resultFileName = AsterixTestHelper.extToResExt(file.getName(), EXTENSION_RESULT);
- File expectedFile = new File(PATH_EXPECTED + path + resultFileName);
File actualFile = new File(PATH_ACTUAL + SEPARATOR + path + resultFileName);
- testArgs.add(new Object[] { file, expectedFile, actualFile });
+ testArgs.add(new Object[] { file, path + resultFileName, actualFile });
}
}
@@ -143,20 +139,15 @@
}
protected final File actualFile;
- protected final File expectedFile;
protected final File queryFile;
- public AbstractOptimizerTest(final File queryFile, final File expectedFile, final File actualFile) {
+ public AbstractOptimizerTest(final File queryFile, final File actualFile) {
this.queryFile = queryFile;
- this.expectedFile = expectedFile;
this.actualFile = actualFile;
}
protected abstract void runAndCompare(String query, ILangCompilationProvider provider,
- Map<String, IAObject> queryParams, IHyracksClientConnection hcc, List<String> linesExpected)
- throws Exception;
-
- protected abstract List<String> getExpectedLines() throws IOException;
+ Map<String, IAObject> queryParams, IHyracksClientConnection hcc) throws Exception;
@Test
public void test() throws Exception {
@@ -202,11 +193,10 @@
provider = extensionLangCompilationProvider;
}
IHyracksClientConnection hcc = integrationUtil.getHyracksClientConnection();
- List<String> linesExpected = getExpectedLines();
if (repeat) {
- runAndRepeat(placeholder, substitutions, query, provider, queryParams, hcc, linesExpected);
+ runAndRepeat(placeholder, substitutions, query, provider, queryParams, hcc);
} else {
- runAndCompare(query, provider, queryParams, hcc, linesExpected);
+ runAndCompare(query, provider, queryParams, hcc);
}
LOGGER.info("Test \"" + queryFile.getPath() + "\" PASSED!");
@@ -222,16 +212,16 @@
}
private void runAndRepeat(String placeholder, JsonNode substitutions, String query,
- ILangCompilationProvider provider, Map<String, IAObject> queryParams, IHyracksClientConnection hcc,
- List<String> linesExpected) throws Exception {
+ ILangCompilationProvider provider, Map<String, IAObject> queryParams, IHyracksClientConnection hcc)
+ throws Exception {
if (substitutions.isArray()) {
for (int i = 0, size = substitutions.size(); i < size; i++) {
String substitute = substitutions.get(i).asText();
String newQuery = query.replaceAll(placeholder, substitute);
- runAndCompare(newQuery, provider, queryParams, hcc, linesExpected);
+ runAndCompare(newQuery, provider, queryParams, hcc);
}
} else {
- runAndCompare(query, provider, queryParams, hcc, linesExpected);
+ runAndCompare(query, provider, queryParams, hcc);
}
}
}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/CBOOptimizerTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/CBOOptimizerTest.java
new file mode 100644
index 0000000..a5a99d9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/CBOOptimizerTest.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.test.optimizer;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Collection;
+import java.util.List;
+
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+/**
+ * Runs the optimization tests with CBO enabled. It compares a test's actual result against the expected result in
+ * {@link CBOOptimizerTest#CBO_PATH_EXPECTED} if one is provided. Otherwise, it compares against the expected result in
+ * {@link OptimizerTest#PATH_EXPECTED}.
+ */
+@RunWith(Parameterized.class)
+public class CBOOptimizerTest extends OptimizerTest {
+
+ private static final String CBO_PATH_EXPECTED = PATH_BASE + "results_cbo" + SEPARATOR;
+ static {
+ TEST_CONFIG_FILE_NAME = "src/main/resources/cc.conf";
+ EXTENSION_RESULT = "plan";
+ PATH_ACTUAL = "target" + SEPARATOR + "cbo_opttest" + SEPARATOR;
+ }
+
+ @Parameters(name = "CBOOptimizerTest {index}: {0}")
+ public static Collection<Object[]> tests() {
+ return AbstractOptimizerTest.tests();
+ }
+
+ public CBOOptimizerTest(File queryFile, String expectedFilePath, File actualFile) {
+ super(queryFile, expectedFilePath, actualFile);
+ }
+
+ @Override
+ protected List<String> getExpectedLines() throws IOException {
+ Path cboFilePath = Path.of(CBO_PATH_EXPECTED, expectedFilePath);
+ return Files.exists(cboFilePath) ? Files.readAllLines(cboFilePath, StandardCharsets.UTF_8)
+ : super.getExpectedLines();
+ }
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/OptimizerTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/OptimizerTest.java
index a89585d..6a85e0c 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/OptimizerTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/OptimizerTest.java
@@ -24,17 +24,16 @@
import java.io.StringReader;
import java.nio.charset.StandardCharsets;
import java.nio.file.Files;
+import java.nio.file.Path;
import java.util.Collection;
-import java.util.Iterator;
import java.util.List;
import java.util.Map;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
import org.apache.asterix.api.java.AsterixJavaClient;
import org.apache.asterix.common.dataflow.ICcApplicationContext;
import org.apache.asterix.compiler.provider.ILangCompilationProvider;
import org.apache.asterix.om.base.IAObject;
+import org.apache.asterix.test.common.TestHelper;
import org.apache.commons.io.FileUtils;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.api.client.IHyracksClientConnection;
@@ -51,12 +50,13 @@
@RunWith(Parameterized.class)
public class OptimizerTest extends AbstractOptimizerTest {
- private static final String PATTERN_VAR_ID_PREFIX = "\\$\\$";
- private static final Pattern PATTERN_VAR_ID = Pattern.compile(PATTERN_VAR_ID_PREFIX + "(\\d+)");
+ protected static final String PATH_EXPECTED = PATH_BASE + "results" + SEPARATOR;
+ protected String expectedFilePath;
static {
+ TEST_CONFIG_FILE_NAME = "src/main/resources/cc_no_cbo.conf";
EXTENSION_RESULT = "plan";
- PATH_ACTUAL = "target" + File.separator + "opttest" + SEPARATOR;
+ PATH_ACTUAL = "target" + SEPARATOR + "opttest" + SEPARATOR;
}
@Parameters(name = "OptimizerTest {index}: {0}")
@@ -64,8 +64,9 @@
return AbstractOptimizerTest.tests();
}
- public OptimizerTest(final File queryFile, final File expectedFile, final File actualFile) {
- super(queryFile, expectedFile, actualFile);
+ public OptimizerTest(File queryFile, String expectedFilePath, File actualFile) {
+ super(queryFile, actualFile);
+ this.expectedFilePath = expectedFilePath;
}
@Test
@@ -75,7 +76,7 @@
@Override
protected void runAndCompare(String query, ILangCompilationProvider provider, Map<String, IAObject> queryParams,
- IHyracksClientConnection hcc, List<String> linesExpected) throws Exception {
+ IHyracksClientConnection hcc) throws Exception {
FileUtils.writeStringToFile(actualFile, "", StandardCharsets.UTF_8);
try (PrintWriter plan = new PrintWriter(actualFile)) {
AsterixJavaClient asterix = new AsterixJavaClient(
@@ -88,72 +89,11 @@
}
List<String> linesActual = Files.readAllLines(actualFile.toPath(), StandardCharsets.UTF_8);
-
- int varBaseExpected = findBaseVarId(linesExpected);
- int varBaseActual = findBaseVarId(linesActual);
-
- Iterator<String> readerExpected = linesExpected.iterator();
- Iterator<String> readerActual = linesActual.iterator();
- String lineExpected, lineActual;
- int num = 1;
- while (readerExpected.hasNext()) {
- lineExpected = readerExpected.next();
- if (!readerActual.hasNext()) {
- throw new Exception(
- "Result for " + queryFile + " changed at line " + num + ":\n< " + lineExpected + "\n> ");
- }
- lineActual = readerActual.next();
-
- if (!planLineEquals(lineExpected, varBaseExpected, lineActual, varBaseActual)) {
- throw new Exception("Result for " + queryFile + " changed at line " + num + ":\n< " + lineExpected
- + "\n> " + lineActual);
- }
- ++num;
- }
- if (readerActual.hasNext()) {
- throw new Exception(
- "Result for " + queryFile + " changed at line " + num + ":\n< \n> " + readerActual.next());
- }
+ List<String> linesExpected = getExpectedLines();
+ TestHelper.comparePlans(linesExpected, linesActual, queryFile);
}
- @Override
protected List<String> getExpectedLines() throws IOException {
- return Files.readAllLines(expectedFile.toPath(), StandardCharsets.UTF_8);
- }
-
- private boolean planLineEquals(String lineExpected, int varIdBaseExpected, String lineActual, int varIdBaseActual) {
- String lineExpectedNorm = normalizePlanLine(lineExpected, varIdBaseExpected);
- String lineActualNorm = normalizePlanLine(lineActual, varIdBaseActual);
- return lineExpectedNorm.equals(lineActualNorm);
- }
-
- // rewrite variable ids in given plan line: $$varId -> $$(varId-varIdBase)
- private String normalizePlanLine(String line, int varIdBase) {
- if (varIdBase == Integer.MAX_VALUE) {
- // plan did not contain any variables -> no rewriting necessary
- return line;
- }
- Matcher m = PATTERN_VAR_ID.matcher(line);
- StringBuffer sb = new StringBuffer(line.length());
- while (m.find()) {
- int varId = Integer.parseInt(m.group(1));
- int newVarId = varId - varIdBase;
- m.appendReplacement(sb, PATTERN_VAR_ID_PREFIX + newVarId);
- }
- m.appendTail(sb);
- return sb.toString();
- }
-
- private int findBaseVarId(Collection<String> plan) {
- int varIdBase = Integer.MAX_VALUE;
- Matcher m = PATTERN_VAR_ID.matcher("");
- for (String line : plan) {
- m.reset(line);
- while (m.find()) {
- int varId = Integer.parseInt(m.group(1));
- varIdBase = Math.min(varIdBase, varId);
- }
- }
- return varIdBase;
+ return Files.readAllLines(Path.of(PATH_EXPECTED, expectedFilePath), StandardCharsets.UTF_8);
}
}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/MetricsExecutionTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/MetricsExecutionTest.java
index a973c63..93031c5 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/MetricsExecutionTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/MetricsExecutionTest.java
@@ -35,7 +35,7 @@
*/
@RunWith(Parameterized.class)
public class MetricsExecutionTest {
- protected static final String TEST_CONFIG_FILE_NAME = "src/main/resources/cc.conf";
+ protected static final String TEST_CONFIG_FILE_NAME = "src/main/resources/cc_no_cbo.conf";
@BeforeClass
public static void setUp() throws Exception {
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/RebalanceTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/RebalanceTest.java
index afde4b2..e0d95a5 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/RebalanceTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/RebalanceTest.java
@@ -34,7 +34,7 @@
*/
@RunWith(Parameterized.class)
public class RebalanceTest {
- protected static final String TEST_CONFIG_FILE_NAME = "src/main/resources/cc.conf";
+ protected static final String TEST_CONFIG_FILE_NAME = "src/main/resources/cc_no_cbo.conf";
@BeforeClass
public static void setUp() throws Exception {
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppNumericIndexRQGTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppNumericIndexRQGTest.java
index e97a389..1eac118 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppNumericIndexRQGTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppNumericIndexRQGTest.java
@@ -119,6 +119,7 @@
static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
static final ObjectReader OBJECT_READER = OBJECT_MAPPER.readerFor(ObjectNode.class);
+ static final String TEST_CONFIG_FILE_NAME = "src/main/resources/cc_no_cbo.conf";
static TestExecutor testExecutor;
@@ -127,7 +128,7 @@
@BeforeClass
public static void setUp() throws Exception {
testExecutor = new TestExecutor();
- LangExecutionUtil.setUp(SqlppRQGTestBase.TEST_CONFIG_FILE_NAME, testExecutor, false);
+ LangExecutionUtil.setUp(TEST_CONFIG_FILE_NAME, testExecutor, false);
StringBuilder sb = new StringBuilder(2048);
addDropDataverse(sb, DATAVERSE_NAME);
diff --git a/asterixdb/asterix-app/src/test/resources/cc-analyze.conf b/asterixdb/asterix-app/src/test/resources/cc-analyze.conf
new file mode 100644
index 0000000..d8b3512
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/cc-analyze.conf
@@ -0,0 +1,59 @@
+<<<<<<< HEAD (5f076a [ASTERIXDB-3221][REPL] Use IO scheduler for replication ops)
+=======
+; Licensed to the Apache Software Foundation (ASF) under one
+; or more contributor license agreements. See the NOTICE file
+; distributed with this work for additional information
+; regarding copyright ownership. The ASF licenses this file
+; to you under the Apache License, Version 2.0 (the
+; "License"); you may not use this file except in compliance
+; with the License. You may obtain a copy of the License at
+;
+; http://www.apache.org/licenses/LICENSE-2.0
+;
+; Unless required by applicable law or agreed to in writing,
+; software distributed under the License is distributed on an
+; "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+; KIND, either express or implied. See the License for the
+; specific language governing permissions and limitations
+; under the License.
+
+[nc/asterix_nc1]
+txn.log.dir=target/tmp/asterix_nc1/txnlog
+core.dump.dir=target/tmp/asterix_nc1/coredump
+iodevices=target/tmp/asterix_nc1/iodevice1,
+iodevices=../asterix-server/target/tmp/asterix_nc1/iodevice2
+nc.api.port=19004
+#jvm.args=-agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=5006
+
+[nc/asterix_nc2]
+ncservice.port=9091
+txn.log.dir=target/tmp/asterix_nc2/txnlog
+core.dump.dir=target/tmp/asterix_nc2/coredump
+iodevices=target/tmp/asterix_nc2/iodevice1,../asterix-server/target/tmp/asterix_nc2/iodevice2
+nc.api.port=19005
+#jvm.args=-agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=5007
+
+[nc]
+credential.file=src/test/resources/security/passwd
+python.cmd.autolocate=true
+python.env=FOO=BAR=BAZ,BAR=BAZ
+address=127.0.0.1
+command=asterixnc
+app.class=org.apache.asterix.hyracks.bootstrap.NCApplication
+jvm.args=-Xmx4096m -Dnode.Resolver="org.apache.asterix.external.util.IdentitiyResolverFactory"
+storage.buffercache.size=128MB
+storage.memorycomponent.globalbudget=512MB
+
+[cc]
+address = 127.0.0.1
+app.class=org.apache.asterix.hyracks.bootstrap.CCApplication
+heartbeat.period=2000
+heartbeat.max.misses=25
+credential.file=src/test/resources/security/passwd
+
+[common]
+log.dir = logs/
+log.level = INFO
+compiler.groupmemory=64MB
+storage.buffercache.pagesize=32KB
+>>>>>>> CHANGE (337517 [NO ISSUE][OTH] Move page size storage config to common conf)
diff --git a/asterixdb/asterix-app/src/test/resources/cc-cloud-storage.conf b/asterixdb/asterix-app/src/test/resources/cc-cloud-storage.conf
new file mode 100644
index 0000000..18627b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/cc-cloud-storage.conf
@@ -0,0 +1,67 @@
+<<<<<<< HEAD (5f076a [ASTERIXDB-3221][REPL] Use IO scheduler for replication ops)
+=======
+; Licensed to the Apache Software Foundation (ASF) under one
+; or more contributor license agreements. See the NOTICE file
+; distributed with this work for additional information
+; regarding copyright ownership. The ASF licenses this file
+; to you under the Apache License, Version 2.0 (the
+; "License"); you may not use this file except in compliance
+; with the License. You may obtain a copy of the License at
+;
+; http://www.apache.org/licenses/LICENSE-2.0
+;
+; Unless required by applicable law or agreed to in writing,
+; software distributed under the License is distributed on an
+; "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+; KIND, either express or implied. See the License for the
+; specific language governing permissions and limitations
+; under the License.
+
+[nc/asterix_nc1]
+txn.log.dir=target/tmp/asterix_nc1/txnlog
+core.dump.dir=target/tmp/asterix_nc1/coredump
+iodevices=target/tmp/asterix_nc1/iodevice1,
+iodevices=../asterix-server/target/tmp/asterix_nc1/iodevice2
+nc.api.port=19004
+#jvm.args=-agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=5006
+
+[nc/asterix_nc2]
+ncservice.port=9091
+txn.log.dir=target/tmp/asterix_nc2/txnlog
+core.dump.dir=target/tmp/asterix_nc2/coredump
+iodevices=target/tmp/asterix_nc2/iodevice1,../asterix-server/target/tmp/asterix_nc2/iodevice2
+nc.api.port=19005
+#jvm.args=-agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=5007
+
+[nc]
+credential.file=src/test/resources/security/passwd
+python.cmd.autolocate=true
+python.env=FOO=BAR=BAZ,BAR=BAZ
+address=127.0.0.1
+command=asterixnc
+app.class=org.apache.asterix.hyracks.bootstrap.NCApplication
+jvm.args=-Xmx4096m -Dnode.Resolver="org.apache.asterix.external.util.IdentitiyResolverFactory"
+storage.buffercache.size=128MB
+storage.memorycomponent.globalbudget=512MB
+
+[cc]
+address = 127.0.0.1
+app.class=org.apache.asterix.hyracks.bootstrap.CCApplication
+heartbeat.period=2000
+heartbeat.max.misses=25
+credential.file=src/test/resources/security/passwd
+
+[common]
+log.dir = logs/
+log.level = INFO
+compiler.framesize=32KB
+compiler.sortmemory=320KB
+compiler.groupmemory=160KB
+compiler.joinmemory=256KB
+compiler.textsearchmemory=160KB
+compiler.windowmemory=192KB
+messaging.frame.size=4096
+messaging.frame.count=512
+cloud.deployment=true
+storage.buffercache.pagesize=32KB
+>>>>>>> CHANGE (337517 [NO ISSUE][OTH] Move page size storage config to common conf)
diff --git a/asterixdb/asterix-app/src/test/resources/cc-compression.conf b/asterixdb/asterix-app/src/test/resources/cc-compression.conf
index c8d9780..373d9bb 100644
--- a/asterixdb/asterix-app/src/test/resources/cc-compression.conf
+++ b/asterixdb/asterix-app/src/test/resources/cc-compression.conf
@@ -47,6 +47,8 @@
[common]
log.level = INFO
+compiler.cbo=false
+compiler.cbotest=true
compiler.framesize=32KB
compiler.sortmemory=320KB
compiler.groupmemory=160KB
diff --git a/asterixdb/asterix-app/src/test/resources/cc-multipart.conf b/asterixdb/asterix-app/src/test/resources/cc-multipart.conf
index 5543e51..6c67624 100644
--- a/asterixdb/asterix-app/src/test/resources/cc-multipart.conf
+++ b/asterixdb/asterix-app/src/test/resources/cc-multipart.conf
@@ -35,7 +35,6 @@
command=asterixnc
app.class=org.apache.asterix.hyracks.bootstrap.NCApplication
jvm.args=-Xmx4096m -Dnode.Resolver="org.apache.asterix.external.util.IdentitiyResolverFactory"
-storage.buffercache.pagesize=32KB
storage.buffercache.size=128MB
storage.memorycomponent.globalbudget=512MB
@@ -54,3 +53,4 @@
compiler.windowmemory=192KB
messaging.frame.size=4096
messaging.frame.count=512
+storage.buffercache.pagesize=32KB
diff --git a/asterixdb/asterix-app/src/test/resources/cc-single-lookup.conf b/asterixdb/asterix-app/src/test/resources/cc-single-lookup.conf
new file mode 100644
index 0000000..1b006b3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/cc-single-lookup.conf
@@ -0,0 +1,67 @@
+<<<<<<< HEAD (5f076a [ASTERIXDB-3221][REPL] Use IO scheduler for replication ops)
+=======
+; Licensed to the Apache Software Foundation (ASF) under one
+; or more contributor license agreements. See the NOTICE file
+; distributed with this work for additional information
+; regarding copyright ownership. The ASF licenses this file
+; to you under the Apache License, Version 2.0 (the
+; "License"); you may not use this file except in compliance
+; with the License. You may obtain a copy of the License at
+;
+; http://www.apache.org/licenses/LICENSE-2.0
+;
+; Unless required by applicable law or agreed to in writing,
+; software distributed under the License is distributed on an
+; "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+; KIND, either express or implied. See the License for the
+; specific language governing permissions and limitations
+; under the License.
+
+[nc/asterix_nc1]
+txn.log.dir=target/tmp/asterix_nc1/txnlog
+core.dump.dir=target/tmp/asterix_nc1/coredump
+iodevices=target/tmp/asterix_nc1/iodevice1,
+iodevices=../asterix-server/target/tmp/asterix_nc1/iodevice2
+nc.api.port=19004
+#jvm.args=-agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=5006
+
+[nc/asterix_nc2]
+ncservice.port=9091
+txn.log.dir=target/tmp/asterix_nc2/txnlog
+core.dump.dir=target/tmp/asterix_nc2/coredump
+iodevices=target/tmp/asterix_nc2/iodevice1,../asterix-server/target/tmp/asterix_nc2/iodevice2
+nc.api.port=19005
+#jvm.args=-agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=5007
+
+[nc]
+credential.file=src/test/resources/security/passwd
+python.cmd.autolocate=true
+python.env=FOO=BAR=BAZ,BAR=BAZ
+address=127.0.0.1
+command=asterixnc
+app.class=org.apache.asterix.hyracks.bootstrap.NCApplication
+jvm.args=-Xmx4096m -Dnode.Resolver="org.apache.asterix.external.util.IdentitiyResolverFactory"
+storage.buffercache.size=128MB
+storage.memorycomponent.globalbudget=512MB
+
+[cc]
+address = 127.0.0.1
+app.class=org.apache.asterix.hyracks.bootstrap.CCApplication
+heartbeat.period=2000
+heartbeat.max.misses=25
+credential.file=src/test/resources/security/passwd
+
+[common]
+log.dir = logs/
+log.level = INFO
+compiler.framesize=32KB
+compiler.sortmemory=320KB
+compiler.groupmemory=160KB
+compiler.joinmemory=256KB
+compiler.textsearchmemory=160KB
+compiler.windowmemory=192KB
+compiler.batch.lookup=false
+messaging.frame.size=4096
+messaging.frame.count=512
+storage.buffercache.pagesize=32KB
+>>>>>>> CHANGE (337517 [NO ISSUE][OTH] Move page size storage config to common conf)
diff --git a/asterixdb/asterix-app/src/test/resources/cc-single.conf b/asterixdb/asterix-app/src/test/resources/cc-single.conf
index 794b5b0..a6f4e89 100644
--- a/asterixdb/asterix-app/src/test/resources/cc-single.conf
+++ b/asterixdb/asterix-app/src/test/resources/cc-single.conf
@@ -27,7 +27,6 @@
command=asterixnc
app.class=org.apache.asterix.hyracks.bootstrap.NCApplication
jvm.args=-Xmx4096m -Dnode.Resolver="org.apache.asterix.external.util.IdentitiyResolverFactory"
-storage.buffercache.pagesize=32KB
storage.buffercache.size=128MB
storage.memorycomponent.globalbudget=512MB
@@ -48,3 +47,4 @@
compiler.windowmemory=192KB
messaging.frame.size=4096
messaging.frame.count=512
+storage.buffercache.pagesize=32KB
diff --git a/asterixdb/asterix-app/src/test/resources/cc-small-txn-log-partition.conf b/asterixdb/asterix-app/src/test/resources/cc-small-txn-log-partition.conf
index dd8edb4..ae91f5e 100644
--- a/asterixdb/asterix-app/src/test/resources/cc-small-txn-log-partition.conf
+++ b/asterixdb/asterix-app/src/test/resources/cc-small-txn-log-partition.conf
@@ -33,7 +33,6 @@
command=asterixnc
app.class=org.apache.asterix.hyracks.bootstrap.NCApplication
jvm.args=-Xmx4096m -Dnode.Resolver="org.apache.asterix.external.util.IdentitiyResolverFactory"
-storage.buffercache.pagesize=32KB
storage.buffercache.size=128MB
storage.memorycomponent.globalbudget=128MB
@@ -56,3 +55,4 @@
txn.log.buffer.pagesize=128KB
txn.log.checkpoint.pollfrequency=2147483647
txn.log.checkpoint.history=0
+storage.buffercache.pagesize=32KB
diff --git a/asterixdb/asterix-app/src/test/resources/cc-ssl.conf b/asterixdb/asterix-app/src/test/resources/cc-ssl.conf
index 499e9fc..1498244 100644
--- a/asterixdb/asterix-app/src/test/resources/cc-ssl.conf
+++ b/asterixdb/asterix-app/src/test/resources/cc-ssl.conf
@@ -43,7 +43,6 @@
command=asterixnc
app.class=org.apache.asterix.hyracks.bootstrap.NCApplication
jvm.args=-Xmx4096m -Dnode.Resolver="org.apache.asterix.external.util.IdentitiyResolverFactory"
-storage.buffercache.pagesize=32KB
storage.buffercache.size=128MB
storage.memorycomponent.globalbudget=512MB
@@ -59,6 +58,8 @@
[common]
log.dir = logs/
log.level = INFO
+compiler.cbo=false
+compiler.cbotest=true
compiler.framesize=32KB
compiler.sortmemory=320KB
compiler.groupmemory=160KB
@@ -67,4 +68,5 @@
compiler.windowmemory=192KB
messaging.frame.size=4096
messaging.frame.count=512
-ssl.enabled=true
\ No newline at end of file
+ssl.enabled=true
+storage.buffercache.pagesize=32KB
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/cc-storage.conf b/asterixdb/asterix-app/src/test/resources/cc-storage.conf
index 9e869c7..68faa17 100644
--- a/asterixdb/asterix-app/src/test/resources/cc-storage.conf
+++ b/asterixdb/asterix-app/src/test/resources/cc-storage.conf
@@ -33,7 +33,6 @@
command=asterixnc
app.class=org.apache.asterix.hyracks.bootstrap.NCApplication
jvm.args=-Xmx4096m -Dnode.Resolver="org.apache.asterix.external.util.IdentitiyResolverFactory"
-storage.buffercache.pagesize=32KB
storage.buffercache.size=128MB
storage.memorycomponent.globalbudget=512MB
@@ -53,6 +52,7 @@
compiler.windowmemory=192KB
messaging.frame.size=4096
messaging.frame.count=512
+storage.buffercache.pagesize=32KB
storage.compression.block=snappy
txn.log.checkpoint.pollfrequency=10
txn.dataset.checkpoint.interval=10
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/cc.conf b/asterixdb/asterix-app/src/test/resources/cc.conf
index 953284964..624455c 100644
--- a/asterixdb/asterix-app/src/test/resources/cc.conf
+++ b/asterixdb/asterix-app/src/test/resources/cc.conf
@@ -39,7 +39,6 @@
command=asterixnc
app.class=org.apache.asterix.hyracks.bootstrap.NCApplication
jvm.args=-Xmx4096m -Dnode.Resolver="org.apache.asterix.external.util.IdentitiyResolverFactory"
-storage.buffercache.pagesize=32KB
storage.buffercache.size=128MB
storage.memorycomponent.globalbudget=512MB
@@ -61,3 +60,4 @@
compiler.windowmemory=192KB
messaging.frame.size=4096
messaging.frame.count=512
+storage.buffercache.pagesize=32KB
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-indexes/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-indexes/query1.sqlpp
new file mode 100644
index 0000000..0412602
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-indexes/query1.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * 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;
+
+CREATE TYPE GenericType AS { _id: uuid };
+CREATE DATASET KSI (GenericType)
+PRIMARY KEY _id AUTOGENERATED;
+
+CREATE INDEX KS1_array_index1 ON KSI (
+ UNNEST uarr_i
+ SELECT a : bigint
+) EXCLUDE UNKNOWN KEY;
+CREATE INDEX KS1_array_index2 ON KSI (
+ UNNEST uarr_i
+ SELECT b : bigint
+) EXCLUDE UNKNOWN KEY;
+
+FROM KSI k
+WHERE (ANY v IN k.uarr_i SATISFIES v.a = 284 AND
+ v.b = 263)
+SELECT *;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-indexes/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-indexes/query2.sqlpp
new file mode 100644
index 0000000..2f2bb8a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-indexes/query2.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE TYPE GenericType AS { _id: uuid };
+CREATE DATASET KSI (GenericType)
+PRIMARY KEY _id AUTOGENERATED;
+
+CREATE INDEX KS1_array_index1 ON KSI (
+ UNNEST uarr_i
+ SELECT a : bigint
+) EXCLUDE UNKNOWN KEY;
+CREATE INDEX KS1_array_index2 ON KSI (
+ UNNEST uarr_i
+ SELECT b : bigint
+) EXCLUDE UNKNOWN KEY;
+CREATE INDEX KS1_array_index3 ON KSI (
+ UNNEST uarr_i
+ SELECT c : bigint
+) EXCLUDE UNKNOWN KEY;
+
+FROM KSI k
+WHERE (ANY v IN k.uarr_i SATISFIES v.a = 284 AND
+ v.b = 263 AND
+ v.c = 123 )
+SELECT *;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-quantifiers/query10.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-quantifiers/query10.sqlpp
new file mode 100644
index 0000000..45a00b8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-quantifiers/query10.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestDataverse IF EXISTS;
+CREATE DATAVERSE TestDataverse;
+USE TestDataverse;
+
+CREATE TYPE TestType AS {
+ _id: uuid
+};
+CREATE DATASET Dataset1 (TestType)
+PRIMARY KEY _id AUTOGENERATED;
+CREATE INDEX d1Idx
+ON Dataset1 ( UNNEST items: string )
+EXCLUDE UNKNOWN KEY;
+
+-- Quantification in subplan, open index.
+FROM Dataset1 D1
+WHERE TRUE IN [ D1.val = "a", "b" IN D1.items ]
+SELECT *;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-quantifiers/query11.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-quantifiers/query11.sqlpp
new file mode 100644
index 0000000..adc5607
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-quantifiers/query11.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestDataverse IF EXISTS;
+CREATE DATAVERSE TestDataverse;
+USE TestDataverse;
+
+CREATE TYPE TestType AS {
+ _id: uuid
+};
+CREATE DATASET Dataset1 (TestType)
+PRIMARY KEY _id AUTOGENERATED;
+CREATE INDEX d1Idx
+ON Dataset1 ( UNNEST items: string )
+EXCLUDE UNKNOWN KEY;
+
+-- Quantification nested in two subplans, open index.
+FROM Dataset1 D1
+WHERE TRUE IN [ D1.val = "a", TRUE IN [ D1.val2 = "c", "b" IN D1.items ] ]
+SELECT *;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/multiple-indexes/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/multiple-indexes/query1.sqlpp
new file mode 100644
index 0000000..687ac95
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/multiple-indexes/query1.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.
+ */
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE TYPE GenericType AS { _id: uuid };
+CREATE DATASET KSI (GenericType)
+PRIMARY KEY _id AUTOGENERATED;
+
+CREATE INDEX KS1_array_index1 ON KSI (
+ UNNEST uarr_i
+ SELECT a : bigint
+) EXCLUDE UNKNOWN KEY;
+CREATE INDEX KS1_array_index2 ON KSI (
+ UNNEST uarr_i
+ SELECT b : bigint
+) EXCLUDE UNKNOWN KEY;
+
+FROM KSI k
+UNNEST k.uarr_i v
+WHERE v.a = 284 AND
+ v.b = 263
+SELECT *;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/multiple-indexes/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/multiple-indexes/query2.sqlpp
new file mode 100644
index 0000000..b1d7640
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/multiple-indexes/query2.sqlpp
@@ -0,0 +1,45 @@
+/*
+ * 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;
+
+CREATE TYPE GenericType AS { _id: uuid };
+CREATE DATASET KSI (GenericType)
+PRIMARY KEY _id AUTOGENERATED;
+
+CREATE INDEX KS1_array_index1 ON KSI (
+ UNNEST uarr_i
+ SELECT a : bigint
+) EXCLUDE UNKNOWN KEY;
+CREATE INDEX KS1_array_index2 ON KSI (
+ UNNEST uarr_i
+ SELECT b : bigint
+) EXCLUDE UNKNOWN KEY;
+CREATE INDEX KS1_array_index3 ON KSI (
+ UNNEST uarr_i
+ SELECT c : bigint
+) EXCLUDE UNKNOWN KEY;
+
+FROM KSI k
+UNNEST k.uarr_i v
+WHERE v.a = 284 AND
+ v.b = 263 AND
+ v.c = 123
+SELECT *;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-25.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-25.sqlpp
new file mode 100644
index 0000000..9c28038
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-25.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.
+ */
+
+// repeat_exec: %with% = ["", "WITH {'merge-policy': {'name': 'correlated-prefix','parameters': { 'max-mergable-component-size': 16384, 'max-tolerance-component-count': 3 }}}" ]
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+// Similar to cast-default-null-20.sqlpp but with s_f2 being not optional
+CREATE TYPE t1 AS { id: int, s_f1: string, s_f2: string, i_f: int? };
+CREATE DATASET ds7(t1) PRIMARY KEY id %with%;
+
+CREATE INDEX idx1 ON ds7(s_f2);
+CREATE INDEX idx2 ON ds7(s_f2: string) CAST (DEFAULT NULL);
+CREATE INDEX idx3 ON ds7(s_f2: int) CAST (DEFAULT NULL);
+
+CREATE VIEW view7_1(id int, s_f2 string, i_f int) DEFAULT NULL AS SELECT id, s_f2, i_f FROM ds7;
+CREATE VIEW view7_2(id int, s_f2 int, i_f int) DEFAULT NULL AS SELECT id, s_f2, i_f FROM ds7;
+
+USE test;
+// index idx2 should (still) be used
+SELECT id, s_f2 FROM view7_1 WHERE s_f2 < "4" ORDER BY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/hints-use-index/hints-use-index-20.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/hints-use-index/hints-use-index-20.sqlpp
new file mode 100644
index 0000000..6ced081
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/hints-use-index/hints-use-index-20.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create dataset orders(id uuid not unknown) open type primary key `id` autogenerated;
+CREATE INDEX orderline_delivery_d ON orders(UNNEST o_orderline SELECT ol_delivery_d:STRING) EXCLUDE UNKNOWN KEY;
+
+/* Test with use-index hint on both arguments of AND condition, use-index hint is honored. */
+SELECT count(*) as revenue
+FROM orders o, o.o_orderline ol
+WHERE ol.ol_delivery_d /*+ use-index (orderline_delivery_d) */ >= '2016-01-01 00:00:00.000000'
+ AND ol.ol_delivery_d /*+ use-index (orderline_delivery_d) */ < '2017-01-01 00:00:00.000000';
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/hints-use-index/hints-use-index-21.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/hints-use-index/hints-use-index-21.sqlpp
new file mode 100644
index 0000000..c925b06
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/hints-use-index/hints-use-index-21.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create dataset orders(id uuid not unknown) open type primary key `id` autogenerated;
+CREATE INDEX orderline_delivery_d ON orders(UNNEST o_orderline SELECT ol_delivery_d:STRING) EXCLUDE UNKNOWN KEY;
+
+/* Test with conflicting use-index and skip-index hint on arguments of AND condition.
+ use-index wins and is honored */
+SELECT count(*) as revenue
+FROM orders o, o.o_orderline ol
+WHERE ol.ol_delivery_d /*+ use-index (orderline_delivery_d) */ >= '2016-01-01 00:00:00.000000'
+ AND ol.ol_delivery_d /*+ skip-index (orderline_delivery_d) */ < '2017-01-01 00:00:00.000000';
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-ternary-inlj/query4.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-ternary-inlj/query4.sqlpp
new file mode 100644
index 0000000..fae9ea4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-ternary-inlj/query4.sqlpp
@@ -0,0 +1,194 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: ASTERIXDB-3043. Part of q18 of tpcds.
+ */
+
+DROP DATAVERSE tpcds IF EXISTS;
+CREATE DATAVERSE tpcds;
+
+USE tpcds;
+
+CREATE TYPE tpcds.catalog_sales_type AS
+ CLOSED {
+ cs_sold_date_sk: bigint?,
+ cs_sold_time_sk: bigint?,
+ cs_ship_date_sk: bigint?,
+ cs_bill_customer_sk: bigint?,
+ cs_bill_cdemo_sk: bigint?,
+ cs_bill_hdemo_sk: bigint?,
+ cs_bill_addr_sk: bigint?,
+ cs_ship_customer_sk: bigint?,
+ cs_ship_cdemo_sk: bigint?,
+ cs_ship_hdemo_sk: bigint?,
+ cs_ship_addr_sk: bigint?,
+ cs_call_center_sk: bigint?,
+ cs_catalog_page_sk: bigint?,
+ cs_ship_mode_sk: bigint?,
+ cs_warehouse_sk: bigint?,
+ cs_item_sk: bigint,
+ cs_promo_sk: bigint?,
+ cs_order_number: bigint,
+ cs_quantity: bigint?,
+ cs_wholesale_cost: double?,
+ cs_list_price: double?,
+ cs_sales_price: double?,
+ cs_ext_discount_amt: double?,
+ cs_ext_sales_price: double?,
+ cs_ext_wholesale_cost: double?,
+ cs_ext_list_price: double?,
+ cs_ext_tax: double?,
+ cs_coupon_amt: double?,
+ cs_ext_ship_cost: double?,
+ cs_net_paid: double?,
+ cs_net_paid_inc_tax: double?,
+ cs_net_paid_inc_ship: double?,
+ cs_net_paid_inc_ship_tax: double?,
+ cs_net_profit: double?
+};
+
+CREATE TYPE tpcds.customer_demographics_type AS
+ CLOSED {
+ cd_demo_sk : int64,
+ cd_gender : string?,
+ cd_marital_status : string?,
+ cd_education_status : string?,
+ cd_purchase_estimate : int64?,
+ cd_credit_rating : string?,
+ cd_dep_count : int64?,
+ cd_dep_employed_count : int64?,
+ cd_dep_college_count : int64?
+};
+
+CREATE TYPE tpcds.customer_type AS
+ CLOSED {
+ c_customer_sk : int64,
+ c_customer_id : string,
+ c_current_cdemo_sk : int64?,
+ c_current_hdemo_sk : int64?,
+ c_current_addr_sk : int64?,
+ c_first_shipto_date_sk : int64?,
+ c_first_sales_date_sk : int64?,
+ c_salutation : string?,
+ c_first_name : string?,
+ c_last_name : string?,
+ c_preferred_cust_flag : string?,
+ c_birth_day : int64?,
+ c_birth_month : int64?,
+ c_birth_year : int64?,
+ c_birth_country : string?,
+ c_login : string?,
+ c_email_address : string?,
+ c_last_review_date : string?
+};
+
+CREATE TYPE tpcds.customer_address_type AS
+ CLOSED {
+ ca_address_sk : bigint,
+ ca_address_id : string,
+ ca_street_number : string?,
+ ca_street_name : string?,
+ ca_street_type : string?,
+ ca_suite_number : string?,
+ ca_city : string?,
+ ca_county : string?,
+ ca_state : string?,
+ ca_zip : string?,
+ ca_country : string?,
+ ca_gmt_offset : double?,
+ ca_location_type : string?
+ };
+
+CREATE TYPE tpcds.item_type AS
+ CLOSED {
+ i_item_sk : bigint,
+ i_item_id : string,
+ i_rec_start_date : string?,
+ i_rec_end_date : string?,
+ i_item_desc : string?,
+ i_current_price : double?,
+ i_wholesale_cost : double?,
+ i_brand_id : bigint?,
+ i_brand : string?,
+ i_class_id : bigint?,
+ i_class : string?,
+ i_category_id : bigint?,
+ i_category : string?,
+ i_manufact_id : bigint?,
+ i_manufact : string?,
+ i_size : string?,
+ i_formulation : string?,
+ i_color : string?,
+ i_units : string?,
+ i_container : string?,
+ i_manager_id : bigint?,
+ i_product_name : string?
+};
+
+CREATE TYPE tpcds.date_dim_type AS
+ CLOSED {
+ d_date_sk : bigint,
+ d_date_id : string,
+ d_date : string?,
+ d_month_seq : bigint?,
+ d_week_seq : bigint?,
+ d_quarter_seq : bigint?,
+ d_year : bigint? ,
+ d_dow : bigint? ,
+ d_moy : bigint?,
+ d_dom : bigint?,
+ d_qoy : bigint?,
+ d_fy_year : bigint?,
+ d_fy_quarter_seq : bigint?,
+ d_fy_week_seq : bigint?,
+ d_day_name : string?,
+ d_quarter_name : string?,
+ d_holiday : string?,
+ d_weekend : string?,
+ d_following_holiday : string?,
+ d_first_dom : bigint?,
+ d_last_dom : bigint?,
+ d_same_day_ly : bigint?,
+ d_same_day_lq : bigint?,
+ d_current_day : string?,
+ d_current_week : string?,
+ d_current_month : string?,
+ d_current_quarter : string?,
+ d_current_year : string?
+};
+
+CREATE DATASET catalog_sales(catalog_sales_type) PRIMARY KEY cs_item_sk, cs_order_number;
+
+CREATE DATASET customer_demographics(customer_demographics_type) PRIMARY KEY cd_demo_sk;
+
+CREATE DATASET customer(customer_type) PRIMARY KEY c_customer_sk;
+
+CREATE DATASET customer_address(customer_address_type) PRIMARY KEY ca_address_sk;
+
+CREATE DATASET item(item_type) PRIMARY KEY i_item_sk;
+
+CREATE DATASET date_dim(date_dim_type) PRIMARY KEY d_date_sk;
+
+SELECT count (*)
+FROM customer c, customer_demographics cd2, customer_address ca
+WHERE
+ c.c_current_cdemo_sk /*+ indexnl */ = cd2.cd_demo_sk
+ AND c.c_current_addr_sk /*+ indexnl */= ca.ca_address_sk
+ AND c.c_birth_month in [4,5];
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/external-cross-product.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/external-cross-product.sqlpp
new file mode 100644
index 0000000..ec96012
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/external-cross-product.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;
+
+CREATE TYPE OpenType AS {
+};
+
+CREATE EXTERNAL DATASET Orders(OpenType) USING localfs
+(
+ ("path"="asterix_nc1://data/json/double-150-11.json"),
+ ("format"="json")
+);
+
+SELECT COUNT(*)
+FROM Orders o1, Orders o2, Orders o3;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/extract-common-operators/extract-common-operators.01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/extract-common-operators/extract-common-operators.01.sqlpp
new file mode 100644
index 0000000..11c7325
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/extract-common-operators/extract-common-operators.01.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * test fix for ASTERIXDB-3075
+ */
+
+LET dv_name = ["test2", "test1"], ds_name = ["ds2", "ds1"], syn_name = [ "syn2", "syn1" ],
+synonym_names = (SELECT s.SynonymName, s.ObjectName
+ FROM Metadata.`Synonym` s, syn_name
+ WHERE s.SynonymName = syn_name),
+
+dataset_ds_dv_names = (SELECT d.DatasetName, d.DataverseName
+ FROM Metadata.`Dataset` d, (SELECT * FROM ds_name, dv_name) AS ds_dv_names
+ WHERE d.DatasetName = ds_dv_names.ds_name AND d.DataverseName = ds_dv_names.dv_name),
+
+dataset_dv_ds_names = (SELECT d.DatasetName, d.DataverseName
+ FROM Metadata.`Dataset` d, (SELECT * FROM dv_name, ds_name) AS dv_ds_names
+ WHERE d.DatasetName = dv_ds_names.ds_name AND d.DataverseName = dv_ds_names.dv_name),
+
+left_branch = (SELECT s.SynonymName, s.DataverseName, s.ObjectName
+ FROM Metadata.`Synonym` s LEFT OUTER JOIN dataset_dv_ds_names
+ ON dataset_dv_ds_names.DatasetName = s.ObjectName
+ ORDER BY s.DataverseName, s.SynonymName),
+
+right_branch = (SELECT synonym_names.SynonymName
+ FROM synonym_names LEFT OUTER JOIN dataset_ds_dv_names
+ ON dataset_ds_dv_names.DatasetName = synonym_names.ObjectName)
+
+FROM left_branch lb LEFT OUTER JOIN right_branch rb
+ON lb.ObjectName = rb.SynonymName
+SELECT lb.DataverseName, lb.SynonymName
+ORDER BY lb.DataverseName, lb.SynonymName;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/hash-join-with-redundant-variable/hash-join-with-redundant-variable.1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/hash-join-with-redundant-variable/hash-join-with-redundant-variable.1.sqlpp
new file mode 100644
index 0000000..2ed73fb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/hash-join-with-redundant-variable/hash-join-with-redundant-variable.1.sqlpp
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+
+USE test;
+
+CREATE TYPE OpenType AS {
+ uid: uuid
+};
+
+CREATE DATASET region(OpenType)
+PRIMARY KEY uid AUTOGENERATED;
+
+CREATE DATASET nation(OpenType)
+PRIMARY KEY uid AUTOGENERATED;
+
+CREATE DATASET customer(OpenType)
+PRIMARY KEY uid AUTOGENERATED;
+
+
+CREATE DATASET orders(OpenType)
+PRIMARY KEY uid AUTOGENERATED;
+
+
+CREATE DATASET lineitem(OpenType)
+PRIMARY KEY uid AUTOGENERATED;
+
+
+CREATE DATASET supplier(OpenType)
+PRIMARY KEY uid AUTOGENERATED;
+
+SELECT VALUE COUNT(*)
+FROM region r, nation n, customer c, orders o, lineitem l, supplier s
+WHERE c.c_custkey = o.o_custkey
+ AND l.l_orderkey = o.o_orderkey
+ AND l.l_suppkey = s.s_suppkey
+ AND s.s_nationkey = n.n_nationkey
+ AND c.c_nationkey = n.n_nationkey
+ -- this predicate should be eliminated as it is implicitly
+ -- inferred by the two predicates above
+ AND c.c_nationkey = s.s_nationkey
+ AND n.n_regionkey = r.r_regionkey
+ AND r.r_name = 'EUROPE'
+ AND o.o_orderdate >= '1993-01-01'
+ AND o.o_orderdate < '1993-04-01';
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/hash-join-with-redundant-variable/hash-join-with-redundant-variable.2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/hash-join-with-redundant-variable/hash-join-with-redundant-variable.2.sqlpp
new file mode 100644
index 0000000..f2c56ad
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/hash-join-with-redundant-variable/hash-join-with-redundant-variable.2.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+SELECT VALUE COUNT(*)
+FROM region r, orders o, nation n
+WHERE o.x = r.x
+ AND r.x = n.x
+ -- this predicate should be eliminated as it is implicitly
+ -- inferred by the two predicates above
+ AND o.x = n.x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/hash-join-with-redundant-variable/hash-join-with-redundant-variable.3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/hash-join-with-redundant-variable/hash-join-with-redundant-variable.3.sqlpp
new file mode 100644
index 0000000..78d7c90
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/hash-join-with-redundant-variable/hash-join-with-redundant-variable.3.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.
+ */
+
+USE test;
+
+SELECT VALUE COUNT(*)
+FROM region r, nation n, orders o
+WHERE o.x = r.x
+ AND r.x = n.x
+ -- this predicate should be eliminated as it is implicitly
+ -- inferred by the two predicates above
+ AND o.x = n.x
+ -- a redundant predicate (i.e., to o.x = r.x)
+ AND r.x = o.x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/hash-join-with-redundant-variable/hash-join-with-redundant-variable.4.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/hash-join-with-redundant-variable/hash-join-with-redundant-variable.4.sqlpp
new file mode 100644
index 0000000..09bda7e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/hash-join-with-redundant-variable/hash-join-with-redundant-variable.4.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.
+ */
+
+USE test;
+
+SELECT VALUE COUNT(*)
+FROM region r, orders o
+WHERE r.x = o.x
+ -- r.x should be also be assigned to a new variable (a copy variable)
+ AND r.x = o.y;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/hints/broadcast_hint_1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/hints/broadcast_join_hint/broadcast_join_hint_1.sqlpp
similarity index 100%
rename from asterixdb/asterix-app/src/test/resources/optimizerts/queries/hints/broadcast_hint_1.sqlpp
rename to asterixdb/asterix-app/src/test/resources/optimizerts/queries/hints/broadcast_join_hint/broadcast_join_hint_1.sqlpp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/hints/broadcast_hint_2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/hints/broadcast_join_hint/broadcast_join_hint_2.sqlpp
similarity index 100%
rename from asterixdb/asterix-app/src/test/resources/optimizerts/queries/hints/broadcast_hint_2.sqlpp
rename to asterixdb/asterix-app/src/test/resources/optimizerts/queries/hints/broadcast_join_hint/broadcast_join_hint_2.sqlpp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/hints/broadcast_hint_3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/hints/broadcast_join_hint/broadcast_join_hint_3.sqlpp
similarity index 100%
rename from asterixdb/asterix-app/src/test/resources/optimizerts/queries/hints/broadcast_hint_3.sqlpp
rename to asterixdb/asterix-app/src/test/resources/optimizerts/queries/hints/broadcast_join_hint/broadcast_join_hint_3.sqlpp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/hints/broadcast_join_hint/broadcast_join_hint_4.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/hints/broadcast_join_hint/broadcast_join_hint_4.sqlpp
new file mode 100644
index 0000000..39a04ec
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/hints/broadcast_join_hint/broadcast_join_hint_4.sqlpp
@@ -0,0 +1,58 @@
+/*
+ * 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 enhanced broadcast join hint
+* /*+ hash-bcast */
+* Expected Res : Success
+* Date : 11/20/2022
+*/
+
+DROP dataverse tpch IF EXISTS;
+CREATE dataverse tpch;
+
+USE tpch;
+
+CREATE TYPE OrderType AS {
+ o_orderkey : integer
+};
+
+CREATE TYPE CustomerType AS {
+ c_custkey : integer
+};
+
+CREATE TYPE NationType AS {
+ n_nationkey : integer
+};
+
+CREATE DATASET orders(OrderType) PRIMARY KEY o_orderkey;
+
+CREATE DATASET customer(CustomerType) PRIMARY KEY c_custkey;
+
+CREATE DATASET nation(NationType) PRIMARY KEY n_nationkey;
+
+SELECT COUNT(*)
+FROM orders o,
+ (
+ SELECT *
+ FROM customer c,
+ nation n
+ WHERE c.c_nationkey = n.n_nationkey) cn
+WHERE cn.c_custkey /*+ hash-bcast */ = o.o_custkey;
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/hints/broadcast_join_hint/broadcast_join_hint_5.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/hints/broadcast_join_hint/broadcast_join_hint_5.sqlpp
new file mode 100644
index 0000000..e6d50ca
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/hints/broadcast_join_hint/broadcast_join_hint_5.sqlpp
@@ -0,0 +1,58 @@
+/*
+ * 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 enhanced broadcast join hint
+* /*+ hash-bcast (<collection_name>) */
+* Expected Res : Success
+* Date : 11/20/2022
+*/
+
+DROP dataverse tpch IF EXISTS;
+CREATE dataverse tpch;
+
+USE tpch;
+
+CREATE TYPE OrderType AS {
+ o_orderkey : integer
+};
+
+CREATE TYPE CustomerType AS {
+ c_custkey : integer
+};
+
+CREATE TYPE NationType AS {
+ n_nationkey : integer
+};
+
+CREATE DATASET orders(OrderType) PRIMARY KEY o_orderkey;
+
+CREATE DATASET customer(CustomerType) PRIMARY KEY c_custkey;
+
+CREATE DATASET nation(NationType) PRIMARY KEY n_nationkey;
+
+SELECT COUNT(*)
+FROM orders o,
+ (
+ SELECT *
+ FROM customer c,
+ nation n
+ WHERE c.c_nationkey = n.n_nationkey) cn
+WHERE cn.c_custkey /*+ hash-bcast (o) */ = o.o_custkey;
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/hints/broadcast_join_hint/broadcast_join_hint_6.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/hints/broadcast_join_hint/broadcast_join_hint_6.sqlpp
new file mode 100644
index 0000000..bf700cd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/hints/broadcast_join_hint/broadcast_join_hint_6.sqlpp
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description : Test enhanced broadcast join hint
+* /*+ hash-bcast */
+* /*+ hash-bcast (<collection_name>) */
+* Expected Res : Success
+* Date : 11/20/2022
+*/
+
+DROP dataverse tpch IF EXISTS;
+CREATE dataverse tpch;
+
+USE tpch;
+
+CREATE TYPE OrderType AS {
+ o_orderkey : integer
+};
+
+CREATE TYPE CustomerType AS {
+ c_custkey : integer
+};
+
+CREATE TYPE NationType AS {
+ n_nationkey : integer
+};
+
+CREATE DATASET orders(OrderType) PRIMARY KEY o_orderkey;
+
+CREATE DATASET customer(CustomerType) PRIMARY KEY c_custkey;
+
+CREATE DATASET nation(NationType) PRIMARY KEY n_nationkey;
+
+SELECT COUNT(*)
+FROM orders o,
+ (
+ SELECT *
+ FROM customer c,
+ nation n
+ WHERE c.c_nationkey = n.n_nationkey) cn
+WHERE cn.c_custkey /*+ hash-bcast (c) */ = o.o_custkey;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/hints/broadcast_join_hint/broadcast_join_hint_7.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/hints/broadcast_join_hint/broadcast_join_hint_7.sqlpp
new file mode 100644
index 0000000..ab9c099
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/hints/broadcast_join_hint/broadcast_join_hint_7.sqlpp
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description : Test enhanced broadcast join hint
+* /*+ hash-bcast */
+* /*+ hash-bcast (<collection_name>) */
+* Expected Res : Success
+* Date : 11/20/2022
+*/
+
+DROP dataverse tpch IF EXISTS;
+CREATE dataverse tpch;
+
+USE tpch;
+
+CREATE TYPE OrderType AS {
+ o_orderkey : integer
+};
+
+CREATE TYPE CustomerType AS {
+ c_custkey : integer
+};
+
+CREATE TYPE NationType AS {
+ n_nationkey : integer
+};
+
+CREATE DATASET orders(OrderType) PRIMARY KEY o_orderkey;
+
+CREATE DATASET customer(CustomerType) PRIMARY KEY c_custkey;
+
+CREATE DATASET nation(NationType) PRIMARY KEY n_nationkey;
+
+SELECT COUNT(*)
+FROM orders o,
+ (
+ SELECT *
+ FROM customer c,
+ nation n
+ WHERE c.c_nationkey = n.n_nationkey) cn
+WHERE cn.c_custkey /*+ hash-bcast (n) */ = o.o_custkey;
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/hints/hashjoin_hint/hashjoin_hint_1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/hints/hashjoin_hint/hashjoin_hint_1.sqlpp
new file mode 100644
index 0000000..de52a17
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/hints/hashjoin_hint/hashjoin_hint_1.sqlpp
@@ -0,0 +1,58 @@
+/*
+ * 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 hash join hint
+* /*+ hashjoin build (<collection_name>) */
+* /*+ hashjoin probe (<collection_name>) */
+* Expected Res : Success
+* Date : 11/20/2022
+*/
+
+DROP dataverse tpch IF EXISTS;
+CREATE dataverse tpch;
+
+USE tpch;
+
+
+CREATE TYPE OrderType AS {
+ o_orderkey : integer
+};
+
+CREATE TYPE CustomerType AS {
+ c_custkey : integer
+};
+
+CREATE TYPE NationType AS {
+ n_nationkey : integer
+};
+
+CREATE DATASET orders(OrderType) PRIMARY KEY o_orderkey;
+
+CREATE DATASET customer(CustomerType) PRIMARY KEY c_custkey;
+
+CREATE DATASET nation(NationType) PRIMARY KEY n_nationkey;
+
+SELECT COUNT(*)
+FROM orders o,
+ (
+ SELECT *
+ FROM customer c,
+ nation n
+ WHERE c.c_nationkey = n.n_nationkey) cn
+WHERE cn.c_custkey /*+ hashjoin build (o) */ = o.o_custkey;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/hints/hashjoin_hint/hashjoin_hint_2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/hints/hashjoin_hint/hashjoin_hint_2.sqlpp
new file mode 100644
index 0000000..b961d3a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/hints/hashjoin_hint/hashjoin_hint_2.sqlpp
@@ -0,0 +1,58 @@
+/*
+ * 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 hash join hint
+* /*+ hashjoin build (<collection_name>) */
+* /*+ hashjoin probe (<collection_name>) */
+* Expected Res : Success
+* Date : 11/20/2022
+*/
+
+DROP dataverse tpch IF EXISTS;
+CREATE dataverse tpch;
+
+USE tpch;
+
+
+CREATE TYPE OrderType AS {
+ o_orderkey : integer
+};
+
+CREATE TYPE CustomerType AS {
+ c_custkey : integer
+};
+
+CREATE TYPE NationType AS {
+ n_nationkey : integer
+};
+
+CREATE DATASET orders(OrderType) PRIMARY KEY o_orderkey;
+
+CREATE DATASET customer(CustomerType) PRIMARY KEY c_custkey;
+
+CREATE DATASET nation(NationType) PRIMARY KEY n_nationkey;
+
+SELECT COUNT(*)
+FROM orders o,
+ (
+ SELECT *
+ FROM customer c,
+ nation n
+ WHERE c.c_nationkey = n.n_nationkey) cn
+WHERE cn.c_custkey /*+ hashjoin build (c) */ = o.o_custkey;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/hints/hashjoin_hint/hashjoin_hint_3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/hints/hashjoin_hint/hashjoin_hint_3.sqlpp
new file mode 100644
index 0000000..67f074f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/hints/hashjoin_hint/hashjoin_hint_3.sqlpp
@@ -0,0 +1,58 @@
+/*
+ * 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 hash join hint
+* /*+ hashjoin build (<collection_name>) */
+* /*+ hashjoin probe (<collection_name>) */
+* Expected Res : Success
+* Date : 11/20/2022
+*/
+
+DROP dataverse tpch IF EXISTS;
+CREATE dataverse tpch;
+
+USE tpch;
+
+
+CREATE TYPE OrderType AS {
+ o_orderkey : integer
+};
+
+CREATE TYPE CustomerType AS {
+ c_custkey : integer
+};
+
+CREATE TYPE NationType AS {
+ n_nationkey : integer
+};
+
+CREATE DATASET orders(OrderType) PRIMARY KEY o_orderkey;
+
+CREATE DATASET customer(CustomerType) PRIMARY KEY c_custkey;
+
+CREATE DATASET nation(NationType) PRIMARY KEY n_nationkey;
+
+SELECT COUNT(*)
+FROM orders o,
+ (
+ SELECT *
+ FROM customer c,
+ nation n
+ WHERE c.c_nationkey = n.n_nationkey) cn
+WHERE cn.c_custkey /*+ hashjoin build (n) */ = o.o_custkey;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/hints/hashjoin_hint/hashjoin_hint_4.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/hints/hashjoin_hint/hashjoin_hint_4.sqlpp
new file mode 100644
index 0000000..47cf50c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/hints/hashjoin_hint/hashjoin_hint_4.sqlpp
@@ -0,0 +1,58 @@
+/*
+ * 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 hash join hint
+* /*+ hashjoin build (<collection_name>) */
+* /*+ hashjoin probe (<collection_name>) */
+* Expected Res : Success
+* Date : 11/20/2022
+*/
+
+DROP dataverse tpch IF EXISTS;
+CREATE dataverse tpch;
+
+USE tpch;
+
+
+CREATE TYPE OrderType AS {
+ o_orderkey : integer
+};
+
+CREATE TYPE CustomerType AS {
+ c_custkey : integer
+};
+
+CREATE TYPE NationType AS {
+ n_nationkey : integer
+};
+
+CREATE DATASET orders(OrderType) PRIMARY KEY o_orderkey;
+
+CREATE DATASET customer(CustomerType) PRIMARY KEY c_custkey;
+
+CREATE DATASET nation(NationType) PRIMARY KEY n_nationkey;
+
+SELECT COUNT(*)
+FROM orders o,
+ (
+ SELECT *
+ FROM customer c,
+ nation n
+ WHERE c.c_nationkey = n.n_nationkey) cn
+WHERE cn.c_custkey /*+ hashjoin probe (o) */ = o.o_custkey;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/hints/hashjoin_hint/hashjoin_hint_5.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/hints/hashjoin_hint/hashjoin_hint_5.sqlpp
new file mode 100644
index 0000000..31af68b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/hints/hashjoin_hint/hashjoin_hint_5.sqlpp
@@ -0,0 +1,58 @@
+/*
+ * 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 hash join hint
+* /*+ hashjoin build (<collection_name>) */
+* /*+ hashjoin probe (<collection_name>) */
+* Expected Res : Success
+* Date : 11/20/2022
+*/
+
+DROP dataverse tpch IF EXISTS;
+CREATE dataverse tpch;
+
+USE tpch;
+
+
+CREATE TYPE OrderType AS {
+ o_orderkey : integer
+};
+
+CREATE TYPE CustomerType AS {
+ c_custkey : integer
+};
+
+CREATE TYPE NationType AS {
+ n_nationkey : integer
+};
+
+CREATE DATASET orders(OrderType) PRIMARY KEY o_orderkey;
+
+CREATE DATASET customer(CustomerType) PRIMARY KEY c_custkey;
+
+CREATE DATASET nation(NationType) PRIMARY KEY n_nationkey;
+
+SELECT COUNT(*)
+FROM orders o,
+ (
+ SELECT *
+ FROM customer c,
+ nation n
+ WHERE c.c_nationkey = n.n_nationkey) cn
+WHERE cn.c_custkey /*+ hashjoin probe (c) */ = o.o_custkey;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/hints/hashjoin_hint/hashjoin_hint_6.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/hints/hashjoin_hint/hashjoin_hint_6.sqlpp
new file mode 100644
index 0000000..de36c1f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/hints/hashjoin_hint/hashjoin_hint_6.sqlpp
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description : Test hash join hint
+* /*+ hashjoin build (<collection_name>) */
+* /*+ hashjoin probe (<collection_name>) */
+* Expected Res : Success
+* Date : 11/20/2022
+*/
+
+DROP dataverse tpch IF EXISTS;
+CREATE dataverse tpch;
+
+USE tpch;
+
+
+CREATE TYPE OrderType AS {
+ o_orderkey : integer
+};
+
+CREATE TYPE CustomerType AS {
+ c_custkey : integer
+};
+
+CREATE TYPE NationType AS {
+ n_nationkey : integer
+};
+
+CREATE DATASET orders(OrderType) PRIMARY KEY o_orderkey;
+
+CREATE DATASET customer(CustomerType) PRIMARY KEY c_custkey;
+
+CREATE DATASET nation(NationType) PRIMARY KEY n_nationkey;
+
+SELECT COUNT(*)
+FROM orders o,
+ (
+ SELECT *
+ FROM customer c,
+ nation n
+ WHERE c.c_nationkey = n.n_nationkey) cn
+WHERE cn.c_custkey /*+ hashjoin probe (n) */ = o.o_custkey;
+
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/joins/nlj_partitioning_property_1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/joins/nlj_partitioning_property_1.sqlpp
new file mode 100644
index 0000000..e0b4ba4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/joins/nlj_partitioning_property_1.sqlpp
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Test the fix for nested loop join delivered partitioning property ASTERIXDB-3066
+ */
+
+DROP DATAVERSE tpch IF EXISTS;
+CREATE DATAVERSE tpch;
+
+USE tpch;
+
+CREATE TYPE tpch.SupplierType AS
+ CLOSED {
+ s_suppkey : bigint,
+ s_name : string,
+ s_address : string,
+ s_nationkey : bigint,
+ s_phone : string,
+ s_acctbal : double,
+ s_comment : string
+};
+
+CREATE TYPE tpch.PartType AS
+ CLOSED {
+ p_partkey : bigint,
+ p_name : string,
+ p_mfgr : string,
+ p_brand : string,
+ p_type : string,
+ p_size : bigint,
+ p_container : string,
+ p_retailprice : double,
+ p_comment : string
+};
+
+CREATE TYPE tpch.PartSuppType AS
+ CLOSED {
+ ps_partkey : bigint,
+ ps_suppkey : bigint,
+ ps_availqty : bigint,
+ ps_supplycost : double,
+ ps_comment : string
+};
+
+CREATE DATASET Supplier(SupplierType) PRIMARY KEY s_suppkey;
+CREATE DATASET Part(PartType) PRIMARY KEY p_partkey;
+CREATE DATASET Partsupp(PartSuppType) PRIMARY KEY ps_partkey,ps_suppkey;
+
+CREATE INDEX partsupp_fk_part ON Partsupp (ps_partkey);
+CREATE INDEX partsupp_fk_supplier ON Partsupp (ps_suppkey);
+
+FROM Supplier s JOIN (FROM Partsupp ps, Part p SELECT ps.ps_suppkey AS suppkey) AS p_ps
+ON s.s_suppkey /*+ hash-bcast */ = p_ps.suppkey
+SELECT count(*) AS count;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/joins/nlj_partitioning_property_2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/joins/nlj_partitioning_property_2.sqlpp
new file mode 100644
index 0000000..8b9a5bf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/joins/nlj_partitioning_property_2.sqlpp
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Test the fix for nested loop join delivered partitioning property ASTERIXDB-3066
+ */
+
+DROP DATAVERSE tpch IF EXISTS;
+CREATE DATAVERSE tpch;
+
+USE tpch;
+
+CREATE TYPE tpch.SupplierType AS
+ CLOSED {
+ s_suppkey : bigint,
+ s_name : string,
+ s_address : string,
+ s_nationkey : bigint,
+ s_phone : string,
+ s_acctbal : double,
+ s_comment : string
+};
+
+CREATE TYPE tpch.PartType AS
+ CLOSED {
+ p_partkey : bigint,
+ p_name : string,
+ p_mfgr : string,
+ p_brand : string,
+ p_type : string,
+ p_size : bigint,
+ p_container : string,
+ p_retailprice : double,
+ p_comment : string
+};
+
+CREATE TYPE tpch.PartSuppType AS
+ CLOSED {
+ ps_partkey : bigint,
+ ps_suppkey : bigint,
+ ps_availqty : bigint,
+ ps_supplycost : double,
+ ps_comment : string
+};
+
+CREATE DATASET Supplier(SupplierType) PRIMARY KEY s_suppkey;
+CREATE DATASET Part(PartType) PRIMARY KEY p_partkey;
+CREATE DATASET Partsupp(PartSuppType) PRIMARY KEY ps_partkey,ps_suppkey;
+
+CREATE INDEX partsupp_fk_part ON Partsupp (ps_partkey);
+CREATE INDEX partsupp_fk_supplier ON Partsupp (ps_suppkey);
+
+USE tpch;
+
+FROM Supplier s JOIN (FROM Partsupp ps, Part p SELECT ps.ps_suppkey AS suppkey) AS p_ps ON TRUE
+SELECT count(*) AS count;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-3334.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-3334.sqlpp
new file mode 100644
index 0000000..c21b569
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-3334.sqlpp
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type dt1 as {Id:int};
+create dataset collection0(dt1) primary key Id;
+
+CREATE VIEW `Staples`(
+`Item Count` BIGINT NOT UNKNOWN, `Ship Priority` STRING NOT UNKNOWN, `Order Priority` STRING NOT UNKNOWN,
+`Order Status` STRING NOT UNKNOWN, `Order Quantity` DOUBLE NOT UNKNOWN, `Sales Total` DOUBLE NOT UNKNOWN,
+`Discount` DOUBLE NOT UNKNOWN, `Tax Rate` DOUBLE NOT UNKNOWN, `Ship Mode` STRING NOT UNKNOWN, `Fill Time` DOUBLE NOT UNKNOWN,
+`Gross Profit` DOUBLE NOT UNKNOWN, `Price` DOUBLE NOT UNKNOWN, `Ship Handle Cost` DOUBLE NOT UNKNOWN, `Employee Name` STRING NOT UNKNOWN,
+`Employee Dept` STRING NOT UNKNOWN, `Manager Name` STRING NOT UNKNOWN, `Employee Yrs Exp` DOUBLE NOT UNKNOWN, `Employee Salary` DOUBLE NOT UNKNOWN,
+`Customer Name` STRING NOT UNKNOWN, `Customer State` STRING NOT UNKNOWN, `Call Center Region` STRING NOT UNKNOWN, `Customer Balance` DOUBLE NOT UNKNOWN,
+`Customer Segment` STRING NOT UNKNOWN, `Prod Type1` STRING NOT UNKNOWN, `Prod Type2` STRING NOT UNKNOWN, `Prod Type3` STRING NOT UNKNOWN,
+`Prod Type4` STRING NOT UNKNOWN, `Product Name` STRING NOT UNKNOWN, `Product Container` STRING NOT UNKNOWN, `Ship Promo` STRING NOT UNKNOWN,
+`Supplier Name` STRING NOT UNKNOWN, `Supplier Balance` DOUBLE NOT UNKNOWN, `Supplier Region` STRING NOT UNKNOWN, `Supplier State` STRING NOT UNKNOWN,
+`Order ID` STRING NOT UNKNOWN, `Order Year` BIGINT NOT UNKNOWN, `Order Month` BIGINT NOT UNKNOWN, `Order Day` BIGINT NOT UNKNOWN,
+`Order Date` DATETIME NOT UNKNOWN, `Order Quarter` STRING NOT UNKNOWN, `Product Base Margin` DOUBLE NOT UNKNOWN, `Product ID` STRING NOT UNKNOWN,
+`Receive Time` DOUBLE NOT UNKNOWN, `Received Date` DATETIME NOT UNKNOWN, `Ship Date` DATETIME NOT UNKNOWN, `Ship Charge` DOUBLE NOT UNKNOWN,
+`Total Cycle Time` DOUBLE NOT UNKNOWN, `Product In Stock` STRING NOT UNKNOWN, `PID` BIGINT NOT UNKNOWN, `Market Segment` STRING NOT UNKNOWN)
+default NULL AS `collection0`;
+
+SELECT `Staples`.`Employee Name` AS `Employee Name`,
+ AVG(`Staples`.`Employee Salary`) AS `avg:Employee Salary:ok`
+FROM `Staples` `Staples`
+ INNER JOIN (
+ SELECT `Staples`.`Call Center Region` AS `Call Center Region`,
+ `Staples`.`Employee Name` AS `Employee Name`
+ FROM `Staples` `Staples`
+ GROUP BY `Staples`.`Call Center Region`,
+ `Staples`.`Employee Name`
+ HAVING ((AVG(`Staples`.`Employee Salary`)>=102499.99999999898) AND (AVG(`Staples`.`Employee Salary`)<=110000.00000000111))
+) `t0` ON ((`Staples`.`Call Center Region` = `t0`.`Call Center Region`) AND (`Staples`.`Employee Name` = `t0`.`Employee Name`))
+GROUP BY `Staples`.`Employee Name`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-3512.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-3512.sqlpp
new file mode 100644
index 0000000..d51df3b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-3512.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type dt1 as {id: int};
+
+create dataset collection1(dt1) PRIMARY KEY id;
+create dataset collection2(dt1) PRIMARY KEY id;
+
+CREATE INDEX idx1 ON collection2 (date: STRING, oid: STRING, sym: STRING, reg: BIGINT);
+
+with tmp1 AS (
+ SELECT S.sym
+ FROM collection1 AS S
+ WHERE S.date < print_date(current_date(), 'YYYY-MM-DD')),
+ tmp2 AS (
+ SELECT O.sym
+ FROM collection2 AS O
+ WHERE O.date < print_date(current_date(), 'YYYY-MM-DD'))
+SELECT sl1.sym
+FROM tmp1 AS sl1
+UNION ALL
+SELECT sl2.sym
+FROM tmp2 as sl2;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query_issue3316.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query_issue3316.sqlpp
new file mode 100644
index 0000000..cc9bb60
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query_issue3316.sqlpp
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : This test case is to verify the fix for issue 3316
+ * Expected Res : SUCCESS
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+create type dt1 as {id:int};
+
+create dataset collection0(dt1) primary key id;
+create dataset collection1(dt1) primary key id;
+
+SELECT DISTINCT T1.*,
+ (SELECT H.u
+ FROM collection1 H
+ WHERE H.y_id = T1.x_id
+ AND H.a IN [12, 66, 67, 13, 26]
+ AND H.to_u = 'aaaaa'
+ AND H.b in
+ (SELECT value MAX(L.b)
+ FROM collection1 L
+ WHERE L.y_id = T1.x_id
+ AND L.a IN [12, 66, 67, 13, 26]
+ AND L.to_u = 'aaaaa')
+ ) AS sub_query1,
+ (SELECT H.u
+ FROM collection1 H
+ WHERE H.y_id = T1.x_id
+ AND H.a IN [12, 66, 67, 13]
+ AND H.posi IN ['a','b']
+ AND H.b in
+ (SELECT value MAX(L.b)
+ FROM collection1 L
+ WHERE L.y_id = T1.x_id
+ AND L.posi IN ['a','b'])
+ ) AS sub_query2
+FROM (SELECT T0.x_id
+ FROM collection0 T0
+) T1;
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/remove_listify.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/remove_listify.sqlpp
new file mode 100644
index 0000000..d6475f3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/remove_listify.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 test if exists;
+create dataverse test;
+
+use test;
+
+
+create type dt as
+{
+ uid : integer
+};
+
+create dataset collection1(dt) primary key uid;
+
+SET `rewrite_or_as_join` "false";
+WITH table23 AS(
+ SELECT f1,
+ f2
+ FROM collection1 b),
+table22 AS(
+ SELECT f1,
+ (
+ SELECT COUNT(1) FILTER(WHERE b.f2>=a.f2-5 AND b.f2<=a.f2) AS counts
+ FROM table23 b
+ WHERE b.f1=a.f1
+ )[0] counts
+ FROM table23 a)
+SELECT
+ value a
+FROM table22 a;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/ASTERIXDB-2402.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ASTERIXDB-2402.plan
index 0f996d1..efd14f7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/ASTERIXDB-2402.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ASTERIXDB-2402.plan
@@ -34,98 +34,48 @@
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$252] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$253] |PARTITIONED|
- {
- -- AGGREGATE |LOCAL|
- -- STREAM_SELECT |LOCAL|
- -- NESTED_TUPLE_SOURCE |LOCAL|
- }
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$253(ASC)] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$253] |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- STABLE_SORT [$$252(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$252] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$251][$$222] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$309][$$222] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$309] |PARTITIONED|
- -- RUNNING_AGGREGATE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- UNNEST |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$317] |PARTITIONED|
- {
- -- AGGREGATE |LOCAL|
- -- MICRO_PRE_CLUSTERED_GROUP_BY[$$319, $$321] |LOCAL|
- {
- -- AGGREGATE |LOCAL|
- -- STREAM_SELECT |LOCAL|
- -- NESTED_TUPLE_SOURCE |LOCAL|
- }
- -- STREAM_SELECT |LOCAL|
- -- NESTED_TUPLE_SOURCE |LOCAL|
- }
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$317(ASC), $$319(ASC), $$321(ASC)] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$317] |PARTITIONED|
- -- UNION_ALL |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (channels.Shelters.Shelters) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- SPLIT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- RTREE_SEARCH (channels.Shelters.s_location) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- NESTED_LOOP |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- NESTED_LOOP |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- RUNNING_AGGREGATE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (channels.EmergenciesNearMeChannelChannelSubscriptions) |PARTITIONED|
- -- BROADCAST_EXCHANGE |PARTITIONED|
- -- ASSIGN |UNPARTITIONED|
- -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
- -- BROADCAST_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (channels.Reports.Reports) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$260(ASC)] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (channels.Reports.report_time) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- BROADCAST_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (channels.UserLocations) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
+ -- RUNNING_AGGREGATE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$313] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- MICRO_PRE_CLUSTERED_GROUP_BY[$$315, $$316] |LOCAL|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$313(ASC), $$315(ASC), $$316(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$313] |PARTITIONED|
+ -- UNION_ALL |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (channels.Shelters.Shelters) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- SPLIT |PARTITIONED|
@@ -133,20 +83,18 @@
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- RTREE_SEARCH (channels.Shelters.s_location) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- NESTED_LOOP |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- NESTED_LOOP |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- RUNNING_AGGREGATE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (channels.EmergenciesNearMeChannelChannelSubscriptions) |PARTITIONED|
- -- BROADCAST_EXCHANGE |PARTITIONED|
- -- ASSIGN |UNPARTITIONED|
- -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (channels.EmergenciesNearMeChannelChannelSubscriptions) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
@@ -154,7 +102,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (channels.Reports.Reports) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$260(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$259(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
@@ -168,22 +116,67 @@
-- DATASOURCE_SCAN (channels.UserLocations) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$222] |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- SPLIT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- RTREE_SEARCH (channels.Shelters.s_location) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- NESTED_LOOP |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- NESTED_LOOP |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (channels.EmergenciesNearMeChannelChannelSubscriptions) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (channels.Reports.Reports) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$259(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (channels.Reports.report_time) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (channels.UserLocations) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$222] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$233, $$235][$$224, $$225] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$233, $$235] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (channels.EmergenciesNearMeChannelBrokerSubscriptions) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$233, $$235][$$224, $$225] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$233, $$235] |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (channels.Broker) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (channels.EmergenciesNearMeChannelBrokerSubscriptions) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (channels.Broker) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query8.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query8.plan
index ea7ceef..3f740b0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query8.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query8.plan
@@ -22,7 +22,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- PRE_SORTED_DISTINCT_BY |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$74(ASC), $$75(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$77(ASC), $$78(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query9.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query9.plan
index b47e122..ed318b1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query9.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query9.plan
@@ -31,7 +31,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- PRE_SORTED_DISTINCT_BY |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$104(ASC), $$105(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$108(ASC), $$109(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-indexes/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-indexes/query1.plan
new file mode 100644
index 0000000..0d37f51
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-indexes/query1.plan
@@ -0,0 +1,43 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.KSI.KSI) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- INTERSECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$40(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.KSI.KS1_array_index1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$44(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.KSI.KS1_array_index2) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-indexes/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-indexes/query2.plan
new file mode 100644
index 0000000..b793939
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-indexes/query2.plan
@@ -0,0 +1,54 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.KSI.KSI) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- INTERSECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$43(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.KSI.KS1_array_index1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$47(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.KSI.KS1_array_index2) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$51(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.KSI.KS1_array_index3) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query10.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query10.plan
new file mode 100644
index 0000000..50d2d2d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query10.plan
@@ -0,0 +1,36 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- UNNEST |LOCAL|
+ -- SUBPLAN |LOCAL|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$33(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query11.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query11.plan
new file mode 100644
index 0000000..f14e55a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query11.plan
@@ -0,0 +1,43 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- UNNEST |LOCAL|
+ -- SUBPLAN |LOCAL|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- UNNEST |LOCAL|
+ -- SUBPLAN |LOCAL|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$41(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/multiple-indexes/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/multiple-indexes/query1.plan
new file mode 100644
index 0000000..4c76ea0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/multiple-indexes/query1.plan
@@ -0,0 +1,35 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.KSI.KSI) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- INTERSECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$39(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.KSI.KS1_array_index1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$43(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.KSI.KS1_array_index2) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/multiple-indexes/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/multiple-indexes/query2.plan
new file mode 100644
index 0000000..7a2f7ce
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/multiple-indexes/query2.plan
@@ -0,0 +1,46 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.KSI.KSI) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- INTERSECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$42(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.KSI.KS1_array_index1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$46(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.KSI.KS1_array_index2) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$50(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.KSI.KS1_array_index3) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-01.plan
index 0dcfc44..a3d4489 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-01.plan
@@ -2,15 +2,11 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$66(ASC) ] |PARTITIONED|
- -- STABLE_SORT [$$66(ASC)] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.ds1) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
+ -- SORT_MERGE_EXCHANGE [$$59(ASC) ] |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-02.plan
index 7997aad..09d5382 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-02.plan
@@ -2,22 +2,18 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$66(ASC) ] |PARTITIONED|
- -- STABLE_SORT [$$66(ASC)] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$59(ASC) ] |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.ds2.ds2) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$72(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.ds2.ds2) |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$72(ASC)] |PARTITIONED|
+ -- BTREE_SEARCH (test.ds2.idx2) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.ds2.idx2) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-03.plan
index d1264cb..40e63cb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-03.plan
@@ -2,15 +2,11 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$66(ASC) ] |PARTITIONED|
- -- STABLE_SORT [$$66(ASC)] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.ds3) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
+ -- SORT_MERGE_EXCHANGE [$$59(ASC) ] |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds3) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-05.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-05.plan
index 6285f162..1c9845f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-05.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-05.plan
@@ -24,4 +24,4 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN (test.ds4) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-10.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-10.plan
index 19fe8ce..9e39687 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-10.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-10.plan
@@ -2,22 +2,18 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$56(ASC) ] |PARTITIONED|
- -- STABLE_SORT [$$56(ASC)] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$48(ASC) ] |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.ds5.ds5) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$59(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.ds5.ds5) |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$59(ASC)] |PARTITIONED|
+ -- BTREE_SEARCH (test.ds5.idx5_dt_fmt) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.ds5.idx5_dt_fmt) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-11.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-11.plan
index b636106..505a834 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-11.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-11.plan
@@ -2,22 +2,18 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$56(ASC) ] |PARTITIONED|
- -- STABLE_SORT [$$56(ASC)] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$48(ASC) ] |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.ds5.ds5) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$59(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.ds5.ds5) |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$59(ASC)] |PARTITIONED|
+ -- BTREE_SEARCH (test.ds5.idx5_d_fmt) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.ds5.idx5_d_fmt) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-12.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-12.plan
index 382c39a..11a2618 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-12.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-12.plan
@@ -2,22 +2,18 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$56(ASC) ] |PARTITIONED|
- -- STABLE_SORT [$$56(ASC)] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$48(ASC) ] |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.ds5.ds5) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$59(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.ds5.ds5) |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$59(ASC)] |PARTITIONED|
+ -- BTREE_SEARCH (test.ds5.idx5_t_fmt) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.ds5.idx5_t_fmt) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-13.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-13.plan
index babda5c..7736d99 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-13.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-13.plan
@@ -2,22 +2,18 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$56(ASC) ] |PARTITIONED|
- -- STABLE_SORT [$$56(ASC)] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$48(ASC) ] |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.ds6.ds6) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$59(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.ds6.ds6) |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$59(ASC)] |PARTITIONED|
+ -- BTREE_SEARCH (test.ds6.idx6_dt) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.ds6.idx6_dt) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-14.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-14.plan
index d55a9a2..d9444e7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-14.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-14.plan
@@ -2,22 +2,18 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$56(ASC) ] |PARTITIONED|
- -- STABLE_SORT [$$56(ASC)] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$48(ASC) ] |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.ds6.ds6) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$59(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.ds6.ds6) |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$59(ASC)] |PARTITIONED|
+ -- BTREE_SEARCH (test.ds6.idx6_d) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.ds6.idx6_d) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-15.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-15.plan
index 2799b48..e4b5cd7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-15.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-15.plan
@@ -2,22 +2,18 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$56(ASC) ] |PARTITIONED|
- -- STABLE_SORT [$$56(ASC)] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$48(ASC) ] |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.ds6.ds6) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$59(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.ds6.ds6) |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$59(ASC)] |PARTITIONED|
+ -- BTREE_SEARCH (test.ds6.idx6_t) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.ds6.idx6_t) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-16.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-16.plan
index 6291577..753d23d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-16.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-16.plan
@@ -2,15 +2,11 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$56(ASC) ] |PARTITIONED|
- -- STABLE_SORT [$$56(ASC)] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.ds5) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
+ -- SORT_MERGE_EXCHANGE [$$48(ASC) ] |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds5) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-17.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-17.plan
index 8757f3e..f6647d4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-17.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-17.plan
@@ -2,15 +2,11 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$56(ASC) ] |PARTITIONED|
- -- STABLE_SORT [$$56(ASC)] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.ds6) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
+ -- SORT_MERGE_EXCHANGE [$$48(ASC) ] |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds6) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-18.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-18.plan
index 6291577..753d23d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-18.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-18.plan
@@ -2,15 +2,11 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$56(ASC) ] |PARTITIONED|
- -- STABLE_SORT [$$56(ASC)] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.ds5) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
+ -- SORT_MERGE_EXCHANGE [$$48(ASC) ] |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds5) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-19.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-19.plan
index cb86b13..2327f39 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-19.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-19.plan
@@ -24,4 +24,4 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN (test.ds6) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-20.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-20.plan
index 885b6ba..d71b03a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-20.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-20.plan
@@ -2,22 +2,18 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$65(ASC) ] |PARTITIONED|
- -- STABLE_SORT [$$65(ASC)] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$58(ASC) ] |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.ds7.ds7) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$70(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.ds7.ds7) |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$70(ASC)] |PARTITIONED|
+ -- BTREE_SEARCH (test.ds7.idx2) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.ds7.idx2) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-21.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-21.plan
index a560c7f..e5fd2d4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-21.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-21.plan
@@ -2,15 +2,11 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$65(ASC) ] |PARTITIONED|
- -- STABLE_SORT [$$65(ASC)] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.ds7) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
+ -- SORT_MERGE_EXCHANGE [$$58(ASC) ] |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds7) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-22.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-22.plan
index a560c7f..e5fd2d4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-22.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-22.plan
@@ -2,15 +2,11 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$65(ASC) ] |PARTITIONED|
- -- STABLE_SORT [$$65(ASC)] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.ds7) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
+ -- SORT_MERGE_EXCHANGE [$$58(ASC) ] |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds7) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-23.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-23.plan
index 88537b6..5972aae 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-23.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-23.plan
@@ -2,22 +2,18 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$65(ASC) ] |PARTITIONED|
- -- STABLE_SORT [$$65(ASC)] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$58(ASC) ] |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.ds7.ds7) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$70(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.ds7.ds7) |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$70(ASC)] |PARTITIONED|
+ -- BTREE_SEARCH (test.ds7.idx3) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.ds7.idx3) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-25.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-25.plan
new file mode 100644
index 0000000..d71b03a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-25.plan
@@ -0,0 +1,19 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$58(ASC) ] |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.ds7.ds7) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$70(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.ds7.idx2) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-20.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-20.plan
new file mode 100644
index 0000000..c226460
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-20.plan
@@ -0,0 +1,28 @@
+-- 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|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.orders.orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$56(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.orders.orderline_delivery_d) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-21.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-21.plan
new file mode 100644
index 0000000..5bde8ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-21.plan
@@ -0,0 +1,28 @@
+-- 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|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.orders.orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$55(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.orders.orderline_delivery_d) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |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/query4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-ternary-inlj/query4.plan
new file mode 100644
index 0000000..4e1a931
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-ternary-inlj/query4.plan
@@ -0,0 +1,32 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$57][$$70] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (tpcds.customer_address.customer_address) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$74(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$74] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (tpcds.customer_demographics.customer_demographics) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$72(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$72] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpcds.customer) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/common-expr-01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/common-expr-01.plan
index 132ec02..616e808 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/common-expr-01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/common-expr-01.plan
@@ -34,7 +34,7 @@
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STABLE_SORT [$$172(ASC)] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$172] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- NESTED_LOOP |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/external-cross-product.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/external-cross-product.plan
new file mode 100644
index 0000000..fd95cc1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/external-cross-product.plan
@@ -0,0 +1,45 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- NESTED_LOOP |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- NESTED_LOOP |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.Orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.Orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.Orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/extract-common-operators/extract-common-operators.01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/extract-common-operators/extract-common-operators.01.plan
new file mode 100644
index 0000000..677a45f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/extract-common-operators/extract-common-operators.01.plan
@@ -0,0 +1,98 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$381(ASC), $$382(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$381(ASC), $$382(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$390][$$386] |PARTITIONED|
+ -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$381(ASC), $$382(ASC)] HASH:[$$390] |PARTITIONED|
+ -- STABLE_SORT [$$381(ASC), $$382(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$390][$$384] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$390] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (Metadata.Synonym) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$384] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$384, $$383][$$ds_name, $$dv_name] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$384, $$383] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (Metadata.Dataset) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$ds_name, $$dv_name] |PARTITIONED|
+ -- NESTED_LOOP |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- REPLICATE |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$386] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$404][$$388] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$404] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$386][$$syn_name] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$386] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (Metadata.Synonym) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$syn_name] |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$388] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$388, $$387][$$ds_name, $$dv_name] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$384, $$383] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (Metadata.Dataset) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$ds_name, $$dv_name] |PARTITIONED|
+ -- NESTED_LOOP |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- REPLICATE |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hash-join-with-redundant-variable/hash-join-with-redundant-variable.1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hash-join-with-redundant-variable/hash-join-with-redundant-variable.1.plan
new file mode 100644
index 0000000..8b2d960
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hash-join-with-redundant-variable/hash-join-with-redundant-variable.1.plan
@@ -0,0 +1,74 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$127, $$117][$$128, $$116] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$127, $$117] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$132][$$131] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$132] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$117][$$115] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$117] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$130][$$129] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$130] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.region) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$129] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.nation) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$115] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$125][$$126] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$125] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.customer) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$126] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$131] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.lineitem) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$128, $$116] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.supplier) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hash-join-with-redundant-variable/hash-join-with-redundant-variable.2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hash-join-with-redundant-variable/hash-join-with-redundant-variable.2.plan
new file mode 100644
index 0000000..d56477e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hash-join-with-redundant-variable/hash-join-with-redundant-variable.2.plan
@@ -0,0 +1,35 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$65][$$63] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$65][$$64] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$65] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.region) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$64] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$63] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.nation) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hash-join-with-redundant-variable/hash-join-with-redundant-variable.3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hash-join-with-redundant-variable/hash-join-with-redundant-variable.3.plan
new file mode 100644
index 0000000..5264d56
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hash-join-with-redundant-variable/hash-join-with-redundant-variable.3.plan
@@ -0,0 +1,35 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$68][$$66] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$67][$$68] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$67] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.region) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$68] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.nation) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$66] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hash-join-with-redundant-variable/hash-join-with-redundant-variable.4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hash-join-with-redundant-variable/hash-join-with-redundant-variable.4.plan
new file mode 100644
index 0000000..c898cde
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hash-join-with-redundant-variable/hash-join-with-redundant-variable.4.plan
@@ -0,0 +1,24 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$49, $$56][$$52, $$53] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$49, $$56] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.region) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$52, $$53] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/broadcast_hint_1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/broadcast_join_hint/broadcast_join_hint_1.plan
similarity index 100%
rename from asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/broadcast_hint_1.plan
rename to asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/broadcast_join_hint/broadcast_join_hint_1.plan
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/broadcast_hint_2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/broadcast_join_hint/broadcast_join_hint_2.plan
similarity index 100%
rename from asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/broadcast_hint_2.plan
rename to asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/broadcast_join_hint/broadcast_join_hint_2.plan
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/broadcast_hint_3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/broadcast_join_hint/broadcast_join_hint_3.plan
similarity index 100%
rename from asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/broadcast_hint_3.plan
rename to asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/broadcast_join_hint/broadcast_join_hint_3.plan
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/broadcast_join_hint/broadcast_join_hint_4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/broadcast_join_hint/broadcast_join_hint_4.plan
new file mode 100644
index 0000000..00c7dfd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/broadcast_join_hint/broadcast_join_hint_4.plan
@@ -0,0 +1,34 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$86][$$85] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$83][$$81] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$83] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.customer) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.nation) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/broadcast_join_hint/broadcast_join_hint_5.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/broadcast_join_hint/broadcast_join_hint_5.plan
new file mode 100644
index 0000000..b5ed081
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/broadcast_join_hint/broadcast_join_hint_5.plan
@@ -0,0 +1,34 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$85][$$86] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$83][$$81] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$83] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.customer) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.nation) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/broadcast_join_hint/broadcast_join_hint_6.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/broadcast_join_hint/broadcast_join_hint_6.plan
new file mode 100644
index 0000000..00c7dfd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/broadcast_join_hint/broadcast_join_hint_6.plan
@@ -0,0 +1,34 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$86][$$85] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$83][$$81] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$83] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.customer) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.nation) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/broadcast_join_hint/broadcast_join_hint_7.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/broadcast_join_hint/broadcast_join_hint_7.plan
new file mode 100644
index 0000000..00c7dfd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/broadcast_join_hint/broadcast_join_hint_7.plan
@@ -0,0 +1,34 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$86][$$85] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$83][$$81] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$83] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.customer) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.nation) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/hashjoin_hint/hashjoin_hint_1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/hashjoin_hint/hashjoin_hint_1.plan
new file mode 100644
index 0000000..c5b8dc2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/hashjoin_hint/hashjoin_hint_1.plan
@@ -0,0 +1,34 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$85][$$86] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$85] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$83][$$81] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$83] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.customer) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.nation) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$86] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/hashjoin_hint/hashjoin_hint_2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/hashjoin_hint/hashjoin_hint_2.plan
new file mode 100644
index 0000000..5610321
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/hashjoin_hint/hashjoin_hint_2.plan
@@ -0,0 +1,34 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$86][$$85] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$86] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$85] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$83][$$81] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$83] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.customer) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.nation) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/hashjoin_hint/hashjoin_hint_3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/hashjoin_hint/hashjoin_hint_3.plan
new file mode 100644
index 0000000..5610321
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/hashjoin_hint/hashjoin_hint_3.plan
@@ -0,0 +1,34 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$86][$$85] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$86] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$85] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$83][$$81] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$83] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.customer) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.nation) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/hashjoin_hint/hashjoin_hint_4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/hashjoin_hint/hashjoin_hint_4.plan
new file mode 100644
index 0000000..5610321
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/hashjoin_hint/hashjoin_hint_4.plan
@@ -0,0 +1,34 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$86][$$85] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$86] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$85] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$83][$$81] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$83] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.customer) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.nation) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/hashjoin_hint/hashjoin_hint_5.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/hashjoin_hint/hashjoin_hint_5.plan
new file mode 100644
index 0000000..c5b8dc2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/hashjoin_hint/hashjoin_hint_5.plan
@@ -0,0 +1,34 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$85][$$86] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$85] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$83][$$81] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$83] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.customer) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.nation) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$86] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/hashjoin_hint/hashjoin_hint_6.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/hashjoin_hint/hashjoin_hint_6.plan
new file mode 100644
index 0000000..c5b8dc2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/hashjoin_hint/hashjoin_hint_6.plan
@@ -0,0 +1,34 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$85][$$86] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$85] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$83][$$81] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$83] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.customer) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.nation) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$86] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/inner_right_corr.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/inner_right_corr.plan
index 37c3434..7a7a543 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/inner_right_corr.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/inner_right_corr.plan
@@ -21,7 +21,7 @@
-- DATASOURCE_SCAN (test.t1) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$87] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- UNNEST |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/nested_query_with_bcast.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/nested_query_with_bcast.plan
index 075f7af..15c163d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/nested_query_with_bcast.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/nested_query_with_bcast.plan
@@ -9,31 +9,29 @@
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$38] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$38] |PARTITIONED|
- {
- -- AGGREGATE |LOCAL|
- -- STREAM_SELECT |LOCAL|
- -- NESTED_TUPLE_SOURCE |LOCAL|
- }
+ -- STABLE_SORT [$$38(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$38(ASC)] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$41][$$39] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$41][$$39] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
+ -- DATASOURCE_SCAN (test.tweetDataset) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.tweetDataset) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- BROADCAST_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.countryDataset) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.countryDataset) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/nlj_partitioning_property_1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/nlj_partitioning_property_1.plan
new file mode 100644
index 0000000..f5d57d5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/nlj_partitioning_property_1.plan
@@ -0,0 +1,29 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$71][$$73] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.Supplier) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- NESTED_LOOP |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.Partsupp) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.Part) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/nlj_partitioning_property_2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/nlj_partitioning_property_2.plan
new file mode 100644
index 0000000..4cd1213
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/nlj_partitioning_property_2.plan
@@ -0,0 +1,29 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- NESTED_LOOP |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.Supplier) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- NESTED_LOOP |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.Partsupp) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.Part) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/primary-key-index/insert-primary-key-index-with-auto-gen-pk.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/primary-key-index/insert-primary-key-index-with-auto-gen-pk.plan
index 792b79d..9a296a5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/primary-key-index/insert-primary-key-index-with-auto-gen-pk.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/primary-key-index/insert-primary-key-index-with-auto-gen-pk.plan
@@ -6,7 +6,5 @@
-- ASSIGN |UNPARTITIONED|
-- STREAM_PROJECT |UNPARTITIONED|
-- ASSIGN |UNPARTITIONED|
- -- STREAM_PROJECT |UNPARTITIONED|
- -- ASSIGN |UNPARTITIONED|
- -- ASSIGN |UNPARTITIONED|
- -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-159-3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-159-3.plan
index e932616..6c713c4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-159-3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-159-3.plan
@@ -27,7 +27,7 @@
-- DATASOURCE_SCAN (TinySocial.FacebookUsers) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$54] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- SUBPLAN |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-3334.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-3334.plan
new file mode 100644
index 0000000..bbb94e3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-3334.plan
@@ -0,0 +1,61 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- SORT_GROUP_BY[$$1112] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- HASH_PARTITION_EXCHANGE [$$1112] |PARTITIONED|
+ -- SORT_GROUP_BY[$$1113] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$1113, $$1114][$$Employee Name, $$Call Center Region] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$1114, $$1113] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection0) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- SORT_GROUP_BY[$$1109, $$1110] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- HASH_PARTITION_EXCHANGE [$$1109, $$1110] |PARTITIONED|
+ -- SORT_GROUP_BY[$$1117, $$1116] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection0) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-3512.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-3512.plan
new file mode 100644
index 0000000..9160a43
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-3512.plan
@@ -0,0 +1,28 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- UNION_ALL |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.collection2.collection2) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$117(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.collection2.idx1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query_issue3316.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query_issue3316.plan
new file mode 100644
index 0000000..630cb14
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query_issue3316.plan
@@ -0,0 +1,1000 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$186(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$186] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$221] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$221(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$221][$$382] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$187] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$187][$$234] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection0) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$234] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$120][$$193] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$219, $$188] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$219(ASC), $$188(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$219, $$188][$$248, $$249] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$219, $$188] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$217][$$201] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$408] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection0) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$201] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$232, $$229] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$232(ASC), $$229(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$232, $$229][$$246, $$243] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$419, $$420] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$428][$$427] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$408] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection0) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$427] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$246, $$243] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$129][$$192] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$237][$$204] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$439][$$442] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$408] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection0) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$442] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$204] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$382] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$156][$$196] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$159][$$195] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$250, $$190] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$250(ASC), $$190(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$250, $$190][$$453, $$454] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$250, $$190] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$252][$$208] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$252] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$251] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$251][$$255] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection0) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$255] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$260][$$259] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$262, $$264] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$262(ASC), $$264(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$262, $$264][$$272, $$273] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$403, $$405] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$408][$$409] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$408] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection0) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$409] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$278, $$279] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$278(ASC), $$279(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$278, $$279][$$288, $$289] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$419, $$420] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$428][$$427] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$408] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection0) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$427] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$288, $$289] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$296][$$295] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$298][$$297] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$439][$$442] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$408] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection0) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$442] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$438] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$208] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$321, $$320] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$321(ASC), $$320(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$321, $$320][$$392, $$391] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$321, $$320] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$314][$$313] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$314] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$322] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$322][$$325] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection0) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$325] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$330][$$329] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$332, $$334] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$332(ASC), $$334(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$332, $$334][$$342, $$343] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$403, $$405] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$408][$$409] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$408] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection0) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$409] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$348, $$349] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$348(ASC), $$349(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$348, $$349][$$358, $$359] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$419, $$420] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$428][$$427] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$408] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection0) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$427] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$358, $$359] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$366][$$365] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$368][$$367] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$439][$$442] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$408] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection0) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$442] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$438] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$313] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$392, $$391] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$166][$$194] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$385][$$210] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$385][$$384] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$385] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$393] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$393][$$396] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection0) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$396] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$401][$$400] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$403, $$405] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$403(ASC), $$405(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$403, $$405][$$413, $$414] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$403, $$405] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$408][$$409] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$408] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection0) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$409] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$419, $$420] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$419(ASC), $$420(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$419, $$420][$$429, $$430] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$419, $$420] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$428][$$427] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$408] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection0) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$427] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$429, $$430] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$437][$$436] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$439][$$438] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$439][$$442] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$408] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection0) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$442] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$438] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$384] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$210] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/remove_listify.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/remove_listify.plan
new file mode 100644
index 0000000..96d4248
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/remove_listify.plan
@@ -0,0 +1,36 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$141] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- ASSIGN |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$141(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$141] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$146][$$148] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$146] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$148] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection1) |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/spatial_join/st_distance_static_partitioning.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/spatial_join/st_distance_static_partitioning.plan
index 25546ce..f66785b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/spatial_join/st_distance_static_partitioning.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/spatial_join/st_distance_static_partitioning.plan
@@ -7,10 +7,10 @@
-- AGGREGATE |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- SPATIAL_JOIN [$$61, $$56] [$$62, $$57] |PARTITIONED|
+ -- SPATIAL_JOIN [$$62, $$56] [$$63, $$57] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$61(ASC), $$56(ASC)] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$61] |PARTITIONED|
+ -- STABLE_SORT [$$62(ASC), $$56(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$62] |PARTITIONED|
-- UNNEST |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -21,8 +21,8 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$62(ASC), $$57(ASC)] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$62] |PARTITIONED|
+ -- STABLE_SORT [$$63(ASC), $$57(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$63] |PARTITIONED|
-- UNNEST |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_correlated.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_correlated.plan
index 540e821..42c8f8f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_correlated.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_correlated.plan
@@ -12,20 +12,13 @@
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
- -- UNNEST |LOCAL|
- -- MICRO_PRE_CLUSTERED_GROUP_BY[] |LOCAL|
- {
- -- AGGREGATE |LOCAL|
- -- STREAM_SELECT |LOCAL|
- -- NESTED_TUPLE_SOURCE |LOCAL|
- }
- -- NESTED_TUPLE_SOURCE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- HYBRID_HASH_JOIN [$$49][$$48] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN (test.Customers) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
@@ -37,4 +30,4 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN (test.Orders) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_correlated_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_correlated_ps.plan
index 6080780..85eb2da 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_correlated_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_correlated_ps.plan
@@ -16,20 +16,13 @@
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
- -- UNNEST |LOCAL|
- -- MICRO_PRE_CLUSTERED_GROUP_BY[] |LOCAL|
- {
- -- AGGREGATE |LOCAL|
- -- STREAM_SELECT |LOCAL|
- -- NESTED_TUPLE_SOURCE |LOCAL|
- }
- -- NESTED_TUPLE_SOURCE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- HYBRID_HASH_JOIN [$$49][$$48] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN (test.Customers) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
@@ -56,20 +49,13 @@
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
- -- UNNEST |LOCAL|
- -- MICRO_PRE_CLUSTERED_GROUP_BY[] |LOCAL|
- {
- -- AGGREGATE |LOCAL|
- -- STREAM_SELECT |LOCAL|
- -- NESTED_TUPLE_SOURCE |LOCAL|
- }
- -- NESTED_TUPLE_SOURCE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- HYBRID_HASH_JOIN [$$49][$$48] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN (test.Customers) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
@@ -81,4 +67,4 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN (test.Orders) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-2845.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-2845.plan
index 4b32064..15e7546 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-2845.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-2845.plan
@@ -56,7 +56,7 @@
-- DATASOURCE_SCAN (test.mds) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$303] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- NESTED_LOOP |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.10.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.10.plan
new file mode 100644
index 0000000..9ea1780
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.10.plan
@@ -0,0 +1,35 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$g(ASC) ] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$g] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- MICRO_PRE_SORTED_DISTINCT_BY |LOCAL|
+ -- MICRO_STABLE_SORT [$$67(ASC)] |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$g(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$g] |PARTITIONED|
+ -- NESTED_LOOP |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- NESTED_LOOP |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.d1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.d2) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.11.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.11.plan
new file mode 100644
index 0000000..ebfcfc2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.11.plan
@@ -0,0 +1,43 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$g(ASC) ] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$g] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- MICRO_PRE_SORTED_DISTINCT_BY |LOCAL|
+ -- MICRO_STABLE_SORT [$$67(ASC)] |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ {
+ -- AGGREGATE |LOCAL|
+ -- MICRO_PRE_SORTED_DISTINCT_BY |LOCAL|
+ -- MICRO_STABLE_SORT [$$72(ASC)] |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$g(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$g] |PARTITIONED|
+ -- NESTED_LOOP |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- NESTED_LOOP |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.d1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.d2) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.12.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.12.plan
new file mode 100644
index 0000000..a8f346f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.12.plan
@@ -0,0 +1,39 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$g(ASC) ] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$g] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ {
+ -- AGGREGATE |LOCAL|
+ -- MICRO_PRE_SORTED_DISTINCT_BY |LOCAL|
+ -- MICRO_STABLE_SORT [$$71(ASC)] |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$g(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$g] |PARTITIONED|
+ -- NESTED_LOOP |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- NESTED_LOOP |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.d1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.d2) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.13.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.13.plan
new file mode 100644
index 0000000..439b1d4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.13.plan
@@ -0,0 +1,57 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$g(ASC) ] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$g] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- MICRO_PRE_SORTED_DISTINCT_BY |LOCAL|
+ -- MICRO_STABLE_SORT [$$82(ASC)] |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ {
+ -- AGGREGATE |LOCAL|
+ -- MICRO_PRE_SORTED_DISTINCT_BY |LOCAL|
+ -- MICRO_STABLE_SORT [$$92(ASC)] |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$g(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$g] |PARTITIONED|
+ -- NESTED_LOOP |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- NESTED_LOOP |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- NESTED_LOOP |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.d1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.d2) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.d3) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.14.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.14.plan
new file mode 100644
index 0000000..c57b6f9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.14.plan
@@ -0,0 +1,63 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$g(ASC) ] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$g] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- MICRO_PRE_SORTED_DISTINCT_BY |LOCAL|
+ -- MICRO_STABLE_SORT [$$106(ASC)] |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ {
+ -- AGGREGATE |LOCAL|
+ -- MICRO_PRE_SORTED_DISTINCT_BY |LOCAL|
+ -- MICRO_STABLE_SORT [$$116(ASC)] |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ {
+ -- AGGREGATE |LOCAL|
+ -- MICRO_PRE_SORTED_DISTINCT_BY |LOCAL|
+ -- MICRO_STABLE_SORT [$$111(ASC)] |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$g(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$g] |PARTITIONED|
+ -- NESTED_LOOP |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- NESTED_LOOP |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- NESTED_LOOP |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.d1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.d2) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.d3) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.9.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.9.plan
new file mode 100644
index 0000000..8802ff5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.9.plan
@@ -0,0 +1,35 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$g(ASC) ] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$g] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- MICRO_PRE_SORTED_DISTINCT_BY |LOCAL|
+ -- MICRO_STABLE_SORT [$$63(ASC)] |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$g(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$g] |PARTITIONED|
+ -- NESTED_LOOP |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- NESTED_LOOP |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.d1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.d2) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/atomic-and-array-queries/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/atomic-and-array-queries/query1.plan
new file mode 100644
index 0000000..d69738a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/atomic-and-array-queries/query1.plan
@@ -0,0 +1,14 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestDataverse.Dataset1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/atomic-and-array-queries/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/atomic-and-array-queries/query2.plan
new file mode 100644
index 0000000..d69738a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/atomic-and-array-queries/query2.plan
@@ -0,0 +1,14 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestDataverse.Dataset1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/atomic-and-array-queries/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/atomic-and-array-queries/query3.plan
new file mode 100644
index 0000000..52f7d02
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/atomic-and-array-queries/query3.plan
@@ -0,0 +1,18 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestDataverse.Dataset1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/atomic-and-array-queries/query8.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/atomic-and-array-queries/query8.plan
new file mode 100644
index 0000000..ea7ceef
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/atomic-and-array-queries/query8.plan
@@ -0,0 +1,38 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$74(ASC), $$75(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- RUNNING_AGGREGATE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestDataverse.Dataset2) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/atomic-and-array-queries/query9.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/atomic-and-array-queries/query9.plan
new file mode 100644
index 0000000..b47e122
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/atomic-and-array-queries/query9.plan
@@ -0,0 +1,47 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- SUBPLAN |LOCAL|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- STREAM_SELECT |LOCAL|
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$104(ASC), $$105(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- RUNNING_AGGREGATE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestDataverse.Dataset2) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-1/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-1/query1.plan
new file mode 100644
index 0000000..834d7d2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-1/query1.plan
@@ -0,0 +1,14 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-1/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-1/query2.plan
new file mode 100644
index 0000000..d18b9b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-1/query2.plan
@@ -0,0 +1,17 @@
+-- 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|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-2/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-2/query1.plan
new file mode 100644
index 0000000..834d7d2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-2/query1.plan
@@ -0,0 +1,14 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-2/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-2/query2.plan
new file mode 100644
index 0000000..d18b9b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-2/query2.plan
@@ -0,0 +1,17 @@
+-- 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|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-3/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-3/query1.plan
new file mode 100644
index 0000000..834d7d2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-3/query1.plan
@@ -0,0 +1,14 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-3/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-3/query2.plan
new file mode 100644
index 0000000..5e46297
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-3/query2.plan
@@ -0,0 +1,19 @@
+-- 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|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-3/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-3/query3.plan
new file mode 100644
index 0000000..834d7d2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-3/query3.plan
@@ -0,0 +1,14 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-4/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-4/query1.plan
new file mode 100644
index 0000000..7c44281
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-4/query1.plan
@@ -0,0 +1,18 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-4/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-4/query2.plan
new file mode 100644
index 0000000..b191fe2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-4/query2.plan
@@ -0,0 +1,21 @@
+-- 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|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-3-level-record-path/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-3-level-record-path/query1.plan
new file mode 100644
index 0000000..834d7d2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-3-level-record-path/query1.plan
@@ -0,0 +1,14 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-3-level-record-path/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-3-level-record-path/query2.plan
new file mode 100644
index 0000000..d18b9b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-3-level-record-path/query2.plan
@@ -0,0 +1,17 @@
+-- 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|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-composite-pk/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-composite-pk/query1.plan
new file mode 100644
index 0000000..070c67b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-composite-pk/query1.plan
@@ -0,0 +1,15 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-composite-pk/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-composite-pk/query2.plan
new file mode 100644
index 0000000..d18b9b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-composite-pk/query2.plan
@@ -0,0 +1,17 @@
+-- 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|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-composite-sk/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-composite-sk/query1.plan
new file mode 100644
index 0000000..e92baf8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-composite-sk/query1.plan
@@ -0,0 +1,16 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-composite-sk/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-composite-sk/query2.plan
new file mode 100644
index 0000000..5e46297
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-composite-sk/query2.plan
@@ -0,0 +1,19 @@
+-- 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|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-filter-fields/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-filter-fields/query1.plan
new file mode 100644
index 0000000..a224a0e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-filter-fields/query1.plan
@@ -0,0 +1,17 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckin) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-filter-fields/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-filter-fields/query2.plan
new file mode 100644
index 0000000..d18b9b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-filter-fields/query2.plan
@@ -0,0 +1,17 @@
+-- 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|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query1.plan
new file mode 100644
index 0000000..30d30c6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query1.plan
@@ -0,0 +1,13 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestDataverse.TestDataset) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query2.plan
new file mode 100644
index 0000000..30d30c6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query2.plan
@@ -0,0 +1,13 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestDataverse.TestDataset) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query3.plan
new file mode 100644
index 0000000..4cbcbbd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query3.plan
@@ -0,0 +1,16 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestDataverse.TestDataset) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query4.plan
new file mode 100644
index 0000000..4cbcbbd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query4.plan
@@ -0,0 +1,16 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestDataverse.TestDataset) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query5.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query5.plan
new file mode 100644
index 0000000..30d30c6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query5.plan
@@ -0,0 +1,13 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestDataverse.TestDataset) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query6.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query6.plan
new file mode 100644
index 0000000..30d30c6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query6.plan
@@ -0,0 +1,13 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestDataverse.TestDataset) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query7.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query7.plan
new file mode 100644
index 0000000..4cbcbbd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query7.plan
@@ -0,0 +1,16 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestDataverse.TestDataset) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query8.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query8.plan
new file mode 100644
index 0000000..4cbcbbd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query8.plan
@@ -0,0 +1,16 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestDataverse.TestDataset) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/multiple-indexes/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/multiple-indexes/query1.plan
new file mode 100644
index 0000000..8a0bc58
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/multiple-indexes/query1.plan
@@ -0,0 +1,13 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.KSI) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/multiple-indexes/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/multiple-indexes/query2.plan
new file mode 100644
index 0000000..8a0bc58
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/multiple-indexes/query2.plan
@@ -0,0 +1,13 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.KSI) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-1/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-1/query1.plan
new file mode 100644
index 0000000..834d7d2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-1/query1.plan
@@ -0,0 +1,14 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-1/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-1/query2.plan
new file mode 100644
index 0000000..d18b9b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-1/query2.plan
@@ -0,0 +1,17 @@
+-- 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|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-2/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-2/query1.plan
new file mode 100644
index 0000000..834d7d2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-2/query1.plan
@@ -0,0 +1,14 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-2/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-2/query2.plan
new file mode 100644
index 0000000..d18b9b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-2/query2.plan
@@ -0,0 +1,17 @@
+-- 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|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-3/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-3/query1.plan
new file mode 100644
index 0000000..834d7d2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-3/query1.plan
@@ -0,0 +1,14 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-3/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-3/query2.plan
new file mode 100644
index 0000000..5e46297
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-3/query2.plan
@@ -0,0 +1,19 @@
+-- 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|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-3/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-3/query3.plan
new file mode 100644
index 0000000..834d7d2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-3/query3.plan
@@ -0,0 +1,14 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-4/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-4/query1.plan
new file mode 100644
index 0000000..7c44281
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-4/query1.plan
@@ -0,0 +1,18 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-4/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-4/query2.plan
new file mode 100644
index 0000000..b191fe2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-4/query2.plan
@@ -0,0 +1,21 @@
+-- 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|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/with-3-level-record-path/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/with-3-level-record-path/query1.plan
new file mode 100644
index 0000000..834d7d2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/with-3-level-record-path/query1.plan
@@ -0,0 +1,14 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/with-3-level-record-path/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/with-3-level-record-path/query2.plan
new file mode 100644
index 0000000..d18b9b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/with-3-level-record-path/query2.plan
@@ -0,0 +1,17 @@
+-- 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|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/with-composite-sk/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/with-composite-sk/query1.plan
new file mode 100644
index 0000000..e92baf8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/with-composite-sk/query1.plan
@@ -0,0 +1,16 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-join/hints-indexnl-params/hints-indexnl-params-3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-join/hints-indexnl-params/hints-indexnl-params-3.plan
new file mode 100644
index 0000000..bd6d8cf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-join/hints-indexnl-params/hints-indexnl-params-3.plan
@@ -0,0 +1,25 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$37(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$37(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.tenk2.tenk2) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$42(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.tenk2.idx_tenk2_1k) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.tenk1.tenk1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-join/hints-indexnl-params/hints-indexnl-params-5.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-join/hints-indexnl-params/hints-indexnl-params-5.plan
new file mode 100644
index 0000000..bd6d8cf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-join/hints-indexnl-params/hints-indexnl-params-5.plan
@@ -0,0 +1,25 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$37(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$37(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.tenk2.tenk2) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$42(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.tenk2.idx_tenk2_1k) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.tenk1.tenk1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-join/hints-indexnl-params/hints-indexnl-params-6.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-join/hints-indexnl-params/hints-indexnl-params-6.plan
new file mode 100644
index 0000000..b6b4ba0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-join/hints-indexnl-params/hints-indexnl-params-6.plan
@@ -0,0 +1,24 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$37(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$37(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$35][$$34] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$35] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.tenk2) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$34] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.tenk1.tenk1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-join/hints-indexnl-params/hints-indexnl-params-7.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-join/hints-indexnl-params/hints-indexnl-params-7.plan
new file mode 100644
index 0000000..bd6d8cf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-join/hints-indexnl-params/hints-indexnl-params-7.plan
@@ -0,0 +1,25 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$37(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$37(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.tenk2.tenk2) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$42(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.tenk2.idx_tenk2_1k) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.tenk1.tenk1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/btree-index-composite-key-04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/btree-index-composite-key-04.plan
new file mode 100644
index 0000000..ce1677d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/btree-index-composite-key-04.plan
@@ -0,0 +1,12 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$21(ASC) ] |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.employee) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-02.plan
new file mode 100644
index 0000000..a41ba60
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-02.plan
@@ -0,0 +1,12 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$59(ASC) ] |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds2) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-10.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-10.plan
new file mode 100644
index 0000000..753d23d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-10.plan
@@ -0,0 +1,12 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$48(ASC) ] |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds5) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-11.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-11.plan
new file mode 100644
index 0000000..753d23d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-11.plan
@@ -0,0 +1,12 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$48(ASC) ] |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds5) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-12.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-12.plan
new file mode 100644
index 0000000..753d23d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-12.plan
@@ -0,0 +1,12 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$48(ASC) ] |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds5) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-13.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-13.plan
new file mode 100644
index 0000000..f6647d4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-13.plan
@@ -0,0 +1,12 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$48(ASC) ] |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds6) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-14.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-14.plan
new file mode 100644
index 0000000..f6647d4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-14.plan
@@ -0,0 +1,12 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$48(ASC) ] |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds6) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-15.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-15.plan
new file mode 100644
index 0000000..f6647d4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-15.plan
@@ -0,0 +1,12 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$48(ASC) ] |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds6) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-20.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-20.plan
new file mode 100644
index 0000000..e5fd2d4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-20.plan
@@ -0,0 +1,12 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$58(ASC) ] |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds7) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-23.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-23.plan
new file mode 100644
index 0000000..e5fd2d4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-23.plan
@@ -0,0 +1,12 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$58(ASC) ] |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds7) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-24.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-24.plan
new file mode 100644
index 0000000..c587ffb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-24.plan
@@ -0,0 +1,12 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$20(ASC) ] |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds7) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-25.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-25.plan
new file mode 100644
index 0000000..e5fd2d4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-25.plan
@@ -0,0 +1,12 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$58(ASC) ] |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds7) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/disjunctive-predicate/disjunctive-predicate-1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/disjunctive-predicate/disjunctive-predicate-1.plan
new file mode 100644
index 0000000..61695a3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/disjunctive-predicate/disjunctive-predicate-1.plan
@@ -0,0 +1,12 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$20][$$21] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestSet) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/disjunctive-predicate/disjunctive-predicate-2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/disjunctive-predicate/disjunctive-predicate-2.plan
new file mode 100644
index 0000000..318f7d5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/disjunctive-predicate/disjunctive-predicate-2.plan
@@ -0,0 +1,15 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$19][$$21] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestSet) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-10.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-10.plan
new file mode 100644
index 0000000..c266c21
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-10.plan
@@ -0,0 +1,12 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$19(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$19(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.tenk) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-11.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-11.plan
new file mode 100644
index 0000000..a2a9642
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-11.plan
@@ -0,0 +1,18 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$23(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$23(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$20][$$22] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.tenk) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-12.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-12.plan
new file mode 100644
index 0000000..a2a9642
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-12.plan
@@ -0,0 +1,18 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$23(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$23(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$20][$$22] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.tenk) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-13.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-13.plan
new file mode 100644
index 0000000..c266c21
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-13.plan
@@ -0,0 +1,12 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$19(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$19(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.tenk) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-4.plan
new file mode 100644
index 0000000..c266c21
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-4.plan
@@ -0,0 +1,12 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$19(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$19(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.tenk) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-6.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-6.plan
new file mode 100644
index 0000000..86e54c9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-6.plan
@@ -0,0 +1,12 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$23(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$23(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.tenk) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-7.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-7.plan
new file mode 100644
index 0000000..86e54c9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-7.plan
@@ -0,0 +1,12 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$23(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$23(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.tenk) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-8.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-8.plan
new file mode 100644
index 0000000..86e54c9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-8.plan
@@ -0,0 +1,12 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$23(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$23(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.tenk) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-9.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-9.plan
new file mode 100644
index 0000000..c266c21
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-9.plan
@@ -0,0 +1,12 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$19(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$19(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.tenk) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-18.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-18.plan
new file mode 100644
index 0000000..c266c21
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-18.plan
@@ -0,0 +1,12 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$19(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$19(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.tenk) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-19.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-19.plan
new file mode 100644
index 0000000..c266c21
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-19.plan
@@ -0,0 +1,12 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$19(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$19(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.tenk) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-7.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-7.plan
new file mode 100644
index 0000000..86e54c9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-7.plan
@@ -0,0 +1,12 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$23(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$23(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.tenk) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-composite-key-03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-composite-key-03.plan
new file mode 100644
index 0000000..e431bd0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-composite-key-03.plan
@@ -0,0 +1,8 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.employee) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-33.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-33.plan
new file mode 100644
index 0000000..5454dab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-33.plan
@@ -0,0 +1,8 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.testdst) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-34.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-34.plan
new file mode 100644
index 0000000..5454dab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-34.plan
@@ -0,0 +1,8 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.testdst) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-35.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-35.plan
new file mode 100644
index 0000000..5454dab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-35.plan
@@ -0,0 +1,8 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.testdst) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-36.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-36.plan
new file mode 100644
index 0000000..5454dab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-36.plan
@@ -0,0 +1,8 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.testdst) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-40.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-40.plan
new file mode 100644
index 0000000..5454dab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-40.plan
@@ -0,0 +1,8 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.testdst) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-42.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-42.plan
new file mode 100644
index 0000000..5454dab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-42.plan
@@ -0,0 +1,8 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.testdst) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-43.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-43.plan
new file mode 100644
index 0000000..5454dab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-43.plan
@@ -0,0 +1,8 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.testdst) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-44.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-44.plan
new file mode 100644
index 0000000..5454dab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-44.plan
@@ -0,0 +1,8 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.testdst) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-45.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-45.plan
new file mode 100644
index 0000000..5454dab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-45.plan
@@ -0,0 +1,8 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.testdst) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-46.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-46.plan
new file mode 100644
index 0000000..5454dab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-46.plan
@@ -0,0 +1,8 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.testdst) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-47.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-47.plan
new file mode 100644
index 0000000..124330a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-47.plan
@@ -0,0 +1,10 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.testdst) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-48.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-48.plan
new file mode 100644
index 0000000..124330a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-48.plan
@@ -0,0 +1,10 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.testdst) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-49.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-49.plan
new file mode 100644
index 0000000..5454dab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-49.plan
@@ -0,0 +1,8 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.testdst) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-51.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-51.plan
new file mode 100644
index 0000000..124330a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-51.plan
@@ -0,0 +1,10 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.testdst) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-52.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-52.plan
new file mode 100644
index 0000000..124330a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-52.plan
@@ -0,0 +1,10 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.testdst) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-53.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-53.plan
new file mode 100644
index 0000000..124330a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-53.plan
@@ -0,0 +1,10 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.testdst) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-54.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-54.plan
new file mode 100644
index 0000000..5454dab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-54.plan
@@ -0,0 +1,8 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.testdst) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-55.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-55.plan
new file mode 100644
index 0000000..5454dab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-55.plan
@@ -0,0 +1,8 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.testdst) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-56.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-56.plan
new file mode 100644
index 0000000..5454dab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-56.plan
@@ -0,0 +1,8 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.testdst) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-57.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-57.plan
new file mode 100644
index 0000000..5454dab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-57.plan
@@ -0,0 +1,8 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.testdst) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-58.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-58.plan
new file mode 100644
index 0000000..124330a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-58.plan
@@ -0,0 +1,10 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.testdst) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-59.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-59.plan
new file mode 100644
index 0000000..124330a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-59.plan
@@ -0,0 +1,10 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.testdst) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-60.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-60.plan
new file mode 100644
index 0000000..5454dab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-60.plan
@@ -0,0 +1,8 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.testdst) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-61.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-61.plan
new file mode 100644
index 0000000..124330a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-61.plan
@@ -0,0 +1,10 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.testdst) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-62.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-62.plan
new file mode 100644
index 0000000..5454dab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-62.plan
@@ -0,0 +1,8 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.testdst) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-63.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-63.plan
new file mode 100644
index 0000000..5454dab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-63.plan
@@ -0,0 +1,8 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.testdst) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-68.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-68.plan
new file mode 100644
index 0000000..4cd09cb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-68.plan
@@ -0,0 +1,14 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$27(ASC), $$28(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$27(ASC), $$28(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.Orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-68_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-68_ps.plan
new file mode 100644
index 0000000..c7a7d7c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-68_ps.plan
@@ -0,0 +1,33 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$27(ASC), $$28(ASC)] |PARTITIONED|
+ -- RANGE_PARTITION_EXCHANGE [$$27(ASC), $$28(ASC)] |PARTITIONED|
+ -- FORWARD |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.Orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.Orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-sidx-idxonly-01-disable-idxonly.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-sidx-idxonly-01-disable-idxonly.plan
new file mode 100644
index 0000000..8dd4216
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-sidx-idxonly-01-disable-idxonly.plan
@@ -0,0 +1,12 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$20(ASC) ] |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.MyData) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-sidx-idxonly-01-disable-idxonly_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-sidx-idxonly-01-disable-idxonly_ps.plan
new file mode 100644
index 0000000..3ebae2c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-sidx-idxonly-01-disable-idxonly_ps.plan
@@ -0,0 +1,33 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
+ -- RANGE_PARTITION_EXCHANGE [$$20(ASC)] |PARTITIONED|
+ -- FORWARD |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.MyData) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.MyData) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-sidx-idxonly-01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-sidx-idxonly-01.plan
new file mode 100644
index 0000000..8dd4216
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-sidx-idxonly-01.plan
@@ -0,0 +1,12 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$20(ASC) ] |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.MyData) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-sidx-idxonly-01_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-sidx-idxonly-01_ps.plan
new file mode 100644
index 0000000..3ebae2c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-sidx-idxonly-01_ps.plan
@@ -0,0 +1,33 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
+ -- RANGE_PARTITION_EXCHANGE [$$20(ASC)] |PARTITIONED|
+ -- FORWARD |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.MyData) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.MyData) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-sidx-idxonly-10.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-sidx-idxonly-10.plan
new file mode 100644
index 0000000..33aec0b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-sidx-idxonly-10.plan
@@ -0,0 +1,15 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (twitter.ds_tweet) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-ternary-inlj/query4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-ternary-inlj/query4.plan
new file mode 100644
index 0000000..9b920b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-ternary-inlj/query4.plan
@@ -0,0 +1,32 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (tpcds.customer_address.customer_address) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$78(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$78] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (tpcds.customer_demographics.customer_demographics) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$76(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$76] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$61][$$74] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpcds.customer) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q16.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q16.plan
new file mode 100644
index 0000000..2214a1e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q16.plan
@@ -0,0 +1,65 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$131(DESC) ] |PARTITIONED|
+ -- STABLE_SORT [$$131(DESC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$121, $$122, $$123] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- MICRO_PRE_SORTED_DISTINCT_BY |LOCAL|
+ -- MICRO_STABLE_SORT [$$142(ASC)] |LOCAL|
+ -- ASSIGN |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$121(ASC), $$122(ASC), $$123(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$121, $$122, $$123] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$125, $$126] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$125(ASC), $$126(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- NESTED_LOOP |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$130][$$129] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.stock) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.item) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.supplier) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q2.plan
new file mode 100644
index 0000000..6d36713
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q2.plan
@@ -0,0 +1,162 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_LIMIT |UNPARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$274(ASC), $$275(ASC), $$240(ASC) ] |PARTITIONED|
+ -- STREAM_LIMIT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [topK: 100] [$$274(ASC), $$275(ASC), $$240(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$252][$$253] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$252] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$259][$$260] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$259] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$265][$$242] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$265] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$240, $$254][$$241, $$263] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$240, $$254] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$s_i_id][$$240] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- SORT_GROUP_BY[$$285] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- HASH_PARTITION_EXCHANGE [$$285] |PARTITIONED|
+ -- SORT_GROUP_BY[$$239] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$255][$$256] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$255] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$257][$$269] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$257] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$261][$$270] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$261] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.stock) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$270] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.supplier) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$269] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.nation) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$256] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.region) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$240] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.item) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$241, $$263] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.stock) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$242] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.supplier) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$260] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.nation) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$256] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.region) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q21.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q21.plan
new file mode 100644
index 0000000..555eb0b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q21.plan
@@ -0,0 +1,116 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_LIMIT |UNPARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$su_name(ASC) ] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_LIMIT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- SORT_GROUP_BY[$$378] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- HASH_PARTITION_EXCHANGE [$$378] |PARTITIONED|
+ -- SORT_GROUP_BY[$$su_name] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- SORT_GROUP_BY[$$369, $$370, $$371, $$372, $$373, $$374, $$375, $$376] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- HASH_PARTITION_EXCHANGE [$$369, $$370, $$371, $$372, $$373, $$374, $$375, $$376] |PARTITIONED|
+ -- SORT_GROUP_BY[$$331, $$357, $$356, $$333, $$334, $$330, $$329, $$358] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- NESTED_LOOP |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$333][$$344] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.nation) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$334][$$345] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.supplier) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$330, $$329][$$331, $$341] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.stock) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q5.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q5.plan
new file mode 100644
index 0000000..95ff63d3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q5.plan
@@ -0,0 +1,97 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$#2(DESC) ] |PARTITIONED|
+ -- STABLE_SORT [$#2(DESC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- SORT_GROUP_BY[$$291] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- HASH_PARTITION_EXCHANGE [$$291] |PARTITIONED|
+ -- SORT_GROUP_BY[$$275] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$257][$$256] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$257] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.region) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$256] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$274, $$268][$$266, $$269] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$274, $$268] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$258][$$274] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$258] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$260, $$254, $$263][$$276, $$277, $$278] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$260, $$254, $$263] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$245, $$246][$$254, $$281] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$245, $$246] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.stock) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$254, $$281] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$276, $$277, $$278] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.customer) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$274] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.nation) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$266, $$269] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.supplier) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q7.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q7.plan
new file mode 100644
index 0000000..98a1725
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q7.plan
@@ -0,0 +1,103 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$su_nationkey(ASC), $#1(ASC), $#2(ASC) ] |PARTITIONED|
+ -- SORT_GROUP_BY[$$325, $$326, $$327] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- HASH_PARTITION_EXCHANGE [$$325, $$326, $$327] |PARTITIONED|
+ -- SORT_GROUP_BY[$$277, $$273, $$274] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$288][$$304] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$288] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$277][$$303] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$277] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$299][$$300] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$299] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$291, $$293, $$295][$$305, $$306, $$307] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$291, $$293, $$295] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$275, $$276][$$310, $$311] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$275, $$276] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.stock) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$310, $$311] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$305, $$306, $$307] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.customer) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$300] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.supplier) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$303] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.nation) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$303] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.nation) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q8.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q8.plan
new file mode 100644
index 0000000..9d2969e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q8.plan
@@ -0,0 +1,127 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$#1(ASC) ] |PARTITIONED|
+ -- SORT_GROUP_BY[$$349] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- HASH_PARTITION_EXCHANGE [$$349] |PARTITIONED|
+ -- SORT_GROUP_BY[$$294] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$325][$$326] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$325] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$320][$$341] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$320] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$308][$$309] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$308] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$311][$$331] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$311] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$313, $$315, $$317][$$333, $$334, $$332] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$313, $$315, $$317] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$306][$$324] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$306] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$296, $$295][$$306, $$336] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$296, $$295] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.stock) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$306, $$336] |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$324] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.item) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$333, $$334, $$332] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.customer) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$331] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.nation) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$309] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.region) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$341] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.supplier) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$326] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.nation) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q9.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q9.plan
new file mode 100644
index 0000000..9eafc6f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q9.plan
@@ -0,0 +1,80 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$n_name(ASC), $#1(DESC) ] |PARTITIONED|
+ -- SORT_GROUP_BY[$$229, $$230] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- HASH_PARTITION_EXCHANGE [$$229, $$230] |PARTITIONED|
+ -- SORT_GROUP_BY[$$225, $$198] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$213][$$214] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$213] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$210][$$224] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$210] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$206][$$207] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$206] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$200, $$199][$$206, $$217] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$200, $$199] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.stock) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$206, $$217] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$207] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.item) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$224] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.supplier) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$214] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.nation) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/extract-common-operators/extract-common-operators.01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/extract-common-operators/extract-common-operators.01.plan
new file mode 100644
index 0000000..29aafbf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/extract-common-operators/extract-common-operators.01.plan
@@ -0,0 +1,90 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$405(ASC), $$406(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$405(ASC), $$406(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$414][$$410] |PARTITIONED|
+ -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$405(ASC), $$406(ASC)] HASH:[$$414] |PARTITIONED|
+ -- STABLE_SORT [$$405(ASC), $$406(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$414][$$408] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$414] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (Metadata.Synonym) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$408] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$408][$$ds_name] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- RANDOM_PARTITION_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (Metadata.Dataset.Dataset) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$410] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$428][$$412] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$428] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$410][$$syn_name] |PARTITIONED|
+ -- RANDOM_PARTITION_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (Metadata.Synonym) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$412] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$412][$$ds_name] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- RANDOM_PARTITION_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (Metadata.Dataset.Dataset) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/filter/inverted-btree-search-return-optional-field.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/filter/inverted-btree-search-return-optional-field.plan
new file mode 100644
index 0000000..6993f25
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/filter/inverted-btree-search-return-optional-field.plan
@@ -0,0 +1,26 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_LIMIT |UNPARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$39(DESC) ] |PARTITIONED|
+ -- STREAM_LIMIT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [topK: 2147483647] [$$39(DESC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$26][$$41] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (twitter.ds_tweet) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/filter_on_meta_with_idx_2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/filter_on_meta_with_idx_2.plan
new file mode 100644
index 0000000..05da710
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/filter_on_meta_with_idx_2.plan
@@ -0,0 +1,25 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$37][$$38] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$37] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.KVStore) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$38] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.OfficerLocations) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/group-by/listify-3.1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/group-by/listify-3.1.plan
new file mode 100644
index 0000000..ec40853
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/group-by/listify-3.1.plan
@@ -0,0 +1,43 @@
+-- DISTRIBUTE_RESULT |LOCAL|
+ -- ONE_TO_ONE_EXCHANGE |LOCAL|
+ -- STREAM_PROJECT |LOCAL|
+ -- ASSIGN |LOCAL|
+ -- AGGREGATE |LOCAL|
+ -- AGGREGATE |LOCAL|
+ -- STREAM_PROJECT |LOCAL|
+ -- ASSIGN |LOCAL|
+ -- ONE_TO_ONE_EXCHANGE |LOCAL|
+ -- NESTED_LOOP |LOCAL|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |LOCAL|
+ -- NESTED_LOOP |LOCAL|
+ -- ONE_TO_ONE_EXCHANGE |LOCAL|
+ -- STREAM_PROJECT |LOCAL|
+ -- ASSIGN |LOCAL|
+ -- STREAM_PROJECT |LOCAL|
+ -- ASSIGN |LOCAL|
+ -- ONE_TO_ONE_EXCHANGE |LOCAL|
+ -- REPLICATE |LOCAL|
+ -- ONE_TO_ONE_EXCHANGE |LOCAL|
+ -- AGGREGATE |LOCAL|
+ -- STREAM_PROJECT |LOCAL|
+ -- ASSIGN |LOCAL|
+ -- AGGREGATE |LOCAL|
+ -- AGGREGATE |LOCAL|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |LOCAL|
+ -- STREAM_PROJECT |LOCAL|
+ -- ASSIGN |LOCAL|
+ -- ONE_TO_ONE_EXCHANGE |LOCAL|
+ -- REPLICATE |LOCAL|
+ -- ONE_TO_ONE_EXCHANGE |LOCAL|
+ -- AGGREGATE |LOCAL|
+ -- STREAM_PROJECT |LOCAL|
+ -- ASSIGN |LOCAL|
+ -- AGGREGATE |LOCAL|
+ -- AGGREGATE |LOCAL|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/hash-join-with-redundant-variable/hash-join-with-redundant-variable.1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/hash-join-with-redundant-variable/hash-join-with-redundant-variable.1.plan
new file mode 100644
index 0000000..65df3c2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/hash-join-with-redundant-variable/hash-join-with-redundant-variable.1.plan
@@ -0,0 +1,74 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$130][$$129] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$130] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.region) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$129] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$127, $$131][$$128, $$132] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$127, $$131] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.lineitem) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$128, $$132] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$126][$$125] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$126] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$125] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$117][$$116] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$117][$$115] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$117] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.nation) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$115] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.customer) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$116] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.supplier) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/inverted-index-basic/ngram-contains.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/inverted-index-basic/ngram-contains.plan
new file mode 100644
index 0000000..8a9e3fa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/inverted-index-basic/ngram-contains.plan
@@ -0,0 +1,9 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$15(ASC) ] |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.DBLP) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/inverted-index-basic/ngram-contains_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/inverted-index-basic/ngram-contains_ps.plan
new file mode 100644
index 0000000..93ca5bd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/inverted-index-basic/ngram-contains_ps.plan
@@ -0,0 +1,28 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$15(ASC)] |PARTITIONED|
+ -- RANGE_PARTITION_EXCHANGE [$$15(ASC)] |PARTITIONED|
+ -- FORWARD |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.DBLP) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.DBLP) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/joins/nlj_partitioning_property_1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/joins/nlj_partitioning_property_1.plan
new file mode 100644
index 0000000..91a6aca
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/joins/nlj_partitioning_property_1.plan
@@ -0,0 +1,31 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- NESTED_LOOP |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$76][$$78] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.Supplier) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.Partsupp) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.Part) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/joins/nlj_partitioning_property_2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/joins/nlj_partitioning_property_2.plan
new file mode 100644
index 0000000..c1554e2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/joins/nlj_partitioning_property_2.plan
@@ -0,0 +1,29 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- NESTED_LOOP |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- NESTED_LOOP |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.Supplier) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.Partsupp) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.Part) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/meta/indexes_on_dataset_with_meta_05.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/meta/indexes_on_dataset_with_meta_05.plan
new file mode 100644
index 0000000..6013ad8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/meta/indexes_on_dataset_with_meta_05.plan
@@ -0,0 +1,18 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$29(ASC) ] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.DS2.DS2) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$34(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$34] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.DS1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/meta/indexes_on_dataset_with_meta_08.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/meta/indexes_on_dataset_with_meta_08.plan
new file mode 100644
index 0000000..2561ec6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/meta/indexes_on_dataset_with_meta_08.plan
@@ -0,0 +1,10 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.DS1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/multipart-dataverse/index/index-01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/multipart-dataverse/index/index-01.plan
new file mode 100644
index 0000000..a794105
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/multipart-dataverse/index/index-01.plan
@@ -0,0 +1,8 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (x.y.employee) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/multipart-dataverse/index/index-02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/multipart-dataverse/index/index-02.plan
new file mode 100644
index 0000000..a794105
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/multipart-dataverse/index/index-02.plan
@@ -0,0 +1,8 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (x.y.employee) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-index/btree-index-join/ASTERIXDB-2199.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-index/btree-index-join/ASTERIXDB-2199.plan
new file mode 100644
index 0000000..46dbf81
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-index/btree-index-join/ASTERIXDB-2199.plan
@@ -0,0 +1,15 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$36][$$37] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (Facebook.Friendship) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (Facebook.Friendship) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-index/inverted-index-basic/ngram-contains_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-index/inverted-index-basic/ngram-contains_ps.plan
new file mode 100644
index 0000000..5a051c1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-index/inverted-index-basic/ngram-contains_ps.plan
@@ -0,0 +1,28 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$17(ASC)] |PARTITIONED|
+ -- RANGE_PARTITION_EXCHANGE [$$17(ASC)] |PARTITIONED|
+ -- FORWARD |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.DBLP) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.DBLP) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/05.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/05.plan
new file mode 100644
index 0000000..9ae6c48
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/05.plan
@@ -0,0 +1,30 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$48(ASC), $$49(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$48(ASC), $$49(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$45][$$46] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$45] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestOpen1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$46] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestOpen2) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/01.plan
new file mode 100644
index 0000000..5454dab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/01.plan
@@ -0,0 +1,8 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.testdst) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/02.plan
new file mode 100644
index 0000000..5454dab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/02.plan
@@ -0,0 +1,8 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.testdst) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/03.plan
new file mode 100644
index 0000000..d874441
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/03.plan
@@ -0,0 +1,15 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$21(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$21(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/04.plan
new file mode 100644
index 0000000..58cd521
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/04.plan
@@ -0,0 +1,17 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$24(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$24(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/05.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/05.plan
new file mode 100644
index 0000000..58cd521
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/05.plan
@@ -0,0 +1,17 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$24(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$24(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/06.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/06.plan
new file mode 100644
index 0000000..58cd521
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/06.plan
@@ -0,0 +1,17 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$24(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$24(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/07.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/07.plan
new file mode 100644
index 0000000..58cd521
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/07.plan
@@ -0,0 +1,17 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$24(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$24(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/08.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/08.plan
new file mode 100644
index 0000000..58cd521
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/08.plan
@@ -0,0 +1,17 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$24(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$24(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/09.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/09.plan
new file mode 100644
index 0000000..58cd521
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/09.plan
@@ -0,0 +1,17 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$24(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$24(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/10.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/10.plan
new file mode 100644
index 0000000..58cd521
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/10.plan
@@ -0,0 +1,17 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$24(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$24(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/11.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/11.plan
new file mode 100644
index 0000000..bbd2dba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/11.plan
@@ -0,0 +1,18 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$29(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$29(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/12.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/12.plan
new file mode 100644
index 0000000..fde8db4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/12.plan
@@ -0,0 +1,17 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$29(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$29(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/inverted-index-basic/ngram-contains_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/inverted-index-basic/ngram-contains_ps.plan
new file mode 100644
index 0000000..5a051c1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/inverted-index-basic/ngram-contains_ps.plan
@@ -0,0 +1,28 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$17(ASC)] |PARTITIONED|
+ -- RANGE_PARTITION_EXCHANGE [$$17(ASC)] |PARTITIONED|
+ -- FORWARD |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.DBLP) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.DBLP) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-enforced/inverted-index-basic/ngram-contains_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-enforced/inverted-index-basic/ngram-contains_ps.plan
new file mode 100644
index 0000000..93ca5bd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-enforced/inverted-index-basic/ngram-contains_ps.plan
@@ -0,0 +1,28 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$15(ASC)] |PARTITIONED|
+ -- RANGE_PARTITION_EXCHANGE [$$15(ASC)] |PARTITIONED|
+ -- FORWARD |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.DBLP) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.DBLP) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-composite-key-non-enforced/btree-composite-key-non-enforced-03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-composite-key-non-enforced/btree-composite-key-non-enforced-03.plan
new file mode 100644
index 0000000..f1a1616
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-composite-key-non-enforced/btree-composite-key-non-enforced-03.plan
@@ -0,0 +1,10 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-composite-key-non-enforced/btree-composite-key-non-enforced-04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-composite-key-non-enforced/btree-composite-key-non-enforced-04.plan
new file mode 100644
index 0000000..f1a1616
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-composite-key-non-enforced/btree-composite-key-non-enforced-04.plan
@@ -0,0 +1,10 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-07.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-07.plan
new file mode 100644
index 0000000..8d30fd0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-07.plan
@@ -0,0 +1,26 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$40(ASC), $$41(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$40(ASC), $$41(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$37][$$38] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$37] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestOpen1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$38] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestOpen2) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-07_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-07_ps.plan
new file mode 100644
index 0000000..6c25f6d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-07_ps.plan
@@ -0,0 +1,56 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$40(ASC), $$41(ASC)] |PARTITIONED|
+ -- RANGE_PARTITION_EXCHANGE [$$40(ASC), $$41(ASC)] |PARTITIONED|
+ -- FORWARD |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$37][$$38] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$37] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestOpen1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$38] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestOpen2) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$37][$$38] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$37] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestOpen1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$38] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestOpen2) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-04.plan
new file mode 100644
index 0000000..0b23e02
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-04.plan
@@ -0,0 +1,15 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$20(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-04_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-04_ps.plan
new file mode 100644
index 0000000..8d94902
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-04_ps.plan
@@ -0,0 +1,34 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
+ -- RANGE_PARTITION_EXCHANGE [$$20(ASC)] |PARTITIONED|
+ -- FORWARD |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-05.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-05.plan
new file mode 100644
index 0000000..0b23e02
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-05.plan
@@ -0,0 +1,15 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$20(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-05_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-05_ps.plan
new file mode 100644
index 0000000..8d94902
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-05_ps.plan
@@ -0,0 +1,34 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
+ -- RANGE_PARTITION_EXCHANGE [$$20(ASC)] |PARTITIONED|
+ -- FORWARD |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-06.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-06.plan
new file mode 100644
index 0000000..0b23e02
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-06.plan
@@ -0,0 +1,15 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$20(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-06_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-06_ps.plan
new file mode 100644
index 0000000..8d94902
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-06_ps.plan
@@ -0,0 +1,34 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
+ -- RANGE_PARTITION_EXCHANGE [$$20(ASC)] |PARTITIONED|
+ -- FORWARD |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-07.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-07.plan
new file mode 100644
index 0000000..0b23e02
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-07.plan
@@ -0,0 +1,15 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$20(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-07_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-07_ps.plan
new file mode 100644
index 0000000..8d94902
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-07_ps.plan
@@ -0,0 +1,34 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
+ -- RANGE_PARTITION_EXCHANGE [$$20(ASC)] |PARTITIONED|
+ -- FORWARD |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-08.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-08.plan
new file mode 100644
index 0000000..0b23e02
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-08.plan
@@ -0,0 +1,15 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$20(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-08_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-08_ps.plan
new file mode 100644
index 0000000..8d94902
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-08_ps.plan
@@ -0,0 +1,34 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
+ -- RANGE_PARTITION_EXCHANGE [$$20(ASC)] |PARTITIONED|
+ -- FORWARD |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-09.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-09.plan
new file mode 100644
index 0000000..0b23e02
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-09.plan
@@ -0,0 +1,15 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$20(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-09_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-09_ps.plan
new file mode 100644
index 0000000..8d94902
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-09_ps.plan
@@ -0,0 +1,34 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
+ -- RANGE_PARTITION_EXCHANGE [$$20(ASC)] |PARTITIONED|
+ -- FORWARD |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-10.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-10.plan
new file mode 100644
index 0000000..0b23e02
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-10.plan
@@ -0,0 +1,15 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$20(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-105.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-105.plan
new file mode 100644
index 0000000..c1080c7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-105.plan
@@ -0,0 +1,15 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$24(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$24(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-105_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-105_ps.plan
new file mode 100644
index 0000000..be79cf5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-105_ps.plan
@@ -0,0 +1,34 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$24(ASC)] |PARTITIONED|
+ -- RANGE_PARTITION_EXCHANGE [$$24(ASC)] |PARTITIONED|
+ -- FORWARD |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-10_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-10_ps.plan
new file mode 100644
index 0000000..8d94902
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-10_ps.plan
@@ -0,0 +1,34 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
+ -- RANGE_PARTITION_EXCHANGE [$$20(ASC)] |PARTITIONED|
+ -- FORWARD |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-11.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-11.plan
new file mode 100644
index 0000000..748b085
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-11.plan
@@ -0,0 +1,16 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$23(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$23(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-11_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-11_ps.plan
new file mode 100644
index 0000000..35ffdd2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-11_ps.plan
@@ -0,0 +1,36 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$23(ASC)] |PARTITIONED|
+ -- RANGE_PARTITION_EXCHANGE [$$23(ASC)] |PARTITIONED|
+ -- FORWARD |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive-open_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive-open_01.plan
new file mode 100644
index 0000000..2326c55
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive-open_01.plan
@@ -0,0 +1,25 @@
+-- SINK |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BULKLOAD |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$2(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$2] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (loadable_dv.loadable_ds) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$23(ASC) ] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.Orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive-open_01_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive-open_01_ps.plan
new file mode 100644
index 0000000..4bdc66b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive-open_01_ps.plan
@@ -0,0 +1,46 @@
+-- SINK |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BULKLOAD |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$2(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$2] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (loadable_dv.loadable_ds) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$23(ASC)] |PARTITIONED|
+ -- RANGE_PARTITION_EXCHANGE [$$23(ASC)] |PARTITIONED|
+ -- FORWARD |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.Orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.Orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive-open_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive-open_02.plan
new file mode 100644
index 0000000..eb1d57c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive-open_02.plan
@@ -0,0 +1,25 @@
+-- SINK |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BULKLOAD |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$2(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$2] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (loadable_dv.loadable_ds) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$25(ASC) ] |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.Orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive-open_02_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive-open_02_ps.plan
new file mode 100644
index 0000000..dce703e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive-open_02_ps.plan
@@ -0,0 +1,46 @@
+-- SINK |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BULKLOAD |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$2(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$2] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (loadable_dv.loadable_ds) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$25(ASC)] |PARTITIONED|
+ -- RANGE_PARTITION_EXCHANGE [$$25(ASC)] |PARTITIONED|
+ -- FORWARD |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.Orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.Orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive_01.plan
new file mode 100644
index 0000000..2326c55
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive_01.plan
@@ -0,0 +1,25 @@
+-- SINK |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BULKLOAD |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$2(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$2] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (loadable_dv.loadable_ds) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$23(ASC) ] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.Orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive_01_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive_01_ps.plan
new file mode 100644
index 0000000..4bdc66b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive_01_ps.plan
@@ -0,0 +1,46 @@
+-- SINK |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BULKLOAD |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$2(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$2] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (loadable_dv.loadable_ds) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$23(ASC)] |PARTITIONED|
+ -- RANGE_PARTITION_EXCHANGE [$$23(ASC)] |PARTITIONED|
+ -- FORWARD |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.Orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.Orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive_02.plan
new file mode 100644
index 0000000..eb1d57c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive_02.plan
@@ -0,0 +1,25 @@
+-- SINK |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BULKLOAD |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$2(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$2] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (loadable_dv.loadable_ds) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$25(ASC) ] |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.Orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive_02_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive_02_ps.plan
new file mode 100644
index 0000000..dce703e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive_02_ps.plan
@@ -0,0 +1,46 @@
+-- SINK |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BULKLOAD |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$2(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$2] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (loadable_dv.loadable_ds) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$25(ASC)] |PARTITIONED|
+ -- RANGE_PARTITION_EXCHANGE [$$25(ASC)] |PARTITIONED|
+ -- FORWARD |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.Orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.Orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/q08_group_by.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/q08_group_by.plan
new file mode 100644
index 0000000..1e558ec
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/q08_group_by.plan
@@ -0,0 +1,78 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$230][$$212] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$230] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$216][$$211] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$216] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$218][$$210] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$218] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$221][$$209] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$221] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$206][$$208] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$206] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$205][$$233] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (q08_group_by.Supplier) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$233] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (q08_group_by.LineItem) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (q08_group_by.Orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (q08_group_by.Customer) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (q08_group_by.Nation) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (q08_group_by.Region) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (q08_group_by.Part) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/q09_group_by.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/q09_group_by.plan
new file mode 100644
index 0000000..26a56eb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/q09_group_by.plan
@@ -0,0 +1,55 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$157][$$151] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$157] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$154][$$150] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$154] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$169, $$154][$$148, $$149] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$169, $$154] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$147][$$169] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (q09_group_by.Part) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$169] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (q09_group_by.LineItem) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (q09_group_by.Partsupp) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (q09_group_by.Supplier) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (q09_group_by.Nation) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/query-ASTERIXDB-3334.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/query-ASTERIXDB-3334.plan
new file mode 100644
index 0000000..bbb94e3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/query-ASTERIXDB-3334.plan
@@ -0,0 +1,61 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- SORT_GROUP_BY[$$1112] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- HASH_PARTITION_EXCHANGE [$$1112] |PARTITIONED|
+ -- SORT_GROUP_BY[$$1113] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$1113, $$1114][$$Employee Name, $$Call Center Region] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$1114, $$1113] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection0) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- SORT_GROUP_BY[$$1109, $$1110] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- HASH_PARTITION_EXCHANGE [$$1109, $$1110] |PARTITIONED|
+ -- SORT_GROUP_BY[$$1117, $$1116] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection0) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/query_issue3316.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/query_issue3316.plan
new file mode 100644
index 0000000..8b22913
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/query_issue3316.plan
@@ -0,0 +1,998 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$186(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$186] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$221] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$221(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$221][$$382] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$187] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$187][$$234] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection0) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$234] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$120][$$193] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$219, $$188] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$219(ASC), $$188(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$219, $$188][$$248, $$249] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$219, $$188] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$217][$$201] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$439] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection0) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$201] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$232, $$229] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$232(ASC), $$229(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$232, $$229][$$246, $$243] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$419, $$420] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$428][$$427] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$439] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection0) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$427] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$246, $$243] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$237][$$236] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$237] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$204][$$237] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$204] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$129][$$192] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$439] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection0) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$442] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$382] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$156][$$196] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$159][$$195] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$250, $$190] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$250(ASC), $$190(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$250, $$190][$$453, $$454] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$250, $$190] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$252][$$208] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$252] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$251] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$251][$$255] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection0) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$255] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$260][$$259] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$262, $$264] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$262(ASC), $$264(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$262, $$264][$$272, $$273] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$403, $$405] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$408][$$409] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$439] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection0) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$409] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$278, $$279] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$278(ASC), $$279(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$278, $$279][$$288, $$289] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$419, $$420] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$428][$$427] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$439] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection0) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$427] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$288, $$289] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$298][$$301] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$298] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$297][$$298] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$297] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$296][$$295] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$439] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection0) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$442] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$208] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$321, $$320] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$321(ASC), $$320(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$321, $$320][$$392, $$391] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$321, $$320] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$314][$$313] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$314] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$322] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$322][$$325] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection0) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$325] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$330][$$329] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$332, $$334] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$332(ASC), $$334(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$332, $$334][$$342, $$343] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$403, $$405] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$408][$$409] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$439] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection0) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$409] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$348, $$349] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$348(ASC), $$349(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$348, $$349][$$358, $$359] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$419, $$420] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$428][$$427] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$439] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection0) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$427] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$358, $$359] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$368][$$371] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$368] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$367][$$368] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$367] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$366][$$365] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$439] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection0) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$442] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$313] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$392, $$391] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$166][$$194] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$385][$$210] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$385][$$384] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$385] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$393] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$393][$$396] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection0) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$396] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$401][$$400] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$403, $$405] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$403(ASC), $$405(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$403, $$405][$$413, $$414] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$403, $$405] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$408][$$409] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$439] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection0) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$409] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$419, $$420] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$419(ASC), $$420(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$419, $$420][$$429, $$430] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$419, $$420] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$428][$$427] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$439] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection0) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$427] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$429, $$430] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$439][$$442] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$439] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$438][$$439] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$438] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$437][$$436] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$439] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection0) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$442] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$384] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$210] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.collection1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/query_issue849.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/query_issue849.plan
new file mode 100644
index 0000000..0f1879f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/query_issue849.plan
@@ -0,0 +1,29 @@
+-- COMMIT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- INSERT_DELETE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$3] |PARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
+-- COMMIT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- INSERT_DELETE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$3] |PARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.s.s) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$52(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$52] |PARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index-geometry/rtree-sidx-idxonly-01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index-geometry/rtree-sidx-idxonly-01.plan
new file mode 100644
index 0000000..7cd72c2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index-geometry/rtree-sidx-idxonly-01.plan
@@ -0,0 +1,8 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (IndexGeoJSON.Geometries) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index-geometry/rtree-sidx-idxonly-02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index-geometry/rtree-sidx-idxonly-02.plan
new file mode 100644
index 0000000..7cd72c2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index-geometry/rtree-sidx-idxonly-02.plan
@@ -0,0 +1,8 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (IndexGeoJSON.Geometries) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index-geometry/rtree-sidx-idxonly-03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index-geometry/rtree-sidx-idxonly-03.plan
new file mode 100644
index 0000000..7cd72c2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index-geometry/rtree-sidx-idxonly-03.plan
@@ -0,0 +1,8 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (IndexGeoJSON.Geometries) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index-geometry/rtree-sidx-idxonly-04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index-geometry/rtree-sidx-idxonly-04.plan
new file mode 100644
index 0000000..7cd72c2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index-geometry/rtree-sidx-idxonly-04.plan
@@ -0,0 +1,8 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (IndexGeoJSON.Geometries) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index-geometry/rtree-sidx-idxonly-05.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index-geometry/rtree-sidx-idxonly-05.plan
new file mode 100644
index 0000000..7cd72c2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index-geometry/rtree-sidx-idxonly-05.plan
@@ -0,0 +1,8 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (IndexGeoJSON.Geometries) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index-geometry/rtree-sidx-idxonly-06.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index-geometry/rtree-sidx-idxonly-06.plan
new file mode 100644
index 0000000..7cd72c2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index-geometry/rtree-sidx-idxonly-06.plan
@@ -0,0 +1,8 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (IndexGeoJSON.Geometries) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index-geometry/rtree-sidx-idxonly-07.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index-geometry/rtree-sidx-idxonly-07.plan
new file mode 100644
index 0000000..7cd72c2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index-geometry/rtree-sidx-idxonly-07.plan
@@ -0,0 +1,8 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (IndexGeoJSON.Geometries) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index/rtree-sidx-idxonly-01-disable-idxonly.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index/rtree-sidx-idxonly-01-disable-idxonly.plan
new file mode 100644
index 0000000..5b57d41
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index/rtree-sidx-idxonly-01-disable-idxonly.plan
@@ -0,0 +1,13 @@
+-- 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|
+ -- DATASOURCE_SCAN (test.Fragile_raw) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index/rtree-sidx-idxonly-01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index/rtree-sidx-idxonly-01.plan
new file mode 100644
index 0000000..5b57d41
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index/rtree-sidx-idxonly-01.plan
@@ -0,0 +1,13 @@
+-- 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|
+ -- DATASOURCE_SCAN (test.Fragile_raw) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/skip-index/skip-secondary-btree-index-2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/skip-index/skip-secondary-btree-index-2.plan
new file mode 100644
index 0000000..124330a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/skip-index/skip-secondary-btree-index-2.plan
@@ -0,0 +1,10 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.testdst) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/split-materialization.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/split-materialization.plan
new file mode 100644
index 0000000..34f5238
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/split-materialization.plan
@@ -0,0 +1,20 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$91(ASC), $$92(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$91(ASC), $$92(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TinySocial.FacebookUsers.FacebookUsers) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TinySocial.FacebookUsers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/split-materialization_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/split-materialization_ps.plan
new file mode 100644
index 0000000..2881c66
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/split-materialization_ps.plan
@@ -0,0 +1,45 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$91(ASC), $$92(ASC)] |PARTITIONED|
+ -- RANGE_PARTITION_EXCHANGE [$$91(ASC), $$92(ASC)] |PARTITIONED|
+ -- FORWARD |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TinySocial.FacebookUsers.FacebookUsers) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TinySocial.FacebookUsers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TinySocial.FacebookUsers.FacebookUsers) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TinySocial.FacebookUsers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/statement-params/statement-params-01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/statement-params/statement-params-01.plan
new file mode 100644
index 0000000..904a882
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/statement-params/statement-params-01.plan
@@ -0,0 +1,10 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/statement-params/statement-params-02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/statement-params/statement-params-02.plan
new file mode 100644
index 0000000..904a882
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/statement-params/statement-params-02.plan
@@ -0,0 +1,10 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/statement-params/statement-params-03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/statement-params/statement-params-03.plan
new file mode 100644
index 0000000..904a882
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/statement-params/statement-params-03.plan
@@ -0,0 +1,10 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/statement-params/statement-params-index-01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/statement-params/statement-params-index-01.plan
new file mode 100644
index 0000000..cecfb1c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/statement-params/statement-params-index-01.plan
@@ -0,0 +1,12 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$29(ASC) ] |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.employee) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/statement-params/statement-params-index-02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/statement-params/statement-params-index-02.plan
new file mode 100644
index 0000000..cce5aaa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/statement-params/statement-params-index-02.plan
@@ -0,0 +1,12 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$30(ASC) ] |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.employee) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpcds/query-ASTERIXDB-1580.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpcds/query-ASTERIXDB-1580.plan
new file mode 100644
index 0000000..eb98d7c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpcds/query-ASTERIXDB-1580.plan
@@ -0,0 +1,73 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_LIMIT |UNPARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$136(ASC) ] |PARTITIONED|
+ -- STREAM_LIMIT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [topK: 100] [$$136(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- SORT_GROUP_BY[$$145] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- HASH_PARTITION_EXCHANGE [$$145] |PARTITIONED|
+ -- SORT_GROUP_BY[$$126] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$129][$$132] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$129] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$137][$$131] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$137] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$128][$$142] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$128] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$127][$$140] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpcds.customer_address) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$140] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpcds.customer) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$142] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpcds.store_sales) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpcds.date_dim) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpcds.item) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpcds/query-ASTERIXDB-1591.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpcds/query-ASTERIXDB-1591.plan
new file mode 100644
index 0000000..d3fce85
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpcds/query-ASTERIXDB-1591.plan
@@ -0,0 +1,151 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_LIMIT |UNPARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$192(ASC) ] |PARTITIONED|
+ -- STREAM_LIMIT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- SORT_GROUP_BY[$$201, $$202] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- HASH_PARTITION_EXCHANGE [$$201, $$202] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$189, $$190] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$189(ASC), $$190(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$189][$$171] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$189] |PARTITIONED|
+ -- SORT_GROUP_BY[$$198, $$199] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- HASH_PARTITION_EXCHANGE [$$198, $$199] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$186, $$187] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$186(ASC), $$187(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$186][$$169] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$186] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- SORT_GROUP_BY[$$195, $$196] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- HASH_PARTITION_EXCHANGE [$$195, $$196] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$150, $$151] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$150(ASC), $$151(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$150][$$167] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$150] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$161][$$151] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$161] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpcds.customer) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpcds.customer_address) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$167] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$172][$$154] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$172] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpcds.store_sales) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpcds.date_dim) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$169] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$174][$$157] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$174] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpcds.web_sales) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpcds.date_dim) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$171] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$176][$$160] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$176] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpcds.catalog_sales) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpcds.date_dim) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpch/q12_broadcast.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpch/q12_broadcast.plan
new file mode 100644
index 0000000..4ad65b5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpch/q12_broadcast.plan
@@ -0,0 +1,43 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$l_shipmode(ASC) ] |PARTITIONED|
+ -- SORT_GROUP_BY[$$131] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- HASH_PARTITION_EXCHANGE [$$131] |PARTITIONED|
+ -- SORT_GROUP_BY[$$114] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$118][$$121] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.Orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$114][$$120] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.LineItem) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpch/q12_broadcast_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpch/q12_broadcast_ps.plan
new file mode 100644
index 0000000..7bf4afa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpch/q12_broadcast_ps.plan
@@ -0,0 +1,95 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$l_shipmode(ASC)] |PARTITIONED|
+ -- RANGE_PARTITION_EXCHANGE [$$l_shipmode(ASC)] |PARTITIONED|
+ -- FORWARD |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- SORT_GROUP_BY[$$131] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- HASH_PARTITION_EXCHANGE [$$131] |PARTITIONED|
+ -- SORT_GROUP_BY[$$114] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$118][$$121] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.Orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$114][$$120] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.LineItem) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- SORT_GROUP_BY[$$131] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- HASH_PARTITION_EXCHANGE [$$131] |PARTITIONED|
+ -- SORT_GROUP_BY[$$114] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$118][$$121] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.Orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$114][$$120] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.LineItem) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpch/q12_shipping.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpch/q12_shipping.plan
new file mode 100644
index 0000000..7609856
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpch/q12_shipping.plan
@@ -0,0 +1,43 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$l_shipmode(ASC) ] |PARTITIONED|
+ -- SORT_GROUP_BY[$$131] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- HASH_PARTITION_EXCHANGE [$$131] |PARTITIONED|
+ -- SORT_GROUP_BY[$$114] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$118][$$122] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.Orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$114][$$120] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.LineItem) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpch/q12_shipping_broadcast.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpch/q12_shipping_broadcast.plan
new file mode 100644
index 0000000..4ad65b5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpch/q12_shipping_broadcast.plan
@@ -0,0 +1,43 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$l_shipmode(ASC) ] |PARTITIONED|
+ -- SORT_GROUP_BY[$$131] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- HASH_PARTITION_EXCHANGE [$$131] |PARTITIONED|
+ -- SORT_GROUP_BY[$$114] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$118][$$121] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.Orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$114][$$120] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.LineItem) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpch/q12_shipping_broadcast_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpch/q12_shipping_broadcast_ps.plan
new file mode 100644
index 0000000..7bf4afa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpch/q12_shipping_broadcast_ps.plan
@@ -0,0 +1,95 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$l_shipmode(ASC)] |PARTITIONED|
+ -- RANGE_PARTITION_EXCHANGE [$$l_shipmode(ASC)] |PARTITIONED|
+ -- FORWARD |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- SORT_GROUP_BY[$$131] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- HASH_PARTITION_EXCHANGE [$$131] |PARTITIONED|
+ -- SORT_GROUP_BY[$$114] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$118][$$121] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.Orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$114][$$120] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.LineItem) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- SORT_GROUP_BY[$$131] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- HASH_PARTITION_EXCHANGE [$$131] |PARTITIONED|
+ -- SORT_GROUP_BY[$$114] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$118][$$121] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.Orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$114][$$120] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.LineItem) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpch/q12_shipping_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpch/q12_shipping_ps.plan
new file mode 100644
index 0000000..f21d402
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpch/q12_shipping_ps.plan
@@ -0,0 +1,95 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$l_shipmode(ASC)] |PARTITIONED|
+ -- RANGE_PARTITION_EXCHANGE [$$l_shipmode(ASC)] |PARTITIONED|
+ -- FORWARD |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- SORT_GROUP_BY[$$131] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- HASH_PARTITION_EXCHANGE [$$131] |PARTITIONED|
+ -- SORT_GROUP_BY[$$114] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$118][$$122] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.Orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$114][$$120] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.LineItem) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- SORT_GROUP_BY[$$131] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- HASH_PARTITION_EXCHANGE [$$131] |PARTITIONED|
+ -- SORT_GROUP_BY[$$114] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$118][$$122] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.Orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$114][$$120] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.LineItem) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/api.xml b/asterixdb/asterix-app/src/test/resources/runtimets/api.xml
index 8b16889..8a163d2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/api.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/api.xml
@@ -41,6 +41,12 @@
</compilation-unit>
</test-case>
<test-case FilePath="api">
+ <compilation-unit name="cluster_state_5">
+ <output-dir compare="Text">cluster_state_5</output-dir>
+ <expected-error>HTTP/1.1 404 Not Found</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="api">
<compilation-unit name="cluster_state_cc_1">
<output-dir compare="Text">cluster_state_cc_1</output-dir>
</compilation-unit>
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/cluster_state_5/cluster_state_5.1.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/cluster_state_5/cluster_state_5.1.get.http
new file mode 100644
index 0000000..d6d5636
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/cluster_state_5/cluster_state_5.1.get.http
@@ -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.
+ */
+/*
+ * Test case Name : cluster_state_2
+ * Description : test cluster state api malformed uri (404)
+ * Expected Result : Negative
+ * Date : 7th September 2016
+ */
+/admin/cluster/
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/median/median.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/median/median.01.ddl.sqlpp
new file mode 100644
index 0000000..8739d27
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/median/median.01.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.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE TYPE openType AS {id: int};
+CREATE DATASET large_ds(openType) primary key id;
+CREATE DATASET odd_ds(openType) primary key id;
+CREATE DATASET even_ds(openType) primary key id;
+CREATE DATASET empty_ds(openType) primary key id;
+CREATE DATASET one_item_ds(openType) primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/median/median.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/median/median.02.update.sqlpp
new file mode 100644
index 0000000..5f3816b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/median/median.02.update.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+UPSERT INTO one_item_ds {"id": 1, "m": 9};
+
+UPSERT INTO large_ds
+(FROM range(0, 5) as v
+ SELECT v AS id, v AS m, v % 15 AS g
+);
+
+UPSERT INTO large_ds
+(FROM range(6, 1000000) as v
+ SELECT v AS id, round_half_to_even(random(8) * 100, 1) AS m, v % 15 AS g
+);
+
+UPSERT INTO large_ds( [{"id": 1000001, "m": null, "g": 1}, {"id": 1000002, "g": 7}] );
+
+UPSERT INTO odd_ds
+(FROM range(1, 15) as v
+ SELECT v AS id, 513 % v AS m
+);
+
+UPSERT INTO even_ds
+(FROM range(1, 14) as v
+ SELECT v AS id, 513 % v AS m
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/median/median.03.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/median/median.03.query.sqlpp
new file mode 100644
index 0000000..9668a90
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/median/median.03.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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 median(m) AS med FROM odd_ds;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/median/median.04.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/median/median.04.query.sqlpp
new file mode 100644
index 0000000..5b52dcd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/median/median.04.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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 median(m) AS med FROM even_ds;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/median/median.05.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/median/median.05.query.sqlpp
new file mode 100644
index 0000000..9773b9f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/median/median.05.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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 median(v) AS med
+FROM [0,1,0,1,3,3,2,1,0,3,7,9,6,9,3] AS v;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/median/median.06.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/median/median.06.query.sqlpp
new file mode 100644
index 0000000..68b7be0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/median/median.06.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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 median(v) AS med
+FROM [0,1,0,1,3,3,2,1,0,3,7,9,6,9] AS v;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/median/median.07.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/median/median.07.query.sqlpp
new file mode 100644
index 0000000..95dc135
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/median/median.07.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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 median(m) AS med
+FROM empty_ds;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/median/median.08.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/median/median.08.query.sqlpp
new file mode 100644
index 0000000..daac19b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/median/median.08.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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 median(m) AS med
+FROM one_item_ds;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/median/median.09.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/median/median.09.query.sqlpp
new file mode 100644
index 0000000..9927bab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/median/median.09.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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 median(m) AS med
+FROM large_ds;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/median/median.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/median/median.10.query.sqlpp
new file mode 100644
index 0000000..e4b1d11
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/median/median.10.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 g, median(m) AS med
+FROM large_ds
+GROUP BY g
+ORDER BY g;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/median/median.11.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/median/median.11.query.sqlpp
new file mode 100644
index 0000000..4880e98
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/median/median.11.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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;
+SET `compiler.sortmemory` "130KB";
+SELECT median(m) AS med
+FROM large_ds;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/median/median.99.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/median/median.99.ddl.sqlpp
new file mode 100644
index 0000000..36b2bab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/median/median.99.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.1.ddl.sqlpp
new file mode 100644
index 0000000..c0aadca
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.1.ddl.sqlpp
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test1 if exists;
+create dataverse test1;
+
+use test1;
+
+create type t1 as closed {
+ c1 : bigint,
+ c2 : bigint
+};
+
+create dataset ds1(t1) primary key c1;
+
+create index idx1 on ds1(c2);
+
+create view v1 as select value v from ds1 v;
+
+create synonym s1 for v1;
+
+create function f1() {
+ select "Dataverse" as k, DataverseName as dv, DataverseName as n from Metadata.`Dataverse`
+ union all
+ select "Datatype" as k, DataverseName as dv, DatatypeName as n from Metadata.`Datatype`
+ union all
+ select "Dataset" as k, DataverseName as dv, DatasetName as n from Metadata.`Dataset`
+ union all
+ select "Index" as k, DataverseName as dv, IndexName as n from Metadata.`Index` where not(isPrimary)
+ union all
+ select "Synonym" as k, DataverseName as dv, SynonymName as n from Metadata.`Synonym`
+ union all
+ select "Function" as k, DataverseName as dv, Name as n from Metadata.`Function`
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.10.query.sqlpp
new file mode 100644
index 0000000..53097a4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.10.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.
+ */
+
+/*
+ * Check that objects were not created
+ */
+
+select count(*) cnt
+from test1.f1() t
+where dv like "test2";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.2.query.sqlpp
new file mode 100644
index 0000000..1e3fb34
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.2.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+
+select k, n
+from test1.f1() t
+where dv like "test1"
+order by k, n;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.3.ddl.sqlpp
new file mode 100644
index 0000000..b913179
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.3.ddl.sqlpp
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Test that DROP DDL statements are not executed in compile-only mode
+ */
+
+-- param compile-only:string=true
+
+drop function test1.f1();
+
+drop synonym test1.s1;
+
+drop view test1.v1;
+
+drop index test1.ds1.idx1;
+
+drop dataset test1.ds1;
+
+drop type test1.t1;
+
+
+
+
+
+
+
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.4.query.sqlpp
new file mode 100644
index 0000000..5760b8e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.4.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.
+ */
+
+/*
+ * Check that objects were not dropped
+ */
+
+select k, n
+from test1.f1() t
+where dv like "test1"
+order by k, n;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.5.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.5.ddl.sqlpp
new file mode 100644
index 0000000..37146d7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.5.ddl.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.
+ */
+
+/*
+ * Test that DROP DDL statements are not executed in compile-only mode
+ */
+
+-- param compile-only:string=true
+
+drop dataverse test1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.6.query.sqlpp
new file mode 100644
index 0000000..5760b8e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.6.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.
+ */
+
+/*
+ * Check that objects were not dropped
+ */
+
+select k, n
+from test1.f1() t
+where dv like "test1"
+order by k, n;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.7.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.7.ddl.sqlpp
new file mode 100644
index 0000000..9b41631
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.7.ddl.sqlpp
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Test that CREATE DDL statements are not executed in compile-only mode
+ */
+
+-- param compile-only:string=true
+
+use test1;
+
+create type t2 as closed {
+ c1 : bigint,
+ c2 : bigint
+};
+
+create dataset ds2(t2) primary key c1;
+
+create index idx2 on ds2(c2);
+
+create view v2 as select value v from ds2 v;
+
+create synonym s2 for v2;
+
+create function f2() {
+ select value v from v2 v
+};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.8.query.sqlpp
new file mode 100644
index 0000000..c5afa49
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.8.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.
+ */
+
+/*
+ * Check that objects were not created
+ */
+
+select k, n
+from test1.f1() t
+where dv like "test1"
+order by k, n;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.9.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.9.ddl.sqlpp
new file mode 100644
index 0000000..7fb26db
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.9.ddl.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.
+ */
+
+/*
+ * Test that CREATE DDL statements are not executed in compile-only mode
+ */
+
+-- param compile-only:string=true
+
+drop dataverse test2 if exists;
+create dataverse test2;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.ddl.sqlpp
new file mode 100644
index 0000000..82e9f1c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.ddl.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test1 if exists;
+create dataverse test1;
+
+create dataset test1.t1(c1 int not unknown, c2 int) primary key c1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
deleted file mode 100644
index 6095b26..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ /dev/null
@@ -1,24 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
-
-select value v from range(1,2) v where v > ?;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.2.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.2.plans.sqlpp
index 2a4952f..6095b26 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.2.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.2.plans.sqlpp
@@ -17,11 +17,8 @@
* under the License.
*/
-/*
- * Test additional information returned when client-type=jdbc
- */
-
--- param client-type:string=jdbc
-- param compile-only:string=true
+-- param logical-plan:string=true
+-- param plan-format:string=json
-select v from range(1,2) v where v between ? and ? ;
+select value v from range(1,2) v where v > ?;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.3.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.3.plans.sqlpp
index d7217a4..2a4952f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.3.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.3.plans.sqlpp
@@ -18,10 +18,10 @@
*/
/*
- * Test additional information returned when client-type=jdbc (with explain)
+ * Test additional information returned when client-type=jdbc
*/
-- param client-type:string=jdbc
-- param compile-only:string=true
-explain select v from range(1,2) v where v between ? and ? ;
+select v from range(1,2) v where v between ? and ? ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.4.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.4.plans.sqlpp
index 8d2bd74..d7217a4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.4.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.4.plans.sqlpp
@@ -18,16 +18,10 @@
*/
/*
- * Test additional information returned when client-type=jdbc (update statement)
+ * Test additional information returned when client-type=jdbc (with explain)
*/
-- param client-type:string=jdbc
-- param compile-only:string=true
-drop dataverse test1 if exists;
-
-create dataverse test1;
-
-create dataset test1.t1(c1 int not unknown, c2 int) primary key c1;
-
-insert into test1.t1 [{"c1": 1, "c2": ? }, {"c1": 3, "c2": ? }];
+explain select v from range(1,2) v where v between ? and ? ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.5.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.5.plans.sqlpp
new file mode 100644
index 0000000..5947c5d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.5.plans.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.
+ */
+
+/*
+ * Test additional information returned when client-type=jdbc (update statement)
+ */
+
+-- param client-type:string=jdbc
+-- param compile-only:string=true
+
+insert into test1.t1 [{"c1": 1, "c2": ? }, {"c1": 3, "c2": ? }];
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/index-bad-fields/index-bad-fields.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/index-bad-fields/index-bad-fields.000.ddl.sqlpp
new file mode 100644
index 0000000..e8af202
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/index-bad-fields/index-bad-fields.000.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Test array indexes where some tuples have malformed values in indexed fields
+ * Expected Res : Said tuples are not indexed
+ */
+
+CREATE DATAVERSE test;
+USE test;
+CREATE TYPE dt1 as {id:int};
+CREATE DATASET ds1(dt1) primary key id;
+
+CREATE INDEX i1 ON ds1(UNNEST a : string) EXCLUDE UNKNOWN KEY;
+CREATE INDEX i2 ON ds1(UNNEST b SELECT x : int) EXCLUDE UNKNOWN KEY;
+CREATE INDEX i3 ON ds1(UNNEST b.x SELECT p: int) EXCLUDE UNKNOWN KEY;
+CREATE INDEX i4 ON ds1(UNNEST b.c UNNEST d.e UNNEST t SELECT x.y : string, q.w: string, u: int) EXCLUDE UNKNOWN KEY;
+CREATE INDEX i5 ON ds1(p: int, (UNNEST b.c UNNEST d.e UNNEST t SELECT x.y : string, q.w: string), z.m: double) EXCLUDE UNKNOWN KEY;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/index-bad-fields/index-bad-fields.001.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/index-bad-fields/index-bad-fields.001.update.sqlpp
new file mode 100644
index 0000000..9c2b805
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/index-bad-fields/index-bad-fields.001.update.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * 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;
+UPSERT INTO ds1 {"id": 1, "a": [10, "hello", 10, "hello"]};
+UPSERT INTO ds1 {"id": 2, "a": [10]};
+UPSERT INTO ds1 {"id": 3, "a": 94};
+UPSERT INTO ds1 {"id": 4, "a": {"x":1}};
+
+UPSERT INTO ds1 {"id": 5, "a": ["hello"], "b": [{"x":1}, {"x":"bb"}]};
+UPSERT INTO ds1 {"id": 6, "a": ["hello"], "b": [{"x":"aa"}]};
+UPSERT INTO ds1 {"id": 7, "a": 10, "b": [{"x":10}]};
+UPSERT INTO ds1 {"id": 8, "a": ["hello"], "b":100};
+UPSERT INTO ds1 {"id": 9, "b": 100};
+
+UPSERT INTO ds1 {"id": 10, "a": ["hello"], "b":{"x":[{"p":1}, {"k":1}]}};
+
+UPSERT INTO ds1 {"id": 11, "a": ["hello"], "b": {"c":[{"d": {"e":[{"t":[{"x":{"y":"aab"}, "q":{"w":"10"}, "u":90}]}]}}]}};
+UPSERT INTO ds1 {"id": 12, "a": ["hello"], "b": {"c":[{"d": {"e":[{"t":{"x":{"y":"aab"}, "q":{"w":"10"}, "u":91}}]}}]}};
+UPSERT INTO ds1 {"id": 13, "a": ["hello"], "b": {"c":[{"d": {"e":{"t":[{"x":{"y":"aab"}, "q":{"w":"10"}, "u":92}]}}}]}};
+
+UPSERT INTO ds1 {"id": 14, "a": ["hello"], "b":{"c":[{"d":{"e":[{"t":[{"x":{"y":"aab"}, "q":{"w":"10"}, "u":93}]}]}}]}, "p":100, "z":{"m":100.10}};
+UPSERT INTO ds1 {"id": 15, "a": ["hello"], "b":{"c":[{"d":{"e":[{"t":[{"x":{"y":"aab"}, "q":{"w":"10"}}]}]}}]}, "p":"kk", "z":{"m":100.10}};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/index-bad-fields/index-bad-fields.002.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/index-bad-fields/index-bad-fields.002.query.sqlpp
new file mode 100644
index 0000000..e1bb086
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/index-bad-fields/index-bad-fields.002.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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(*) from ds1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/index-bad-fields/index-bad-fields.003.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/index-bad-fields/index-bad-fields.003.query.sqlpp
new file mode 100644
index 0000000..18132bb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/index-bad-fields/index-bad-fields.003.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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds1", "i1") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/index-bad-fields/index-bad-fields.004.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/index-bad-fields/index-bad-fields.004.query.sqlpp
new file mode 100644
index 0000000..afb1d61
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/index-bad-fields/index-bad-fields.004.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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds1", "i2") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/index-bad-fields/index-bad-fields.005.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/index-bad-fields/index-bad-fields.005.query.sqlpp
new file mode 100644
index 0000000..57ad1d9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/index-bad-fields/index-bad-fields.005.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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds1", "i3") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/index-bad-fields/index-bad-fields.006.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/index-bad-fields/index-bad-fields.006.query.sqlpp
new file mode 100644
index 0000000..dcf23cf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/index-bad-fields/index-bad-fields.006.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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds1", "i4") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/index-bad-fields/index-bad-fields.007.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/index-bad-fields/index-bad-fields.007.query.sqlpp
new file mode 100644
index 0000000..b070cbc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/index-bad-fields/index-bad-fields.007.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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds1", "i5") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/index-bad-fields/index-bad-fields.008.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/index-bad-fields/index-bad-fields.008.ddl.sqlpp
new file mode 100644
index 0000000..7850fde
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/index-bad-fields/index-bad-fields.008.ddl.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 : Test array indexes where some tuples have malformed values in indexed fields
+ * Expected Res : Said tuples are not indexed
+ */
+
+USE test;
+
+CREATE INDEX i6 ON ds1(UNNEST a : string) EXCLUDE UNKNOWN KEY;
+CREATE INDEX i7 ON ds1(UNNEST b SELECT x : int) EXCLUDE UNKNOWN KEY;
+CREATE INDEX i8 ON ds1(UNNEST b.x SELECT p: int) EXCLUDE UNKNOWN KEY;
+CREATE INDEX i9 ON ds1(UNNEST b.c UNNEST d.e UNNEST t SELECT x.y : string, q.w: string, u: int) EXCLUDE UNKNOWN KEY;
+CREATE INDEX i10 ON ds1(p: int, (UNNEST b.c UNNEST d.e UNNEST t SELECT x.y : string, q.w: string), z.m: double) EXCLUDE UNKNOWN KEY;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/index-bad-fields/index-bad-fields.009.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/index-bad-fields/index-bad-fields.009.query.sqlpp
new file mode 100644
index 0000000..a33bb23
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/index-bad-fields/index-bad-fields.009.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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds1", "i6") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/index-bad-fields/index-bad-fields.010.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/index-bad-fields/index-bad-fields.010.query.sqlpp
new file mode 100644
index 0000000..4134554
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/index-bad-fields/index-bad-fields.010.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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds1", "i7") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/index-bad-fields/index-bad-fields.011.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/index-bad-fields/index-bad-fields.011.query.sqlpp
new file mode 100644
index 0000000..75b56d1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/index-bad-fields/index-bad-fields.011.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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds1", "i8") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/index-bad-fields/index-bad-fields.012.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/index-bad-fields/index-bad-fields.012.query.sqlpp
new file mode 100644
index 0000000..5364ebb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/index-bad-fields/index-bad-fields.012.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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds1", "i9") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/index-bad-fields/index-bad-fields.013.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/index-bad-fields/index-bad-fields.013.query.sqlpp
new file mode 100644
index 0000000..44f59c8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/index-bad-fields/index-bad-fields.013.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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds1", "i10") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/index-bad-fields/index-bad-fields.999.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/index-bad-fields/index-bad-fields.999.ddl.sqlpp
new file mode 100644
index 0000000..86a1b59
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/index-bad-fields/index-bad-fields.999.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/multiple-indexes/multiple-indexes.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/multiple-indexes/multiple-indexes.1.ddl.sqlpp
new file mode 100644
index 0000000..d589457
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/multiple-indexes/multiple-indexes.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 test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE TYPE GenericType AS { id: bigint };
+CREATE DATASET KSI (GenericType)
+PRIMARY KEY id;
+
+CREATE INDEX KS1_array_index1 ON KSI (
+ UNNEST uarr_i
+ SELECT a : bigint
+) EXCLUDE UNKNOWN KEY;
+CREATE INDEX KS1_array_index2 ON KSI (
+ UNNEST uarr_i
+ SELECT b : bigint
+) EXCLUDE UNKNOWN KEY;
+CREATE INDEX KS1_array_index3 ON KSI (
+ UNNEST uarr_i
+ SELECT c : bigint
+) EXCLUDE UNKNOWN KEY;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/multiple-indexes/multiple-indexes.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/multiple-indexes/multiple-indexes.2.update.sqlpp
new file mode 100644
index 0000000..1d1f638
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/multiple-indexes/multiple-indexes.2.update.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+USE test;
+
+INSERT INTO KSI [
+ { "id": 1, "uarr_i": [ {"a": 1, "b": 1, "c": 1 } ] },
+ { "id": 2, "uarr_i": [ {"a": 1, "b": 1, "c": 1 },
+ {"a": 2, "b": 2, "c": 2 } ] },
+ { "id": 3, "uarr_i": [ {"a": 1, "b": 1, "c": 1 },
+ {"a": 3, "b": 3, "c": 3 } ] },
+ { "id": 4, "uarr_i": [ {"a": 2, "b": 2, "c": 2 },
+ {"a": 3, "b": 3, "c": 3 } ] },
+ { "id": 5, "uarr_i": [ {"a": 1, "b": 1, "c": 1 },
+ {"a": 2, "b": 2, "c": 2 },
+ {"a": 3, "b": 3, "c": 3 } ] }
+];
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/multiple-indexes/multiple-indexes.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/multiple-indexes/multiple-indexes.3.query.sqlpp
new file mode 100644
index 0000000..ce7f053
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/multiple-indexes/multiple-indexes.3.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.
+ */
+USE test;
+
+FROM KSI k
+WHERE ( SOME i IN k.uarr_i
+ SATISFIES i.a = 1 AND
+ i.b = 1 )
+SELECT k.id
+ORDER BY k.id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/multiple-indexes/multiple-indexes.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/multiple-indexes/multiple-indexes.4.query.sqlpp
new file mode 100644
index 0000000..1314ad8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/multiple-indexes/multiple-indexes.4.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 test;
+
+FROM KSI k
+WHERE ( SOME i IN k.uarr_i
+ SATISFIES i.a = 1 AND
+ i.b = 1 AND
+ i.c = 1 )
+SELECT k.id
+ORDER BY k.id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/multiple-indexes/multiple-indexes.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/multiple-indexes/multiple-indexes.1.ddl.sqlpp
new file mode 100644
index 0000000..d589457
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/multiple-indexes/multiple-indexes.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 test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE TYPE GenericType AS { id: bigint };
+CREATE DATASET KSI (GenericType)
+PRIMARY KEY id;
+
+CREATE INDEX KS1_array_index1 ON KSI (
+ UNNEST uarr_i
+ SELECT a : bigint
+) EXCLUDE UNKNOWN KEY;
+CREATE INDEX KS1_array_index2 ON KSI (
+ UNNEST uarr_i
+ SELECT b : bigint
+) EXCLUDE UNKNOWN KEY;
+CREATE INDEX KS1_array_index3 ON KSI (
+ UNNEST uarr_i
+ SELECT c : bigint
+) EXCLUDE UNKNOWN KEY;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/multiple-indexes/multiple-indexes.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/multiple-indexes/multiple-indexes.2.update.sqlpp
new file mode 100644
index 0000000..1d1f638
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/multiple-indexes/multiple-indexes.2.update.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+USE test;
+
+INSERT INTO KSI [
+ { "id": 1, "uarr_i": [ {"a": 1, "b": 1, "c": 1 } ] },
+ { "id": 2, "uarr_i": [ {"a": 1, "b": 1, "c": 1 },
+ {"a": 2, "b": 2, "c": 2 } ] },
+ { "id": 3, "uarr_i": [ {"a": 1, "b": 1, "c": 1 },
+ {"a": 3, "b": 3, "c": 3 } ] },
+ { "id": 4, "uarr_i": [ {"a": 2, "b": 2, "c": 2 },
+ {"a": 3, "b": 3, "c": 3 } ] },
+ { "id": 5, "uarr_i": [ {"a": 1, "b": 1, "c": 1 },
+ {"a": 2, "b": 2, "c": 2 },
+ {"a": 3, "b": 3, "c": 3 } ] }
+];
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/multiple-indexes/multiple-indexes.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/multiple-indexes/multiple-indexes.3.query.sqlpp
new file mode 100644
index 0000000..c9b3208
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/multiple-indexes/multiple-indexes.3.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.
+ */
+USE test;
+
+FROM KSI k
+UNNEST k.uarr_i i
+WHERE i.a = 1 AND
+ i.b = 1
+SELECT k.id
+ORDER BY k.id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/multiple-indexes/multiple-indexes.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/multiple-indexes/multiple-indexes.4.query.sqlpp
new file mode 100644
index 0000000..96aee1c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/multiple-indexes/multiple-indexes.4.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 test;
+
+FROM KSI k
+UNNEST k.uarr_i i
+WHERE i.a = 1 AND
+ i.b = 1 AND
+ i.c = 1
+SELECT k.id
+ORDER BY k.id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/incomparable_types/incomparable_types.003.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/incomparable_types/incomparable_types.003.query.sqlpp
index 66e6595..c9f0619 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/incomparable_types/incomparable_types.003.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/incomparable_types/incomparable_types.003.query.sqlpp
@@ -17,10 +17,11 @@
* under the License.
*/
+set `compiler.parallelism` "1";
-- param max-warnings:string=10
use test;
from ds1 join ds2 on ds1.f /*+ hash-bcast */ = ds2.f
where ds1.f > 1
-select ds1.f;
\ No newline at end of file
+select ds1.f;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.1.ddl.sqlpp
new file mode 100644
index 0000000..e1d6b10
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.1.ddl.sqlpp
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Test ANALYZE DATASET statement
+ */
+
+set `import-private-functions` `true`;
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create function listMetadata(showSourceAvgItemSize, showSeed) {
+ select i.DatasetName, i.IndexName, i.SampleCardinalityTarget, i.SourceCardinality,
+ case when showSourceAvgItemSize then i.SourceAvgItemSize else i.SourceAvgItemSize > 0 end as SourceAvgItemSize,
+ case when showSeed then i.SampleSeed else i.SampleSeed is known end as SampleSeed
+ from Metadata.`Index` i
+ where i.DataverseName = "test" and i.IndexName like "sample_idx%"
+ order by i.IndexName
+};
+
+create function showSampleStats(dsName, idxName, showMinMax) {
+ select count(v.values) as cnt,
+ case when showMinMax then min(v.values[0]) else min(v.values[0]) > 0 end as min_pk,
+ case when showMinMax then max(v.values[0]) else max(v.values[0]) > 0 end as max_pk,
+ case when showMinMax then min(v.values[1].x) else min(v.values[1].x) < 0 end as min_x,
+ case when showMinMax then max(v.values[1].x) else max(v.values[1].x) < 0 end as max_x
+ from dump_index("test", dsName, idxName) as v
+};
+
+create type t1 as open {
+ id : bigint
+};
+
+create dataset ds1(t1) primary key id;
+
+-- analyze on an empty dataset
+
+analyze dataset ds1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.10.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.10.ddl.sqlpp
new file mode 100644
index 0000000..da5fe13
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.10.ddl.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.
+ */
+
+/*
+ * Description: analyze dataset with sample=medium.
+ * Note, there are fewer tuples in the dataset that the target sample size
+ */
+
+use test;
+
+analyze dataset test.ds1 with { "sample": "medium", "sample-seed": 234.0 };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.11.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.11.query.sqlpp
new file mode 100644
index 0000000..38ded0a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.11.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: check that the sample was re-created
+ */
+
+set `import-private-functions` `true`;
+
+use test;
+
+select * from
+ listMetadata(false, true) metadata,
+ showSampleStats("ds1", "sample_idx_2_ds1", true) stats;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.12.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.12.update.sqlpp
new file mode 100644
index 0000000..0e3886d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.12.update.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Insert more data
+ */
+
+use test;
+
+insert into ds1
+select element {'id':x, 'x':-x }
+from range(1101, 4400) as x;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.13.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.13.ddl.sqlpp
new file mode 100644
index 0000000..b098016
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.13.ddl.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.
+ */
+
+/*
+ * Description: analyze dataset with sample=medium
+ * Note, there are more tuples in the dataset that the target sample size
+ */
+
+use test;
+
+analyze dataset ds1 with { "sample": "medium" };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.14.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.14.query.sqlpp
new file mode 100644
index 0000000..4cae202
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.14.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: check that the sample was re-created
+ */
+
+set `import-private-functions` `true`;
+
+use test;
+
+select * from
+ listMetadata(false, false) metadata,
+ showSampleStats("ds1", "sample_idx_1_ds1", false) stats;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.15.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.15.ddl.sqlpp
new file mode 100644
index 0000000..6ceb8141
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.15.ddl.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.
+ */
+
+/*
+ * Description: analyze dataset with sample=high.
+ * Note, there are fewer tuples in the dataset that the target sample size
+ */
+
+use test;
+
+analyze dataset ds1 with { "sample": "high", "sample-seed": "345" };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.16.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.16.query.sqlpp
new file mode 100644
index 0000000..38ded0a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.16.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: check that the sample was re-created
+ */
+
+set `import-private-functions` `true`;
+
+use test;
+
+select * from
+ listMetadata(false, true) metadata,
+ showSampleStats("ds1", "sample_idx_2_ds1", true) stats;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.17.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.17.update.sqlpp
new file mode 100644
index 0000000..0c5f8ad
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.17.update.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Insert more data
+ */
+
+use test;
+
+insert into ds1
+select element {'id':x, 'x':-x }
+from range(4401, 17100) as x;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.18.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.18.ddl.sqlpp
new file mode 100644
index 0000000..4385f95
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.18.ddl.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.
+ */
+
+/*
+ * Description: analyze dataset with sample=high
+ * Note, there are more tuples in the dataset that the target sample size
+ */
+
+use test;
+
+analyze dataset ds1 with { "sample": "high" };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.19.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.19.query.sqlpp
new file mode 100644
index 0000000..4cae202
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.19.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: check that the sample was re-created
+ */
+
+set `import-private-functions` `true`;
+
+use test;
+
+select * from
+ listMetadata(false, false) metadata,
+ showSampleStats("ds1", "sample_idx_1_ds1", false) stats;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.2.query.sqlpp
new file mode 100644
index 0000000..e786e0e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.2.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.
+ */
+
+/*
+ * Description: Check that the sample index was created even thought
+ * the source dataset is empty
+ */
+
+use test;
+
+listMetadata(true, false);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.20.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.20.ddl.sqlpp
new file mode 100644
index 0000000..3dfad77
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.20.ddl.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.
+ */
+
+/*
+ * Description: analyze dataset with sample=high
+ * Note, there are more tuples in the dataset that the target sample size
+ */
+
+use test;
+
+analyze dataset ds1 with { "sample": "high", "sample-seed": -10 };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.21.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.21.query.sqlpp
new file mode 100644
index 0000000..9e7988c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.21.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: check that the sample was re-created
+ */
+
+set `import-private-functions` `true`;
+
+use test;
+
+select * from
+ listMetadata(false, false) metadata,
+ showSampleStats("ds1", "sample_idx_2_ds1", false) stats;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.22.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.22.ddl.sqlpp
new file mode 100644
index 0000000..3190b2d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.22.ddl.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 that the sample index is dropped using "analyze dataset drop statistics" statement
+ */
+
+use test;
+
+analyze dataset ds1 drop statistics;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.23.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.23.query.sqlpp
new file mode 100644
index 0000000..759fc3f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.23.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.
+ */
+
+/*
+ * Description: check that the sample was dropped
+ */
+
+use test;
+
+select count(*) cnt
+from listMetadata(true, false) v;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.24.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.24.ddl.sqlpp
new file mode 100644
index 0000000..7d6bf92
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.24.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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;
+
+analyze dataset ds1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.25.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.25.query.sqlpp
new file mode 100644
index 0000000..a593df3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.25.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.
+ */
+
+/*
+ * Description: check that the sample was re-created again
+ */
+
+use test;
+
+select count(*) cnt
+from listMetadata(true, false) v;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.26.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.26.ddl.sqlpp
new file mode 100644
index 0000000..151309a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.26.ddl.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 that the sample index is dropped when its source dataset is dropped
+ */
+
+use test;
+
+drop dataset ds1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.27.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.27.query.sqlpp
new file mode 100644
index 0000000..759fc3f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.27.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.
+ */
+
+/*
+ * Description: check that the sample was dropped
+ */
+
+use test;
+
+select count(*) cnt
+from listMetadata(true, false) v;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.3.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.3.update.sqlpp
new file mode 100644
index 0000000..1b83b99
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.3.update.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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 ds1
+select element {'id':x, 'x':-x }
+from range(1, 8) as x;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.4.ddl.sqlpp
new file mode 100644
index 0000000..ed97897
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.4.ddl.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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 sample size parameter
+ */
+
+analyze dataset test.ds1 with { "sample": "low", "sample-seed": 123 };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.5.query.sqlpp
new file mode 100644
index 0000000..e0cd6cc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.5.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: check that sample index was recreated with a new name.
+ * Also check that all 8 dataset tuples are in the sample
+ */
+
+set `import-private-functions` `true`;
+
+use test;
+
+select * from
+ listMetadata(false, true) metadata,
+ showSampleStats("ds1", "sample_idx_2_ds1", true) stats
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.6.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.6.update.sqlpp
new file mode 100644
index 0000000..340fbb3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.6.update.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Insert more data
+ */
+
+use test;
+
+insert into ds1
+select element {'id':x, 'x':-x }
+from range(9, 1100) as x;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.7.query.sqlpp
new file mode 100644
index 0000000..c4930b0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.7.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: check that the sample content remains the same
+ * after INSERT because we did not run ANALYZE DATASET after that
+ */
+
+set `import-private-functions` `true`;
+
+use test;
+
+select * from
+ listMetadata(false, true) metadata,
+ showSampleStats("ds1", "sample_idx_2_ds1", true) stats;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.8.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.8.ddl.sqlpp
new file mode 100644
index 0000000..2f452f0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.8.ddl.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.
+ */
+
+/*
+ * Description: analyze dataset with sample=low.
+ * Note, there are more tuples in the dataset that the target sample size
+ */
+
+use test;
+
+analyze dataset ds1 with { "sample": "low" };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.9.query.sqlpp
new file mode 100644
index 0000000..4cae202
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.9.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: check that the sample was re-created
+ */
+
+set `import-private-functions` `true`;
+
+use test;
+
+select * from
+ listMetadata(false, false) metadata,
+ showSampleStats("ds1", "sample_idx_1_ds1", false) stats;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-with-indexes/analyze-dataset-with-indexes.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-with-indexes/analyze-dataset-with-indexes.01.ddl.sqlpp
new file mode 100644
index 0000000..8c374a2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-with-indexes/analyze-dataset-with-indexes.01.ddl.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Test collecting secondary indexes stats with ANALYZE DATASET statement
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE TYPE testType AS OPEN {
+ id : uuid
+};
+
+CREATE DATASET ds1(testType) PRIMARY KEY id AUTOGENERATED;
+CREATE DATASET ds2(testType) PRIMARY KEY id AUTOGENERATED;
+CREATE DATASET ds3(testType) PRIMARY KEY id AUTOGENERATED;
+CREATE DATASET ds4(testType) PRIMARY KEY id AUTOGENERATED;
+
+CREATE INDEX ds1_idx1 ON ds1(name: string);
+CREATE INDEX ds1_idx2 ON ds1(UNNEST interests: string) EXCLUDE UNKNOWN KEY;;
+CREATE PRIMARY INDEX ds1_idx3 ON ds1;
+
+CREATE INDEX ds2_idx1 ON ds2(name: string);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-with-indexes/analyze-dataset-with-indexes.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-with-indexes/analyze-dataset-with-indexes.02.update.sqlpp
new file mode 100644
index 0000000..711b914
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-with-indexes/analyze-dataset-with-indexes.02.update.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;
+
+LOAD DATASET ds1 USING localfs (("path"="asterix_nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+LOAD DATASET ds2 USING localfs (("path"="asterix_nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+LOAD DATASET ds3 USING localfs (("path"="asterix_nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+LOAD DATASET ds4 USING localfs (("path"="asterix_nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-with-indexes/analyze-dataset-with-indexes.03.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-with-indexes/analyze-dataset-with-indexes.03.ddl.sqlpp
new file mode 100644
index 0000000..97e07ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-with-indexes/analyze-dataset-with-indexes.03.ddl.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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;
+
+ANALYZE DATASET ds1;
+ANALYZE DATASET ds2;
+ANALYZE DATASET ds3;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-with-indexes/analyze-dataset-with-indexes.04.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-with-indexes/analyze-dataset-with-indexes.04.query.sqlpp
new file mode 100644
index 0000000..e674c09
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-with-indexes/analyze-dataset-with-indexes.04.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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 t.DatasetName, t.IndexName, t.IndexStructure, t.SampleCardinalityTarget, t.SourceCardinality, t.SourceAvgItemSize, t.IndexStats
+FROM `Metadata`.`Index` t WHERE t.IndexStructure = "SAMPLE"
+ORDER BY t.DatasetName, t.IndexName;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-with-indexes/analyze-dataset-with-indexes.05.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-with-indexes/analyze-dataset-with-indexes.05.ddl.sqlpp
new file mode 100644
index 0000000..4a2edf0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-with-indexes/analyze-dataset-with-indexes.05.ddl.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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;
+
+DROP INDEX ds1.ds1_idx3;
+ANALYZE DATASET ds1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-with-indexes/analyze-dataset-with-indexes.06.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-with-indexes/analyze-dataset-with-indexes.06.query.sqlpp
new file mode 100644
index 0000000..dcb57b2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-with-indexes/analyze-dataset-with-indexes.06.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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 t.DatasetName, t.IndexName, t.IndexStructure, t.SampleCardinalityTarget, t.SourceCardinality, t.SourceAvgItemSize, t.IndexStats
+FROM `Metadata`.`Index` t WHERE t.IndexStructure = "SAMPLE"
+ORDER BY t.DatasetName, t.IndexName;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-with-indexes/analyze-dataset-with-indexes.99.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-with-indexes/analyze-dataset-with-indexes.99.ddl.sqlpp
new file mode 100644
index 0000000..36b2bab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-with-indexes/analyze-dataset-with-indexes.99.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/explain/explain_same_datasource_function_different_arguments/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/explain/explain_same_datasource_function_different_arguments/test.000.ddl.sqlpp
new file mode 100644
index 0000000..737c344
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/explain/explain_same_datasource_function_different_arguments/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 {id: int};
+
+DROP DATASET test IF EXISTS;
+CREATE DATASET test(test) PRIMARY KEY id;
+
+DROP INDEX test.test_int_idx IF EXISTS;
+CREATE INDEX test_int_idx ON test(intField: int);
+
+DROP INDEX test.test_string_idx IF EXISTS;
+CREATE INDEX test_string_idx ON test(stringField: string);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/explain/explain_same_datasource_function_different_arguments/test.010.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/explain/explain_same_datasource_function_different_arguments/test.010.update.sqlpp
new file mode 100644
index 0000000..aeadafc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/explain/explain_same_datasource_function_different_arguments/test.010.update.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;
+INSERT INTO test({"id": 1, "intField": 15, "stringField": "foo"});
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/explain/explain_same_datasource_function_different_arguments/test.020.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/explain/explain_same_datasource_function_different_arguments/test.020.query.sqlpp
new file mode 100644
index 0000000..9752c75
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/explain/explain_same_datasource_function_different_arguments/test.020.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+SET `import-private-functions` `true`;
+
+SELECT dump_index("test", "test", "test_int_idx") as intIndex
+UNION ALL
+SELECT dump_index("test", "test", "test_string_idx") as stringIndex;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/explain/explain_same_datasource_function_different_arguments/test.030.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/explain/explain_same_datasource_function_different_arguments/test.030.query.sqlpp
new file mode 100644
index 0000000..b4209fc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/explain/explain_same_datasource_function_different_arguments/test.030.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+SET `import-private-functions` `true`;
+
+explain
+SELECT dump_index("test", "test", "test_int_idx") as intIndex
+UNION ALL
+SELECT dump_index("test", "test", "test_string_idx") as stringIndex;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/explain/explain_same_datasource_function_different_arguments/test.999.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/explain/explain_same_datasource_function_different_arguments/test.999.ddl.sqlpp
new file mode 100644
index 0000000..20dc6fd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/explain/explain_same_datasource_function_different_arguments/test.999.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/json/json/external_dataset.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/json/json/external_dataset.000.ddl.sqlpp
index f203da0..1538bf9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/json/json/external_dataset.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/json/json/external_dataset.000.ddl.sqlpp
@@ -63,4 +63,27 @@
("container"="playground"),
("definition"="json-data/single-line/json-array-of-objects"),
("format"="json")
+);
+
+drop dataset test6 if exists;
+CREATE EXTERNAL DATASET test6(test) USING %adapter% (
+%template%,
+("container"="playground"),
+("definition"="json-data/reviews"),
+("format"="json")
+);
+
+drop dataset test7 if exists;
+CREATE EXTERNAL DATASET test7(test) USING %adapter% (
+%template%,
+("container"="playground"),
+("format"="json")
+);
+
+drop dataset test8 if exists;
+CREATE EXTERNAL DATASET test8(test) USING %adapter% (
+%template%,
+("container"="playground"),
+("definition"="json-data"),
+("format"="json")
);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/json/json/external_dataset.008.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/json/json/external_dataset.008.query.sqlpp
new file mode 100644
index 0000000..1b265f1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/json/json/external_dataset.008.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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;
+// testing subpath hint. the result should be = scanning test3
+select value test6 from /*+ subpath /multi-lines-with-arrays/json */ test6 order by id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/json/json/external_dataset.009.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/json/json/external_dataset.009.query.sqlpp
new file mode 100644
index 0000000..9356366
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/json/json/external_dataset.009.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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;
+// testing subpath hint. the result should be = scanning test4
+select value test7 from /*+ subpath json-data/reviews/multi-lines-with-nested-objects/json */ test7 order by id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/json/json/external_dataset.010.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/json/json/external_dataset.010.query.sqlpp
new file mode 100644
index 0000000..8425714
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/json/json/external_dataset.010.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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;
+// testing subpath hint. the result should be = scanning test5
+select value count(*) from /*+ subpath single-line/json-array-of-objects */ test8;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/json/json/external_dataset.011.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/json/json/external_dataset.011.query.sqlpp
new file mode 100644
index 0000000..d0f08aa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/json/json/external_dataset.011.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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;
+// testing that test6 a and test6 b are two different data sources and hence no replicate should exist in the plan
+explain select count(a.quarter) as cnt
+from /*+ subpath /multi-lines/json */ test6 a
+join test6 b on a.quarter = b.quarter;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/json/json/external_dataset.012.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/json/json/external_dataset.012.query.sqlpp
new file mode 100644
index 0000000..3a675ef
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/json/json/external_dataset.012.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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;
+// testing that test6 a and test6 b are two different data sources and hence no replicate should exist in the plan
+select count(a.quarter) as cnt
+from /*+ subpath /multi-lines/json */ test6 a
+join test6 b on a.quarter = b.quarter;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/json/json/external_dataset.013.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/json/json/external_dataset.013.query.sqlpp
new file mode 100644
index 0000000..e886adb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/json/json/external_dataset.013.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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;
+// testing that test6 a and test6 b are the same data sources and hence replicate should exist in the plan
+explain select count(a.quarter) as cnt
+from /*+ subpath /multi-lines/json */ test6 a
+join /*+ subpath /multi-lines/json */ test6 b on a.quarter = b.quarter;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/json/json/external_dataset.014.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/json/json/external_dataset.014.query.sqlpp
new file mode 100644
index 0000000..5b39d4f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/json/json/external_dataset.014.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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;
+// testing that test6 a and test6 b are the same data sources and hence replicate should exist in the plan
+select count(a.quarter) as cnt
+from /*+ subpath /multi-lines/json */ test6 a
+join /*+ subpath /multi-lines/json */ test6 b on a.quarter = b.quarter;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/ASTERIXDB-3502/ASTERIXDB-3502.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/ASTERIXDB-3502/ASTERIXDB-3502.01.ddl.sqlpp
new file mode 100644
index 0000000..e4c59d2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/ASTERIXDB-3502/ASTERIXDB-3502.01.ddl.sqlpp
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+DROP DATAVERSE tpch IF EXISTS;
+CREATE DATAVERSE tpch;
+
+USE tpch;
+
+CREATE TYPE tpch.CustomerType AS
+ CLOSED {
+ c_custkey : integer,
+ c_name : string,
+ c_address : string,
+ c_nationkey : integer,
+ c_phone : string,
+ c_acctbal : double,
+ c_mktsegment : string,
+ c_comment : string
+};
+
+CREATE TYPE tpch.SupplierType AS
+ CLOSED {
+ s_suppkey : integer,
+ s_name : string,
+ s_address : string,
+ s_nationkey : integer,
+ s_phone : string,
+ s_acctbal : double,
+ s_comment : string
+};
+
+CREATE TYPE tpch.NationType AS
+ CLOSED {
+ n_nationkey : integer,
+ n_name : string,
+ n_regionkey : integer,
+ n_comment : string
+};
+
+CREATE DATASET Supplier(SupplierType) PRIMARY KEY s_suppkey;
+
+CREATE DATASET Nation(NationType) PRIMARY KEY n_nationkey;
+
+CREATE DATASET Customer(CustomerType) PRIMARY KEY c_custkey;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/ASTERIXDB-3502/ASTERIXDB-3502.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/ASTERIXDB-3502/ASTERIXDB-3502.02.update.sqlpp
new file mode 100644
index 0000000..b4ae2b6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/ASTERIXDB-3502/ASTERIXDB-3502.02.update.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.
+ */
+
+USE tpch;
+
+LOAD DATASET Supplier USING localfs ((`path`=`asterix_nc1://data/tpch0.001/supplier.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`));
+
+LOAD DATASET Nation USING localfs ((`path`=`asterix_nc1://data/tpch0.001/nation.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`));
+
+LOAD DATASET Customer USING localfs ((`path`=`asterix_nc1://data/tpch0.001/customer.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/ASTERIXDB-3502/ASTERIXDB-3502.03.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/ASTERIXDB-3502/ASTERIXDB-3502.03.query.sqlpp
new file mode 100644
index 0000000..76750f9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/ASTERIXDB-3502/ASTERIXDB-3502.03.query.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * 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.
+ */
+/*
+ * Similar to hash-join-with-redundant-variable.04.query.sqlpp
+ * But with Index NL. The plan of this test should has three
+ * hash-partition-exchange (as opposed to test 13 & 14). Because the parallelism
+ * is set to 3, then the last join requires both sides to be hash partitioned.
+ * Customer will need to duplicate its variable to join both with Nation and Supplier.
+ * This is the effect of using Index NL with parallelism != # of partitions
+ */
+
+USE tpch;
+
+-- this query should not give any results
+
+SELECT n.n_nationkey, s.s_nationkey, c.c_nationkey
+FROM Nation n, Supplier s, Customer c
+WHERE s.s_nationkey = n.n_nationkey
+AND c.c_nationkey = n.n_nationkey
+AND (s.s_nationkey = c.c_nationkey) = (s.s_nationkey = c.c_nationkey)
+-- before ASTERIXDB-3502, below expression was getting removed, hence evaluated to wrong result
+AND (s.s_nationkey != c.c_nationkey) = (s.s_nationkey = c.c_nationkey)
+ORDER BY n.n_nationkey, s.s_nationkey, c.c_nationkey;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.01.ddl.sqlpp
new file mode 100644
index 0000000..48e46f6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.01.ddl.sqlpp
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+DROP DATAVERSE tpch IF EXISTS;
+CREATE DATAVERSE tpch;
+
+USE tpch;
+
+
+CREATE TYPE tpch.LineItemType AS
+ CLOSED {
+ l_orderkey : integer,
+ l_partkey : integer,
+ l_suppkey : integer,
+ l_linenumber : integer,
+ l_quantity : integer,
+ l_extendedprice : double,
+ l_discount : double,
+ l_tax : double,
+ l_returnflag : string,
+ l_linestatus : string,
+ l_shipdate : string,
+ l_commitdate : string,
+ l_receiptdate : string,
+ l_shipinstruct : string,
+ l_shipmode : string,
+ l_comment : string
+};
+
+CREATE TYPE tpch.OrderType AS
+ CLOSED {
+ o_orderkey : integer,
+ o_custkey : integer,
+ o_orderstatus : string,
+ o_totalprice : double,
+ o_orderdate : string,
+ o_orderpriority : string,
+ o_clerk : string,
+ o_shippriority : integer,
+ o_comment : string
+};
+
+CREATE TYPE tpch.CustomerType AS
+ CLOSED {
+ c_custkey : integer,
+ c_name : string,
+ c_address : string,
+ c_nationkey : integer,
+ c_phone : string,
+ c_acctbal : double,
+ c_mktsegment : string,
+ c_comment : string
+};
+
+CREATE TYPE tpch.SupplierType AS
+ CLOSED {
+ s_suppkey : integer,
+ s_name : string,
+ s_address : string,
+ s_nationkey : integer,
+ s_phone : string,
+ s_acctbal : double,
+ s_comment : string
+};
+
+CREATE TYPE tpch.NationType AS
+ CLOSED {
+ n_nationkey : integer,
+ n_name : string,
+ n_regionkey : integer,
+ n_comment : string
+};
+
+CREATE TYPE tpch.RegionType AS
+ CLOSED {
+ r_regionkey : integer,
+ r_name : string,
+ r_comment : string
+};
+
+CREATE TYPE tpch.PartType AS
+ CLOSED {
+ p_partkey : integer,
+ p_name : string,
+ p_mfgr : string,
+ p_brand : string,
+ p_type : string,
+ p_size : integer,
+ p_container : string,
+ p_retailprice : double,
+ p_comment : string
+};
+
+CREATE TYPE tpch.PartSuppType AS
+ CLOSED {
+ ps_partkey : integer,
+ ps_suppkey : integer,
+ ps_availqty : integer,
+ ps_supplycost : double,
+ ps_comment : string
+};
+
+CREATE DATASET LineItem(LineItemType) PRIMARY KEY l_orderkey,l_linenumber;
+
+CREATE DATASET Orders(OrderType) PRIMARY KEY o_orderkey;
+
+CREATE DATASET Supplier(SupplierType) PRIMARY KEY s_suppkey;
+
+CREATE DATASET Region(RegionType) PRIMARY KEY r_regionkey;
+
+CREATE DATASET Nation(NationType) PRIMARY KEY n_nationkey;
+
+CREATE DATASET Part(PartType) PRIMARY KEY p_partkey;
+
+CREATE DATASET Partsupp(PartSuppType) PRIMARY KEY ps_partkey,ps_suppkey;
+
+CREATE DATASET Customer(CustomerType) PRIMARY KEY c_custkey;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.02.update.sqlpp
new file mode 100644
index 0000000..62a6ef4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.02.update.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE tpch;
+
+LOAD DATASET LineItem USING localfs ((`path`=`asterix_nc1://data/tpch0.001/lineitem.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`));
+
+LOAD DATASET Orders USING localfs ((`path`=`asterix_nc1://data/tpch0.001/orders.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`));
+
+LOAD DATASET Supplier USING localfs ((`path`=`asterix_nc1://data/tpch0.001/supplier.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`));
+
+LOAD DATASET Region USING localfs ((`path`=`asterix_nc1://data/tpch0.001/region.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`));
+
+LOAD DATASET Nation USING localfs ((`path`=`asterix_nc1://data/tpch0.001/nation.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`));
+
+LOAD DATASET Part USING localfs ((`path`=`asterix_nc1://data/tpch0.001/part.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`));
+
+LOAD DATASET Partsupp USING localfs ((`path`=`asterix_nc1://data/tpch0.001/partsupp.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`));
+
+LOAD DATASET Customer USING localfs ((`path`=`asterix_nc1://data/tpch0.001/customer.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.03.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.03.query.sqlpp
new file mode 100644
index 0000000..f04553e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.03.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE tpch;
+
+-- Disabled for a simpler plan
+SET `compiler.sort.parallel` "false";
+SET `compiler.parallelism` "0";
+
+
+SELECT n.n_nationkey, s.s_nationkey, c.c_nationkey
+FROM Nation n, Supplier s, Customer c
+WHERE s.s_nationkey = n.n_nationkey
+ AND c.c_nationkey = n.n_nationkey
+ AND s.s_nationkey = c.c_nationkey
+ORDER BY n.n_nationkey, s.s_nationkey, c.c_nationkey
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.04.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.04.query.sqlpp
new file mode 100644
index 0000000..6de6bc9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.04.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE tpch;
+
+-- Disabled for a simpler plan
+SET `compiler.sort.parallel` "false";
+SET `compiler.parallelism` "0";
+
+EXPLAIN
+SELECT n.n_nationkey, s.s_nationkey, c.c_nationkey
+FROM Nation n, Supplier s, Customer c
+WHERE s.s_nationkey = n.n_nationkey
+ AND c.c_nationkey = n.n_nationkey
+ AND s.s_nationkey = c.c_nationkey
+ORDER BY n.n_nationkey, s.s_nationkey, c.c_nationkey
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.05.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.05.query.sqlpp
new file mode 100644
index 0000000..336a545
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.05.query.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * 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.
+ */
+
+/*
+ * This query does not make a logical sense; however,
+ * the idea here is to test that a hash-join can be used when we have a case
+ * where one relation uses one attribute to join on two attributes on the
+ * the other relation
+ */
+
+USE tpch;
+
+-- Disabled for a simpler plan
+SET `compiler.sort.parallel` "false";
+-- To prevent plan changes
+SET `compiler.parallelism` "0";
+
+
+SELECT o.o_orderkey, l.l_orderkey, l.l_suppkey
+FROM Orders o, LineItem l
+WHERE o.o_orderkey = l.l_orderkey
+ AND o.o_orderkey = l.l_suppkey
+ORDER BY o.o_orderkey, l.l_orderkey, l.l_suppkey
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.06.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.06.query.sqlpp
new file mode 100644
index 0000000..271a528
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.06.query.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * 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.
+ */
+
+/*
+ * This query does not make a logical sense; however,
+ * the idea here is to test that a hash-join can be used when we have a case
+ * where one relation uses one attribute to join on two attributes on the
+ * the other relation
+ */
+
+USE tpch;
+
+-- Disabled for a simpler plan
+SET `compiler.sort.parallel` "false";
+-- To prevent plan changes
+SET `compiler.parallelism` "0";
+
+EXPLAIN
+SELECT o.o_orderkey, l.l_orderkey, l.l_suppkey
+FROM Orders o, LineItem l
+WHERE o.o_orderkey = l.l_orderkey
+ AND o.o_orderkey = l.l_suppkey
+ORDER BY o.o_orderkey, l.l_orderkey, l.l_suppkey
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.07.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.07.query.sqlpp
new file mode 100644
index 0000000..5bd7308
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.07.query.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * 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.
+ */
+
+/*
+ * This query does not make a logical sense; however,
+ * the idea here is to test that a hash-join can be used when we have a case
+ * where one relation uses one attribute to join on two attributes on the
+ * the other relation
+ */
+
+USE tpch;
+
+-- Disabled for a simpler plan
+SET `compiler.sort.parallel` "false";
+SET `compiler.parallelism` "0";
+
+
+SELECT o.o_orderkey, l.l_orderkey, l.l_suppkey
+FROM Orders o
+LEFT OUTER JOIN LineItem l
+ ON o.o_orderkey = l.l_orderkey
+ AND o.o_orderkey = l.l_suppkey
+ORDER BY o.o_orderkey, l.l_orderkey, l.l_suppkey
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.08.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.08.query.sqlpp
new file mode 100644
index 0000000..5be6513
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.08.query.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * 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.
+ */
+
+/*
+ * This query does not make a logical sense; however,
+ * the idea here is to test that a hash-join can be used when we have a case
+ * where one relation uses one attribute to join on two attributes on the
+ * the other relation
+ */
+
+USE tpch;
+
+-- Disabled for a simpler plan
+SET `compiler.sort.parallel` "false";
+SET `compiler.parallelism` "0";
+
+EXPLAIN
+SELECT o.o_orderkey, l.l_orderkey, l.l_suppkey
+FROM Orders o
+LEFT OUTER JOIN LineItem l
+ ON o.o_orderkey = l.l_orderkey
+ AND o.o_orderkey = l.l_suppkey
+ORDER BY o.o_orderkey, l.l_orderkey, l.l_suppkey
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.09.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.09.query.sqlpp
new file mode 100644
index 0000000..ee5a92d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.09.query.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 tpch;
+
+-- Disabled for a simpler plan
+SET `compiler.sort.parallel` "false";
+-- To ensure that Nation's key is hash partitioned
+SET `compiler.parallelism` "3";
+-- To disable reordering joins
+SET `compiler.cbo` "true";
+
+
+SELECT n.n_nationkey, s.s_nationkey, c.c_nationkey
+FROM Nation n, Supplier s, Customer c
+WHERE s.s_nationkey = n.n_nationkey
+ AND c.c_nationkey = n.n_nationkey
+ AND s.s_nationkey = c.c_nationkey
+ORDER BY n.n_nationkey, s.s_nationkey, c.c_nationkey
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.10.query.sqlpp
new file mode 100644
index 0000000..5a13fc3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.10.query.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 tpch;
+
+-- Disabled for a simpler plan
+SET `compiler.sort.parallel` "false";
+-- To ensure that Nation's key is hash partitioned
+SET `compiler.parallelism` "3";
+-- To disable reordering joins
+SET `compiler.cbo` "true";
+
+EXPLAIN
+SELECT n.n_nationkey, s.s_nationkey, c.c_nationkey
+FROM Nation n, Supplier s, Customer c
+WHERE s.s_nationkey = n.n_nationkey
+ AND c.c_nationkey = n.n_nationkey
+ AND s.s_nationkey = c.c_nationkey
+ORDER BY n.n_nationkey, s.s_nationkey, c.c_nationkey
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.11.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.11.query.sqlpp
new file mode 100644
index 0000000..59f3fb8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.11.query.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 tpch;
+
+-- Disabled for a simpler plan
+SET `compiler.sort.parallel` "false";
+-- To ensure that Nation's key is hash partitioned
+SET `compiler.parallelism` "-1";
+-- To disable reordering joins
+SET `compiler.cbo` "true";
+
+
+SELECT n.n_nationkey, s.s_nationkey, c.c_nationkey
+FROM Nation n, Supplier s, Customer c
+WHERE s.s_nationkey = n.n_nationkey
+ AND c.c_nationkey = n.n_nationkey
+ AND s.s_nationkey = c.c_nationkey
+ORDER BY n.n_nationkey, s.s_nationkey, c.c_nationkey
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.12.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.12.query.sqlpp
new file mode 100644
index 0000000..85077fe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.12.query.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 tpch;
+
+-- Disabled for a simpler plan
+SET `compiler.sort.parallel` "false";
+-- To ensure that Nation's key is hash partitioned
+SET `compiler.parallelism` "-1";
+-- To disable reordering joins
+SET `compiler.cbo` "true";
+
+EXPLAIN
+SELECT n.n_nationkey, s.s_nationkey, c.c_nationkey
+FROM Nation n, Supplier s, Customer c
+WHERE s.s_nationkey = n.n_nationkey
+ AND c.c_nationkey = n.n_nationkey
+ AND s.s_nationkey = c.c_nationkey
+ORDER BY n.n_nationkey, s.s_nationkey, c.c_nationkey
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.13.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.13.query.sqlpp
new file mode 100644
index 0000000..9f52f75
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.13.query.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * 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.
+ */
+/*
+ * Similar to hash-join-with-redundant-variable.04.query.sqlpp
+ * But with Index NL. The plan of this test should has two
+ * hash-partition-exchange -- both of which with one variable.
+ * One from 'Supplier' and one from 'Customer'. Nation should not be
+ * partitioned here to join with Customer (already partitioned by its PK)
+ */
+
+USE tpch;
+
+-- Disabled for a simpler plan
+SET `compiler.sort.parallel` "false";
+SET `compiler.parallelism` "0";
+
+
+SELECT n.n_nationkey, s.s_nationkey, c.c_nationkey
+FROM Supplier s, Nation n, Customer c
+WHERE s.s_nationkey /* +indexnl */ = n.n_nationkey
+ AND c.c_nationkey = n.n_nationkey
+ AND s.s_nationkey = c.c_nationkey
+ORDER BY n.n_nationkey, s.s_nationkey, c.c_nationkey
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.14.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.14.query.sqlpp
new file mode 100644
index 0000000..379055a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.14.query.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * 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.
+ */
+/*
+ * Similar to hash-join-with-redundant-variable.04.query.sqlpp
+ * But with Index NL. The plan of this test should has two
+ * hash-partition-exchange -- both of which with one variable.
+ * One from 'Supplier' and one from 'Customer'. Nation should not be
+ * partitioned here to join with Customer (already partitioned by its PK)
+ */
+
+USE tpch;
+
+-- Disabled for a simpler plan
+SET `compiler.sort.parallel` "false";
+SET `compiler.parallelism` "0";
+
+EXPLAIN
+SELECT n.n_nationkey, s.s_nationkey, c.c_nationkey
+FROM Supplier s, Nation n, Customer c
+WHERE s.s_nationkey /* +indexnl */ = n.n_nationkey
+ AND c.c_nationkey = n.n_nationkey
+ AND s.s_nationkey = c.c_nationkey
+ORDER BY n.n_nationkey, s.s_nationkey, c.c_nationkey
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.15.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.15.query.sqlpp
new file mode 100644
index 0000000..c77b25e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.15.query.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.
+ */
+/*
+ * Similar to hash-join-with-redundant-variable.04.query.sqlpp
+ * But with Index NL. The plan of this test should has three
+ * hash-partition-exchange (as opposed to test 13 & 14). Because the parallelism
+ * is set to 3, then the last join requires both sides to be hash partitioned.
+ * Customer will need to duplicate its variable to join both with Nation and Supplier.
+ * This is the effect of using Index NL with parallelism != # of partitions
+ */
+
+USE tpch;
+
+-- Disabled for a simpler plan
+SET `compiler.sort.parallel` "false";
+SET `compiler.parallelism` "3";
+
+
+SELECT n.n_nationkey, s.s_nationkey, c.c_nationkey
+FROM Supplier s, Nation n, Customer c
+WHERE s.s_nationkey /* +indexnl */ = n.n_nationkey
+ AND c.c_nationkey = n.n_nationkey
+ AND s.s_nationkey = c.c_nationkey
+ORDER BY n.n_nationkey, s.s_nationkey, c.c_nationkey
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.16.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.16.query.sqlpp
new file mode 100644
index 0000000..90aff2e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.16.query.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.
+ */
+/*
+ * Similar to hash-join-with-redundant-variable.04.query.sqlpp
+ * But with Index NL. The plan of this test should has three
+ * hash-partition-exchange (as opposed to test 13 & 14). Because the parallelism
+ * is set to 3, then the last join requires both sides to be hash partitioned.
+ * Customer will need to duplicate its variable to join both with Nation and Supplier.
+ * This is the effect of using Index NL with parallelism != # of partitions
+ */
+
+USE tpch;
+
+-- Disabled for a simpler plan
+SET `compiler.sort.parallel` "false";
+SET `compiler.parallelism` "3";
+
+EXPLAIN
+SELECT n.n_nationkey, s.s_nationkey, c.c_nationkey
+FROM Supplier s, Nation n, Customer c
+WHERE s.s_nationkey /* +indexnl */ = n.n_nationkey
+ AND c.c_nationkey = n.n_nationkey
+ AND s.s_nationkey = c.c_nationkey
+ORDER BY n.n_nationkey, s.s_nationkey, c.c_nationkey
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/inner_right_corr/inner_right_corr.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/inner_right_corr/inner_right_corr.1.ddl.sqlpp
new file mode 100644
index 0000000..1bea9c2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/inner_right_corr/inner_right_corr.1.ddl.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.
+ */
+
+/*
+ * Test plan when right branch of an inner join uses an outer variable.
+ * Currently this results in NL join
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create dataset t1(id bigint not unknown) open type primary key id;
+create dataset t2(id uuid not unknown) open type primary key id autogenerated;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/inner_right_corr/inner_right_corr.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/inner_right_corr/inner_right_corr.2.update.sqlpp
new file mode 100644
index 0000000..7f3bae1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/inner_right_corr/inner_right_corr.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.
+ */
+
+/*
+ * Test plan when right branch of an inner join uses an outer variable.
+ * Currently this results in NL join
+ */
+
+use test;
+
+insert into t1([
+ {"id": 1, "x": [{"b": 1}, {"b": 2}, {"b": 1}, {"b": 2}, {"b": 4}, {"b": 3}, {"b": 5}] },
+ {"id": 2, "x": [{"b": 9}, {"b": 2}, {"b": 7}, {"b": 2}, {"b": 4}, {"b": 3}, {"b": 5}] }
+]);
+insert into t2([
+ {"y": 1},
+ {"y": 3}
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/inner_right_corr/inner_right_corr.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/inner_right_corr/inner_right_corr.3.query.sqlpp
new file mode 100644
index 0000000..aea825f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/inner_right_corr/inner_right_corr.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.
+ */
+
+/*
+ * Test plan when right branch of an inner join uses an outer variable.
+ * Currently this results in NL join
+ */
+
+use test;
+
+select a
+from t1
+let a = (select value count(*) from t2 join t1.x as z on t2.y = z.b )
+order by t1.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj_partitioning_property/nlj_partitioning_property.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj_partitioning_property/nlj_partitioning_property.1.ddl.sqlpp
new file mode 100644
index 0000000..ab0aa32
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj_partitioning_property/nlj_partitioning_property.1.ddl.sqlpp
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Test the fix for nested loop join delivered partitioning property ASTERIXDB-3066
+ */
+
+DROP DATAVERSE tpch IF EXISTS;
+CREATE DATAVERSE tpch;
+
+USE tpch;
+
+CREATE TYPE tpch.SupplierType AS
+ CLOSED {
+ s_suppkey : bigint,
+ s_name : string,
+ s_address : string,
+ s_nationkey : bigint,
+ s_phone : string,
+ s_acctbal : double,
+ s_comment : string
+};
+
+CREATE TYPE tpch.PartType AS
+ CLOSED {
+ p_partkey : bigint,
+ p_name : string,
+ p_mfgr : string,
+ p_brand : string,
+ p_type : string,
+ p_size : bigint,
+ p_container : string,
+ p_retailprice : double,
+ p_comment : string
+};
+
+CREATE TYPE tpch.PartSuppType AS
+ CLOSED {
+ ps_partkey : bigint,
+ ps_suppkey : bigint,
+ ps_availqty : bigint,
+ ps_supplycost : double,
+ ps_comment : string
+};
+
+CREATE DATASET Supplier(SupplierType) PRIMARY KEY s_suppkey;
+CREATE DATASET Part(PartType) PRIMARY KEY p_partkey;
+CREATE DATASET Partsupp(PartSuppType) PRIMARY KEY ps_partkey,ps_suppkey;
+
+CREATE INDEX partsupp_fk_part ON Partsupp (ps_partkey);
+CREATE INDEX partsupp_fk_supplier ON Partsupp (ps_suppkey);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj_partitioning_property/nlj_partitioning_property.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj_partitioning_property/nlj_partitioning_property.2.update.sqlpp
new file mode 100644
index 0000000..b7396c3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj_partitioning_property/nlj_partitioning_property.2.update.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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 tpch;
+
+LOAD DATASET Supplier USING localfs ((`path`=`asterix_nc1://data/tpch0.001/supplier.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`));
+LOAD DATASET Part USING localfs ((`path`=`asterix_nc1://data/tpch0.001/part.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`));
+LOAD DATASET Partsupp USING localfs ((`path`=`asterix_nc1://data/tpch0.001/partsupp.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj_partitioning_property/nlj_partitioning_property.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj_partitioning_property/nlj_partitioning_property.3.query.sqlpp
new file mode 100644
index 0000000..ec57b20
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj_partitioning_property/nlj_partitioning_property.3.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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 tpch;
+
+FROM Supplier s JOIN (FROM Partsupp ps, Part p SELECT ps.ps_suppkey AS suppkey) AS p_ps
+ON s.s_suppkey /*+ hash-bcast */ = p_ps.suppkey
+SELECT count(*) AS count;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj_partitioning_property/nlj_partitioning_property.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj_partitioning_property/nlj_partitioning_property.4.query.sqlpp
new file mode 100644
index 0000000..20eeff2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj_partitioning_property/nlj_partitioning_property.4.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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 tpch;
+
+FROM Supplier s JOIN (FROM Partsupp ps, Part p SELECT ps.ps_suppkey AS suppkey) AS p_ps
+ON s.s_suppkey = p_ps.suppkey
+SELECT count(*) AS count;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj_partitioning_property/nlj_partitioning_property.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj_partitioning_property/nlj_partitioning_property.5.query.sqlpp
new file mode 100644
index 0000000..cb3e800
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj_partitioning_property/nlj_partitioning_property.5.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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 tpch;
+
+FROM Supplier s JOIN (FROM Partsupp ps, Part p SELECT ps.ps_suppkey AS suppkey) AS p_ps ON TRUE
+SELECT count(*) AS count;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit-non-pure-function/limit-non-pure-function.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit-non-pure-function/limit-non-pure-function.1.query.sqlpp
new file mode 100644
index 0000000..c23e98c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit-non-pure-function/limit-non-pure-function.1.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+
+EXPLAIN
+SELECT VALUE A
+FROM [1, 2, 3] AS A
+LIMIT random()
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit-non-pure-function/limit-non-pure-function.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit-non-pure-function/limit-non-pure-function.2.query.sqlpp
new file mode 100644
index 0000000..9c995e0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit-non-pure-function/limit-non-pure-function.2.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Make sure that the call to random() (rand) is not inlined
+ */
+
+EXPLAIN
+WITH rand AS random()
+SELECT A, rand
+FROM [1, 2, 3] AS A
+LIMIT rand
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.3.query.sqlpp
index b8eac5d..7f50a08 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.3.query.sqlpp
@@ -18,7 +18,7 @@
*/
use test;
-
+SET `compiler.cbo` "true";
explain select element c
from LineItem as c
where c.l_suppkey < 150 AND l_extendedprice < 10000
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.5.query.sqlpp
index ab6451b..f68ec55 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.5.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.5.query.sqlpp
@@ -18,7 +18,7 @@
*/
use test;
-
+SET `compiler.cbo` "true";
explain
select
substring(c.l_shipdate, 0, 4) as shipdate,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.3.query.sqlpp
index 43f7d94..4df8ecd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.3.query.sqlpp
@@ -18,7 +18,7 @@
*/
use test;
-
+SET `compiler.cbo` "true";
explain select element c
from LineItem as c
where (c.l_suppkey < 150)
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.5.query.sqlpp
index 5673992..f9657e4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.5.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.5.query.sqlpp
@@ -18,7 +18,7 @@
*/
use test;
-
+SET `compiler.cbo` "true";
explain select element c
from LineItem as c
where (c.l_suppkey < 150)
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index/secondary-index.3.metrics.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index/secondary-index.3.metrics.sqlpp
index 98166d7..49e3df2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index/secondary-index.3.metrics.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index/secondary-index.3.metrics.sqlpp
@@ -25,6 +25,10 @@
use test;
set `compiler.indexonly` "false";
+set `compiler.cbo` "false";
+
+
+
select count(*) from Customers
where name = "Marvella Loud";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/active_requests/active_requests.2.pollquery.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/active_requests/active_requests.2.pollquery.sqlpp
index 7eee42e..07f8eea 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/active_requests/active_requests.2.pollquery.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/active_requests/active_requests.2.pollquery.sqlpp
@@ -18,5 +18,5 @@
*/
-- param client_context_id=ensure_running_query
-- polltimeoutsecs=15
-SELECT rqst.cancellable, rqst.jobId, rqst.state, rqst.uuid FROM active_requests() rqst
+SELECT rqst.cancellable, rqst.jobId, rqst.state, rqst.uuid, rqst.jobStatus FROM active_requests() rqst
WHERE rqst.clientContextID = 'sleep_async_query';
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/completed_requests/completed_requests.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/completed_requests/completed_requests.4.query.sqlpp
new file mode 100644
index 0000000..de35939
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/completed_requests/completed_requests.4.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+ -- param client_context_id=ensure_completed_query
+ -- param ignoreextrafields=true
+SELECT VALUE (SELECT VALUE r FROM completed_requests() r
+WHERE r.state="completed" AND r.clientContextID = "completed_requests_query")[0];
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/constant_folding/constant_folding.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/constant_folding/constant_folding.7.query.sqlpp
index 517a996..dfb47e9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/constant_folding/constant_folding.7.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/constant_folding/constant_folding.7.query.sqlpp
@@ -19,6 +19,10 @@
/*
* Description: No constant folding of OR with a non functional argument
+ *
+ * Update/note: this should be folded as the non functional argument would short-circuited in runtime. Due to
+ * ASTERIXDB-3103, the constant folding rule doesn't see a non functional argument, but a variable. Hence,
+ * the expression 'get_year(current_date()) < x' is eliminated
*/
explain select value true or get_year(current_date()) < x
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/dump_index/dump_index.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/dump_index/dump_index.1.ddl.sqlpp
index 2e97872..00de4a4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/dump_index/dump_index.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/dump_index/dump_index.1.ddl.sqlpp
@@ -20,5 +20,9 @@
CREATE DATAVERSE test;
USE test;
CREATE TYPE t1 AS {id:int, name:string?};
+CREATE TYPE t2 AS {id:int};
+
CREATE DATASET ds(t1) PRIMARY KEY id;
-CREATE INDEX name_idx ON ds(name);
\ No newline at end of file
+CREATE INDEX name_idx ON ds(name);
+
+CREATE DATASET ds2(t2) PRIMARY KEY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/dump_index/dump_index.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/dump_index/dump_index.10.query.sqlpp
new file mode 100644
index 0000000..0c228fc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/dump_index/dump_index.10.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.
+ */
+SET `import-private-functions` `true`;
+USE test;
+SELECT VALUE DUMP_INDEX("test", "ds2", "sample_idx_1_ds2");
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/dump_index/dump_index.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/dump_index/dump_index.2.update.sqlpp
index 21ce2b6..3469c9a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/dump_index/dump_index.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/dump_index/dump_index.2.update.sqlpp
@@ -17,4 +17,7 @@
* under the License.
*/
USE test;
-INSERT INTO ds ([{"id":1, "name": "name1"}, {"id":2, "name": "name2"}]);
\ No newline at end of file
+INSERT INTO ds ([{"id":1, "name": "name1"}, {"id":2, "name": "name2"}]);
+UPSERT INTO ds2 ([{"id": 1,"age":"t\\'s", "a1": {"b": [{"x": [1,2]}, {"x": [1,2]}]}, "a2": [{"x": [1,2]}, {"x": [1,2]}]
+}]);
+ANALYZE DATASET ds2;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/metadata_only_02/metadata_only_02.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/metadata_only_02/metadata_only_02.3.query.sqlpp
new file mode 100644
index 0000000..11c7325
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/metadata_only_02/metadata_only_02.3.query.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * test fix for ASTERIXDB-3075
+ */
+
+LET dv_name = ["test2", "test1"], ds_name = ["ds2", "ds1"], syn_name = [ "syn2", "syn1" ],
+synonym_names = (SELECT s.SynonymName, s.ObjectName
+ FROM Metadata.`Synonym` s, syn_name
+ WHERE s.SynonymName = syn_name),
+
+dataset_ds_dv_names = (SELECT d.DatasetName, d.DataverseName
+ FROM Metadata.`Dataset` d, (SELECT * FROM ds_name, dv_name) AS ds_dv_names
+ WHERE d.DatasetName = ds_dv_names.ds_name AND d.DataverseName = ds_dv_names.dv_name),
+
+dataset_dv_ds_names = (SELECT d.DatasetName, d.DataverseName
+ FROM Metadata.`Dataset` d, (SELECT * FROM dv_name, ds_name) AS dv_ds_names
+ WHERE d.DatasetName = dv_ds_names.ds_name AND d.DataverseName = dv_ds_names.dv_name),
+
+left_branch = (SELECT s.SynonymName, s.DataverseName, s.ObjectName
+ FROM Metadata.`Synonym` s LEFT OUTER JOIN dataset_dv_ds_names
+ ON dataset_dv_ds_names.DatasetName = s.ObjectName
+ ORDER BY s.DataverseName, s.SynonymName),
+
+right_branch = (SELECT synonym_names.SynonymName
+ FROM synonym_names LEFT OUTER JOIN dataset_ds_dv_names
+ ON dataset_ds_dv_names.DatasetName = synonym_names.ObjectName)
+
+FROM left_branch lb LEFT OUTER JOIN right_branch rb
+ON lb.ObjectName = rb.SynonymName
+SELECT lb.DataverseName, lb.SynonymName
+ORDER BY lb.DataverseName, lb.SynonymName;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/metadata_only_02/metadata_only_02.9.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/metadata_only_02/metadata_only_02.9.ddl.sqlpp
new file mode 100644
index 0000000..bd99c9d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/metadata_only_02/metadata_only_02.9.ddl.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.
+ */
+
+drop dataverse test1 if exists;
+drop dataverse test2 if exists;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3316/query-ASTERIXDB-3316.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3316/query-ASTERIXDB-3316.1.ddl.sqlpp
new file mode 100644
index 0000000..fbc5858
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3316/query-ASTERIXDB-3316.1.ddl.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: This test case is to verify the fix for ASTERIXDB-2947
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type dt1 as {id:int};
+create dataset collection0(dt1) primary key id;
+create dataset collection1(dt1) primary key id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3316/query-ASTERIXDB-3316.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3316/query-ASTERIXDB-3316.2.update.sqlpp
new file mode 100644
index 0000000..2ab1a10
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3316/query-ASTERIXDB-3316.2.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into collection0
+([
+ {"id": 1, "x_id":5, "y_id":5, "a":12, "b":20, "to_u":"aaaaa", "posi":"a", "u":"col0"}
+]);
+
+insert into collection1
+([
+ {"id": 1, "x_id":5, "y_id":5, "a":12, "b":20, "to_u":"aaaaa", "posi":"a", "u":"col1" }
+]);
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3316/query-ASTERIXDB-3316.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3316/query-ASTERIXDB-3316.3.query.sqlpp
new file mode 100644
index 0000000..09fdcda
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3316/query-ASTERIXDB-3316.3.query.sqlpp
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: This test case is to verify the fix for ASTERIXDB-3316
+ */
+
+use test;
+
+SELECT DISTINCT T1.*,
+ (SELECT H.u
+ FROM collection1 H
+ WHERE H.y_id = T1.x_id
+ AND H.a IN [12, 66, 67, 13, 26]
+ AND H.to_u = 'aaaaa'
+ AND H.b in
+ (SELECT value MAX(L.b)
+ FROM collection1 L
+ WHERE L.y_id = T1.x_id
+ AND L.a IN [12, 66, 67, 13, 26]
+ AND L.to_u = 'aaaaa')
+ ) AS sub_query1,
+ (SELECT H.u
+ FROM collection1 H
+ WHERE H.y_id = T1.x_id
+ AND H.a IN [12, 66, 67, 13]
+ AND H.posi IN ['a','b']
+ AND H.b in
+ (SELECT value MAX(L.b)
+ FROM collection1 L
+ WHERE L.y_id = T1.x_id
+ AND L.posi IN ['a','b'])
+ ) AS sub_query2
+FROM (SELECT T0.x_id
+ FROM collection0 T0
+) T1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3334/query-ASTERIXDB-3334.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3334/query-ASTERIXDB-3334.1.ddl.sqlpp
new file mode 100644
index 0000000..ac587d4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3334/query-ASTERIXDB-3334.1.ddl.sqlpp
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: This test case is to verify the fix for ASTERIXDB-2947
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type dt1 as {Id:int};
+create dataset collection0(dt1) primary key Id;
+
+CREATE VIEW `Staples`(
+`Item Count` BIGINT NOT UNKNOWN, `Ship Priority` STRING NOT UNKNOWN, `Order Priority` STRING NOT UNKNOWN,
+`Order Status` STRING NOT UNKNOWN, `Order Quantity` DOUBLE NOT UNKNOWN, `Sales Total` DOUBLE NOT UNKNOWN,
+`Discount` DOUBLE NOT UNKNOWN, `Tax Rate` DOUBLE NOT UNKNOWN, `Ship Mode` STRING NOT UNKNOWN, `Fill Time` DOUBLE NOT UNKNOWN,
+`Gross Profit` DOUBLE NOT UNKNOWN, `Price` DOUBLE NOT UNKNOWN, `Ship Handle Cost` DOUBLE NOT UNKNOWN, `Employee Name` STRING NOT UNKNOWN,
+`Employee Dept` STRING NOT UNKNOWN, `Manager Name` STRING NOT UNKNOWN, `Employee Yrs Exp` DOUBLE NOT UNKNOWN, `Employee Salary` DOUBLE NOT UNKNOWN,
+`Customer Name` STRING NOT UNKNOWN, `Customer State` STRING NOT UNKNOWN, `Call Center Region` STRING NOT UNKNOWN, `Customer Balance` DOUBLE NOT UNKNOWN,
+`Customer Segment` STRING NOT UNKNOWN, `Prod Type1` STRING NOT UNKNOWN, `Prod Type2` STRING NOT UNKNOWN, `Prod Type3` STRING NOT UNKNOWN,
+`Prod Type4` STRING NOT UNKNOWN, `Product Name` STRING NOT UNKNOWN, `Product Container` STRING NOT UNKNOWN, `Ship Promo` STRING NOT UNKNOWN,
+`Supplier Name` STRING NOT UNKNOWN, `Supplier Balance` DOUBLE NOT UNKNOWN, `Supplier Region` STRING NOT UNKNOWN, `Supplier State` STRING NOT UNKNOWN,
+`Order ID` STRING NOT UNKNOWN, `Order Year` BIGINT NOT UNKNOWN, `Order Month` BIGINT NOT UNKNOWN, `Order Day` BIGINT NOT UNKNOWN,
+`Order Date` DATETIME NOT UNKNOWN, `Order Quarter` STRING NOT UNKNOWN, `Product Base Margin` DOUBLE NOT UNKNOWN, `Product ID` STRING NOT UNKNOWN,
+`Receive Time` DOUBLE NOT UNKNOWN, `Received Date` DATETIME NOT UNKNOWN, `Ship Date` DATETIME NOT UNKNOWN, `Ship Charge` DOUBLE NOT UNKNOWN,
+`Total Cycle Time` DOUBLE NOT UNKNOWN, `Product In Stock` STRING NOT UNKNOWN, `PID` BIGINT NOT UNKNOWN, `Market Segment` STRING NOT UNKNOWN)
+default NULL AS `collection0`
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3334/query-ASTERIXDB-3334.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3334/query-ASTERIXDB-3334.2.update.sqlpp
new file mode 100644
index 0000000..c971b27
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3334/query-ASTERIXDB-3334.2.update.sqlpp
@@ -0,0 +1,132 @@
+/*
+ * 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 collection0
+([
+ {
+ "Id": 1,
+ "Item Count": 1,
+ "Ship Priority": "0",
+ "Order Priority": "1-URGENT",
+ "Order Status": "F",
+ "Order Quantity": 38,
+ "Sales Total": 195.19,
+ "Discount": 0.08,
+ "Tax Rate": 0.03,
+ "Ship Mode": "REGULAR AIR",
+ "Fill Time": 2,
+ "Gross Profit": -71.33,
+ "Price": 5.28,
+ "Ship Handle Cost": 5.06,
+ "Employee Name": "Purkey, Jan",
+ "Employee Dept": "1054",
+ "Manager Name": "Collins, Babs",
+ "Employee Yrs Exp": 6,
+ "Employee Salary": 104677,
+ "Customer Name": "Roy Skaria",
+ "Customer State": "OHIO",
+ "Call Center Region": "CENTRAL",
+ "Customer Balance": 6496,
+ "Customer Segment": "CONSUMER",
+ "Prod Type1": "OFFICE SUPPLIES",
+ "Prod Type2": "PAPER",
+ "Prod Type3": "STATIONARY",
+ "Prod Type4": "BUSINESS STATIONARY",
+ "Product Name": "Astroparche® Fine Business Paper",
+ "Product Container": "SMALL BOX",
+ "Ship Promo": "REGULAR SHIPPING",
+ "Supplier Name": "Supplier_098",
+ "Supplier Balance": 5873,
+ "Supplier Region": "WEST",
+ "Supplier State": "CALIFORNIA",
+ "Order ID": "21284",
+ "Order Year": 1997,
+ "Order Month": 9,
+ "Order Day": 15,
+ "Order Date": "1997-09-15T00:00:00.000",
+ "Order Quarter": "Q3",
+ "Product Base Margin": 0.37,
+ "Product ID": "1811",
+ "Receive Time": 3,
+ "Received Date": "1997-09-20T00:00:00.000",
+ "Ship Date": "1997-09-17T00:00:00.000",
+ "Ship Charge": 5.06,
+ "Total Cycle Time": 5,
+ "Product In Stock": "YES",
+ "PID": 6632,
+ "Market Segment": "CONSUMER"
+ },
+{
+ "Id": 2,
+ "Item Count": 1,
+ "Ship Priority": "0",
+ "Order Priority": "4-NOT SPECIFIED",
+ "Order Status": "F",
+ "Order Quantity": 36,
+ "Sales Total": 603.9,
+ "Discount": 0.07,
+ "Tax Rate": 0.02,
+ "Ship Mode": "REGULAR AIR",
+ "Fill Time": 1,
+ "Gross Profit": 168.93,
+ "Price": 17.52,
+ "Ship Handle Cost": 5.6,
+ "Employee Name": "Purkey, Jan",
+ "Employee Dept": "1054",
+ "Manager Name": "Collins, Babs",
+ "Employee Yrs Exp": 6,
+ "Employee Salary": 104677,
+ "Customer Name": "Nathan Mautz",
+ "Customer State": "ILLINOIS",
+ "Call Center Region": "CENTRAL",
+ "Customer Balance": 1820,
+ "Customer Segment": "HOME OFFICE",
+ "Prod Type1": "OFFICE SUPPLIES",
+ "Prod Type2": "PAPER",
+ "Prod Type3": "WRITINGS PADS",
+ "Prod Type4": "MEMO SLIPS",
+ "Product Name": "TOPS Carbonless Receipt Book, Four 2-3/4 x 7-1/4 Money Receipts per Page",
+ "Product Container": "WRAP BAG",
+ "Ship Promo": "REGULAR SHIPPING",
+ "Supplier Name": "Supplier_090",
+ "Supplier Balance": 6202,
+ "Supplier Region": "WEST",
+ "Supplier State": "NEW MEXICO",
+ "Order ID": "1250",
+ "Order Year": 1997,
+ "Order Month": 9,
+ "Order Day": 29,
+ "Order Date": "1997-09-29T00:00:00.000",
+ "Order Quarter": "Q3",
+ "Product Base Margin": 0.37,
+ "Product ID": "1860",
+ "Receive Time": 2,
+ "Received Date": "1997-10-02T00:00:00.000",
+ "Ship Date": "1997-09-30T00:00:00.000",
+ "Ship Charge": 5.6,
+ "Total Cycle Time": 3,
+ "Product In Stock": "YES",
+ "PID": 6981,
+ "Market Segment": "HOME OFFICE"
+}
+]);
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3334/query-ASTERIXDB-3334.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3334/query-ASTERIXDB-3334.3.query.sqlpp
new file mode 100644
index 0000000..b6104e5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3334/query-ASTERIXDB-3334.3.query.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: This test case is to verify the fix for ASTERIXDB-3316
+ */
+
+use test;
+
+SELECT `Staples`.`Employee Name` AS `Employee Name`,
+ AVG(`Staples`.`Employee Salary`) AS `avg:Employee Salary:ok`
+FROM `Staples` `Staples`
+ INNER JOIN (
+ SELECT `Staples`.`Call Center Region` AS `Call Center Region`,
+ `Staples`.`Employee Name` AS `Employee Name`
+ FROM `Staples` `Staples`
+ GROUP BY `Staples`.`Call Center Region`,
+ `Staples`.`Employee Name`
+ HAVING ((AVG(`Staples`.`Employee Salary`)>=102499.99999999898) AND (AVG(`Staples`.`Employee Salary`)<=110000.00000000111))
+) `t0` ON ((`Staples`.`Call Center Region` = `t0`.`Call Center Region`) AND (`Staples`.`Employee Name` = `t0`.`Employee Name`))
+GROUP BY `Staples`.`Employee Name`
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3403/query-ASTERIXDB-3403.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3403/query-ASTERIXDB-3403.1.ddl.sqlpp
new file mode 100644
index 0000000..0b94eb1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3403/query-ASTERIXDB-3403.1.ddl.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-3403
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type dt1 as {id:int};
+create dataset collection1(dt1) primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3403/query-ASTERIXDB-3403.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3403/query-ASTERIXDB-3403.2.update.sqlpp
new file mode 100644
index 0000000..327d1d6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3403/query-ASTERIXDB-3403.2.update.sqlpp
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into collection1
+([
+ {
+ "id": 1,
+ "f1": "f1",
+ "x1": [{"date":"01-02-2024", "item": '1234', "cnt":2}]
+
+ },
+ {
+ "id": 2,
+ "f1": "f1",
+ "x2": [{"date":"01-02-2024", "item": '5678', "cnt":2}]
+ },
+ {
+ "id": 3,
+ "f1": "f1",
+ "x3": [{"su": {"x4":2}, "item": "i1", "cnt":2}]
+ },
+ {
+ "id": 4,
+ "f1": "f1",
+ "x3": [{"su": {"x4":5}, "item": 1234, "cnt":2}]
+ }
+]);
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3403/query-ASTERIXDB-3403.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3403/query-ASTERIXDB-3403.3.query.sqlpp
new file mode 100644
index 0000000..ca70034
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3403/query-ASTERIXDB-3403.3.query.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.
+ */
+
+/*
+ * Description: This test case is to verify the fix for ASTERIXDB-3403
+ */
+
+use test;
+
+ SELECT COUNT(id) AS matches
+ FROM collection1 AS d
+ WHERE d.`f1` = 'f1'
+ AND (ARRAY_SUM((
+ SELECT VALUE i.`cnt`
+ FROM d.`x1` AS i
+ WHERE i.`date` BETWEEN "01-01-2024" AND "02-02-2024"
+ AND i.`item` IN ['1234', '5678'] )) >= 1
+ OR ARRAY_SUM((
+ SELECT VALUE i.`cnt`
+ FROM d.`x2` AS i
+ WHERE i.`date` BETWEEN "01-01-2024" AND "02-02-2024"
+ AND i.`item` IN ['1234', '5678'] )) >= 1
+ OR (ANY e IN d.x3 SATISFIES e.item IN ['i1', 'i2', 'i3']
+ AND e.su.`x4` >= 1 END));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3418/query-ASTERIXDB-3418.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3418/query-ASTERIXDB-3418.1.query.sqlpp
new file mode 100644
index 0000000..f69df6e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3418/query-ASTERIXDB-3418.1.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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-3418
+ */
+
+select value 0.0=-0.0
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3418/query-ASTERIXDB-3418.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3418/query-ASTERIXDB-3418.2.query.sqlpp
new file mode 100644
index 0000000..0080dcf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3418/query-ASTERIXDB-3418.2.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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-3418
+ */
+
+select x.a, count(*) from [{"a":0.0, "b":1}, {"a":-0.0, "b":2}] x group by x.a;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3418/query-ASTERIXDB-3418.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3418/query-ASTERIXDB-3418.3.query.sqlpp
new file mode 100644
index 0000000..f460366
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3418/query-ASTERIXDB-3418.3.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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-3418
+ */
+
+select distinct(x.a) from [{"a":0.0, "b":1}, {"a":-0.0, "b":2}] x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3418/query-ASTERIXDB-3418.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3418/query-ASTERIXDB-3418.4.query.sqlpp
new file mode 100644
index 0000000..e5e3bb7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3418/query-ASTERIXDB-3418.4.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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-3418
+ */
+
+select value -0.0<0.0
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3490/query-ASTERIXDB-3490.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3490/query-ASTERIXDB-3490.1.ddl.sqlpp
new file mode 100644
index 0000000..c240470
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3490/query-ASTERIXDB-3490.1.ddl.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * 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-3490
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type dt1 as {id: int};
+
+create dataset collection1(dt1) PRIMARY KEY id;
+create dataset collection2(dt1) PRIMARY KEY id;
+create dataset collection3(dt1) PRIMARY KEY id;
+create dataset collection4(dt1) PRIMARY KEY id;
+create dataset collection5(dt1) PRIMARY KEY id;
+create dataset collection6(dt1) PRIMARY KEY id;
+
+create dataset dataset0(dt1) PRIMARY KEY id;
+create dataset dataset1(dt1) PRIMARY KEY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3490/query-ASTERIXDB-3490.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3490/query-ASTERIXDB-3490.2.update.sqlpp
new file mode 100644
index 0000000..8b127ab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3490/query-ASTERIXDB-3490.2.update.sqlpp
@@ -0,0 +1,123 @@
+/*
+ * 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;
+
+upsert into collection1
+[
+ {
+ "id":1,
+ "t1":10,
+ "t2":10,
+ "t":10,
+ "p1":3,
+ "p2":3,
+ "p":3
+ }
+];
+
+upsert into collection2
+[
+ {
+ "id":1,
+ "t1":10,
+ "t2":10,
+ "t":10,
+ "p1":3,
+ "p2":3,
+ "p":3
+ }
+];
+
+upsert into collection3
+[
+ {
+ "id":1,
+ "t1":10,
+ "t2":10,
+ "t":10,
+ "p1":3,
+ "p2":3,
+ "p":3
+ }
+];
+
+upsert into collection4
+[
+ {
+ "id":1,
+ "t1":10,
+ "t2":10,
+ "t":10,
+ "p1":3,
+ "p2":3,
+ "p":3
+ }
+];
+
+upsert into collection5
+[
+ {
+ "id":1,
+ "t1":10,
+ "t2":10,
+ "t":10,
+ "p1":3,
+ "p2":3,
+ "p":3
+ }
+];
+
+upsert into collection6
+[
+ {
+ "id":1,
+ "t1":10,
+ "t2":10,
+ "t":10,
+ "p1":3,
+ "p2":3,
+ "p":3
+ }
+];
+
+
+upsert into dataset0
+ [{
+ "id":10,
+ "a":12,
+ "y_id":10,
+ "b":20,
+ "u": 101,
+ "x_id":10,
+ "p":3
+ }];
+
+upsert into dataset1
+ [{
+ "id":1,
+ "a":12,
+ "y_id":10,
+ "b":20,
+ "u": 101,
+ "x_id":10,
+ "p":3
+ }];
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3490/query-ASTERIXDB-3490.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3490/query-ASTERIXDB-3490.3.query.sqlpp
new file mode 100644
index 0000000..f319efc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3490/query-ASTERIXDB-3490.3.query.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: This test case is to verify the fix for ASTERIXDB-3490
+ */
+
+use test;
+
+SELECT value a1
+FROM collection1 a1
+WHERE (
+ SELECT value count(*)
+ FROM collection2 a2
+ WHERE a1.t=a2.t and a1.p=3 and (
+ SELECT value count(*)
+ FROM collection3 a3
+ WHERE a2.t1=a3.t1 and a2.p1=3 and (
+ SELECT value count(*)
+ FROM collection4 a4
+ WHERE a3.t2=a4.t2 and a3.p2=3 and (
+ SELECT value count(*)
+ FROM collection5 a5
+ WHERE a4.t=a5.t and a4.p=3 and (
+ SELECT value count(*)
+ FROM collection6 a6
+ WHERE a5.t1=a6.t1 and a5.p1=3
+ )[0] >= 1
+ )[0] >= 1
+ )[0] >= 1
+ )[0] >= 1
+ )[0] >= 1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3490/query-ASTERIXDB-3490.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3490/query-ASTERIXDB-3490.4.query.sqlpp
new file mode 100644
index 0000000..2313a6d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3490/query-ASTERIXDB-3490.4.query.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * 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-3490
+ */
+
+use test;
+
+SET `rewrite_or_as_join` "false";
+SELECT DISTINCT T1.*,
+ (SELECT VALUE H.u
+ FROM dataset1 H
+ WHERE H.y_id = to_string(T1.x_id)
+ AND H.a IN [12, 66, 67, 13, 26]
+ AND H.b IN
+ (SELECT value MAX(L.b)
+ FROM dataset1 L
+ WHERE L.y_id = T1.x_id
+ AND L.a IN [12, 66, 67, 13, 26]
+ )
+ LIMIT 1
+ )[0] AS sub_q1,
+ b_max
+FROM (SELECT ROW_NUMBER() OVER(ORDER BY sub_q2.b_max DESC NULLS LAST) AS ROW_NUMBER,
+ T0.x_id,
+ sub_q2.b_max
+ FROM dataset0 T0 INNER JOIN
+ (SELECT y_id, MAX(b) b_max
+ FROM dataset1 T2
+ GROUP BY y_id) AS sub_q2
+ ON sub_q2.y_id = T0.id
+) T1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3512/query-ASTERIXDB-3512.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3512/query-ASTERIXDB-3512.1.ddl.sqlpp
new file mode 100644
index 0000000..980d4d4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3512/query-ASTERIXDB-3512.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: This test case is to verify the fix for ASTERIXDB-3512
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type dt1 as {id: int};
+
+create dataset collection1(dt1) PRIMARY KEY id;
+create dataset collection2(dt1) PRIMARY KEY id;
+
+CREATE INDEX idx1 ON collection2 (date: STRING, oid: STRING, sym: STRING, reg: BIGINT);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3512/query-ASTERIXDB-3512.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3512/query-ASTERIXDB-3512.2.update.sqlpp
new file mode 100644
index 0000000..41879a7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3512/query-ASTERIXDB-3512.2.update.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.
+ */
+
+use test;
+
+upsert into collection1
+[
+ {
+ "id":1,
+ "sym":10,
+ "date": "2024-01-01"
+ }
+];
+
+upsert into collection2
+[
+ {
+ "id":1,
+ "sym":10,
+ "date": "2024-01-01"
+ }
+];
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3512/query-ASTERIXDB-3512.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3512/query-ASTERIXDB-3512.3.query.sqlpp
new file mode 100644
index 0000000..60f91ad
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3512/query-ASTERIXDB-3512.3.query.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.
+ */
+
+/*
+ * Description: This test case is to verify the fix for ASTERIXDB-3512
+ */
+
+use test;
+
+with tmp1 AS (
+ SELECT S.sym
+ FROM collection1 AS S
+ WHERE S.date < print_date(current_date(), 'YYYY-MM-DD')),
+ tmp2 AS (
+ SELECT O.sym
+ FROM collection2 AS O
+ WHERE O.date < print_date(current_date(), 'YYYY-MM-DD'))
+SELECT sl1.sym
+FROM tmp1 AS sl1
+UNION ALL
+SELECT sl2.sym
+FROM tmp2 as sl2;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/remove_listify/remove_listify.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/remove_listify/remove_listify.1.ddl.sqlpp
new file mode 100644
index 0000000..498f374
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/remove_listify/remove_listify.1.ddl.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.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type dt as
+{
+ uid : integer
+};
+
+create dataset collection1(dt) primary key uid;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/remove_listify/remove_listify.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/remove_listify/remove_listify.2.update.sqlpp
new file mode 100644
index 0000000..ac2cc1b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/remove_listify/remove_listify.2.update.sqlpp
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into collection1
+([
+
+ {
+ "uid":1,
+ "f1":101,
+ "f2": 24
+ },
+ {
+ "uid":2,
+ "f1":102,
+ "f2": 25
+ },
+ {
+ "uid":3,
+ "f1":102,
+ "f2": 25
+ },
+ {
+ "uid":4,
+ "f1":102,
+ "f2": 31
+ },
+ {
+ "uid":5,
+ "f1":103,
+ "f2": 24
+ },
+ {
+ "uid":6,
+ "f1":103,
+ "f2": 30
+ },
+ {
+ "uid":7,
+ "f1":103,
+ "f2": 31
+ },
+ {
+ "uid":8,
+ "f1":103,
+ "f2": 35
+ },
+ {
+ "uid":9,
+ "f1":103,
+ "f2": 41
+ },
+ {
+ "uid":10,
+ "f1":104,
+ "f2": 42
+ },
+ {
+ "uid":11,
+ "f1":104,
+ "f2": 24
+ },
+ {
+ "uid":12,
+ "f1":104,
+ "f2": 25
+ },
+ {
+ "uid":13,
+ "f1":105,
+ "f2": 24
+ },
+ {
+ "uid":14,
+ "f1":105,
+ "f2": 25
+ },
+ {
+ "uid":15,
+ "f1":105,
+ "f2": "40"
+ }
+])
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/remove_listify/remove_listify.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/remove_listify/remove_listify.3.query.sqlpp
new file mode 100644
index 0000000..8652bba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/remove_listify/remove_listify.3.query.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.
+ */
+
+use test;
+
+SET `rewrite_or_as_join` "false";
+WITH table23 AS(
+ SELECT f1,
+ f2
+ FROM collection1 b),
+table22 AS(
+ SELECT f1,
+ (
+ SELECT COUNT(1) FILTER(WHERE b.f2>=a.f2-5 AND b.f2<=a.f2) AS counts
+ FROM table23 b
+ WHERE b.f1=a.f1
+ )[0] obj
+ FROM table23 a)
+SELECT
+ a.f1, a.obj.counts
+FROM table22 a
+order by a.f1, a.obj.counts;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/serialized_size_fun/serialized_size_fun.01.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/serialized_size_fun/serialized_size_fun.01.query.sqlpp
new file mode 100644
index 0000000..d3c8f1a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/serialized_size_fun/serialized_size_fun.01.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.
+ */
+SET `import-private-functions` `true`;
+
+from [
+{"id": 1, "f": 1},
+{"id": 2, "f": true},
+{"id": 3, "f": "test"},
+{"id": 4, "f": [1,2]},
+{"id": 5, "f": {"f1": 1, "f2": [{"n": "str"}]}}
+] AS t
+select t.f AS val, serialized_size(t.f) AS serialized_size
+order by t.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/ObjectsQueries.xml b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/ObjectsQueries.xml
index 623237b..8532fe9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/ObjectsQueries.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/ObjectsQueries.xml
@@ -244,4 +244,9 @@
<expected-warn>Duplicate field name 'fname1' (in line 25, at column 45)</expected-warn>
</compilation-unit>
</test-case>
+ <test-case FilePath="objects">
+ <compilation-unit name="load-record-fields">
+ <output-dir compare="Text">load-record-fields</output-dir>
+ </compilation-unit>
+ </test-case>
</test-group>
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/load-record-fields/load-record-fields.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/load-record-fields/load-record-fields.1.ddl.sqlpp
new file mode 100644
index 0000000..99898f9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/load-record-fields/load-record-fields.1.ddl.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.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+
+USE test;
+
+CREATE TYPE OpenType AS {
+ id: int
+};
+
+CREATE DATASET MyDataset(OpenType)
+PRIMARY KEY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/load-record-fields/load-record-fields.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/load-record-fields/load-record-fields.2.update.sqlpp
new file mode 100644
index 0000000..1f80ae1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/load-record-fields/load-record-fields.2.update.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;
+
+INSERT INTO MyDataset (
+ {"id": 1, "name": "Alice"},
+ {"id": 2, "name": "Bob"}
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/load-record-fields/load-record-fields.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/load-record-fields/load-record-fields.3.query.sqlpp
new file mode 100644
index 0000000..1ed0c05
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/load-record-fields/load-record-fields.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.
+ */
+
+USE test;
+
+-- Disabled for a simpler plan
+SET `compiler.sort.parallel` "false";
+
+
+SELECT VALUE md.name
+FROM MyDataset md
+LET myObject = {"myUid": uuid()}
+WHERE myObject.myUid != uuid()
+ORDER BY md.id
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/load-record-fields/load-record-fields.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/load-record-fields/load-record-fields.4.query.sqlpp
new file mode 100644
index 0000000..96095b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/load-record-fields/load-record-fields.4.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.
+ */
+
+USE test;
+
+-- Disabled for a simpler plan
+SET `compiler.sort.parallel` "false";
+
+EXPLAIN
+SELECT VALUE md.name
+FROM MyDataset md
+LET myObject = {"myUid": uuid()}
+WHERE myObject.myUid != uuid()
+ORDER BY md.id
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/load-record-fields/load-record-fields.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/load-record-fields/load-record-fields.5.query.sqlpp
new file mode 100644
index 0000000..60f5d4b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/load-record-fields/load-record-fields.5.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.
+ */
+
+USE test;
+
+-- Disabled for a simpler plan
+SET `compiler.sort.parallel` "false";
+
+
+SELECT VALUE md.name
+FROM MyDataset md
+LET currentData = {"myDate": current_date()}
+WHERE currentData.myDate != date("1980-09-10")
+ORDER BY md.id
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/load-record-fields/load-record-fields.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/load-record-fields/load-record-fields.6.query.sqlpp
new file mode 100644
index 0000000..5dae0bc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/load-record-fields/load-record-fields.6.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.
+ */
+
+USE test;
+
+-- Disabled for a simpler plan
+SET `compiler.sort.parallel` "false";
+
+EXPLAIN
+SELECT VALUE md.name
+FROM MyDataset md
+LET currentData = {"myDate": current_date()}
+WHERE currentData.myDate != date("1980-09-10")
+ORDER BY md.id
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-closed/query-issue453-2/query-issue453-2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-closed/query-issue453-2/query-issue453-2.3.query.sqlpp
index 31e68b0..0a81db6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-closed/query-issue453-2/query-issue453-2.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-closed/query-issue453-2/query-issue453-2.3.query.sqlpp
@@ -25,8 +25,7 @@
use test;
-
-select element d
+select d.int_m, d.int_o, d.string_m, d.string_o
from DataOpen as d
-order by d.id
+order by d.int_m, d.int_o, d.string_m, d.string_o;
;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/negative/negative.001.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/negative/negative.001.ddl.sqlpp
new file mode 100644
index 0000000..31934ae
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/negative/negative.001.ddl.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.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE TYPE t1 AS { id: int };
+
+CREATE DATASET ds1(t1) PRIMARY KEY id;
+
+CREATE INDEX ds1_array_idx ON ds1(UNNEST a : string) EXCLUDE UNKNOWN KEY;
+CREATE PRIMARY INDEX pk_idx ON ds1;
+ANALYZE DATASET ds1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/negative/negative.002.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/negative/negative.002.query.sqlpp
new file mode 100644
index 0000000..796f598
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/negative/negative.002.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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;
+// cannot use primary index
+FROM query_index("test", "ds1", "ds1") as v SELECT VALUE v;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/negative/negative.003.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/negative/negative.003.query.sqlpp
new file mode 100644
index 0000000..f5f5774
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/negative/negative.003.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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;
+// cannot use primary key index
+FROM query_index("test", "ds1", "pk_idx") as v SELECT VALUE v;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/negative/negative.004.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/negative/negative.004.query.sqlpp
new file mode 100644
index 0000000..9bea68e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/negative/negative.004.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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;
+// cannot use samples index
+FROM query_index("test", "ds1", "sample_idx_1_ds1") as v SELECT VALUE v;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/negative/negative.005.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/negative/negative.005.query.sqlpp
new file mode 100644
index 0000000..cf3a0e3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/negative/negative.005.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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;
+// cannot use array index
+FROM query_index("test", "ds1", "ds1_array_idx") as v SELECT VALUE v;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/negative/negative.999.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/negative/negative.999.ddl.sqlpp
new file mode 100644
index 0000000..36b2bab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/negative/negative.999.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.001.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.001.ddl.sqlpp
new file mode 100644
index 0000000..270a1d4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.001.ddl.sqlpp
@@ -0,0 +1,45 @@
+/*
+ * 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;
+
+CREATE TYPE t1 AS { id: int };
+CREATE TYPE t2 AS { id: int, age: int };
+CREATE TYPE t3 AS { id: int, age: int?, dept: string? };
+
+CREATE DATASET ds1(t1) PRIMARY KEY id;
+CREATE DATASET ds2(t2) PRIMARY KEY id;
+CREATE DATASET ds3(t3) PRIMARY KEY id;
+
+CREATE INDEX ds1_age ON ds1(age: int);
+CREATE INDEX ds1_dept ON ds1(dept: string);
+CREATE INDEX ds1_age_dept ON ds1(age: int, dept: string);
+CREATE INDEX ds1_dept_age ON ds1(dept: string, age: int);
+
+CREATE INDEX ds2_age ON ds2(age);
+CREATE INDEX ds2_dept ON ds2(dept: string);
+CREATE INDEX ds2_age_dept ON ds2(age, dept: string);
+CREATE INDEX ds2_dept_age ON ds2(dept: string, age);
+
+CREATE INDEX ds3_age ON ds3(age);
+CREATE INDEX ds3_dept ON ds3(dept);
+CREATE INDEX ds3_age_dept ON ds3(age, dept);
+CREATE INDEX ds3_dept_age ON ds3(dept, age);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.002.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.002.update.sqlpp
new file mode 100644
index 0000000..04fd66f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.002.update.sqlpp
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+UPSERT INTO ds1 ([
+{"id":1, "age": 34, "dept": "cs"},
+{"id":2, "age": 20, "dept": "ms"},
+{"id":3, "age": null, "dept": "cs"},
+{"id":4 , "dept": "ms"},
+{"id":5, "age": 34, "dept": null},
+{"id":6, "age": 34 },
+{"id":7, "age": 20, "dept": "cs"},
+{"id":8, "age": 34, "dept": "ms"},
+{"id":9, "age": 34, "dept": "cs"},
+{"id":10, "age": 34, "dept": "ms"},
+{"id":11 },
+{"id":12, "age": null, "dept": null},
+{"id":13, "age": null },
+{"id":14, "dept": null}
+]);
+
+UPSERT INTO ds2 ([
+{"id":1, "age": 34, "dept": "cs"},
+{"id":2, "age": 20, "dept": "ms"},
+{"id":5, "age": 34, "dept": null},
+{"id":6, "age": 34 },
+{"id":7, "age": 20, "dept": "cs"},
+{"id":8, "age": 34, "dept": "ms"},
+{"id":9, "age": 34, "dept": "cs"},
+{"id":10, "age": 34, "dept": "ms"}
+]);
+
+UPSERT INTO ds3 ([
+{"id":1, "age": 34, "dept": "cs"},
+{"id":2, "age": 20, "dept": "ms"},
+{"id":3, "age": null, "dept": "cs"},
+{"id":4 , "dept": "ms"},
+{"id":5, "age": 34, "dept": null},
+{"id":6, "age": 34 },
+{"id":7, "age": 20, "dept": "cs"},
+{"id":8, "age": 34, "dept": "ms"},
+{"id":9, "age": 34, "dept": "cs"},
+{"id":10, "age": 34, "dept": "ms"},
+{"id":11 },
+{"id":12, "age": null, "dept": null},
+{"id":13, "age": null },
+{"id":14, "dept": null}
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.003.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.003.query.sqlpp
new file mode 100644
index 0000000..1910ffc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.003.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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;
+
+FROM query_index("test", "ds1", "ds1_age") as v
+SELECT VALUE v ORDER BY v.PK0;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.004.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.004.query.sqlpp
new file mode 100644
index 0000000..f801201
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.004.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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;
+
+FROM query_index("test", "ds1", "ds1_dept") as v
+SELECT VALUE v ORDER BY v.PK0;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.005.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.005.query.sqlpp
new file mode 100644
index 0000000..729a4f5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.005.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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;
+
+FROM query_index("test", "ds1", "ds1_age_dept") as v
+SELECT VALUE v ORDER BY v.PK0;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.006.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.006.query.sqlpp
new file mode 100644
index 0000000..2914aab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.006.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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;
+
+FROM query_index("test", "ds1", "ds1_dept_age") as v
+SELECT VALUE v ORDER BY v.PK0;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.007.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.007.query.sqlpp
new file mode 100644
index 0000000..2ed609d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.007.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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;
+
+FROM query_index("test", "ds3", "ds3_age") as v
+SELECT VALUE v ORDER BY v.PK0;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.008.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.008.query.sqlpp
new file mode 100644
index 0000000..504cae6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.008.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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;
+
+FROM query_index("test", "ds3", "ds3_dept") as v
+SELECT VALUE v ORDER BY v.PK0;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.009.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.009.query.sqlpp
new file mode 100644
index 0000000..2272bbe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.009.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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;
+
+FROM query_index("test", "ds3", "ds3_age_dept") as v
+SELECT VALUE v ORDER BY v.PK0;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.010.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.010.query.sqlpp
new file mode 100644
index 0000000..9ddc171
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.010.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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;
+
+FROM query_index("test", "ds3", "ds3_dept_age") as v
+SELECT VALUE v ORDER BY v.PK0;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.011.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.011.query.sqlpp
new file mode 100644
index 0000000..41f3505
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.011.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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;
+
+FROM query_index("test", "ds2", "ds2_age") as v
+SELECT VALUE v ORDER BY v.PK0;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.012.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.012.query.sqlpp
new file mode 100644
index 0000000..bce624f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.012.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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;
+
+FROM query_index("test", "ds2", "ds2_dept") as v
+SELECT VALUE v ORDER BY v.PK0;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.013.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.013.query.sqlpp
new file mode 100644
index 0000000..62c5622
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.013.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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;
+
+FROM query_index("test", "ds2", "ds2_age_dept") as v
+SELECT VALUE v ORDER BY v.PK0;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.014.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.014.query.sqlpp
new file mode 100644
index 0000000..f4f0560
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.014.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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;
+
+FROM query_index("test", "ds2", "ds2_dept_age") as v
+SELECT VALUE v ORDER BY v.PK0;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.015.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.015.query.sqlpp
new file mode 100644
index 0000000..31d5fbc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.015.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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;
+
+FROM ds1 as v SELECT COUNT(*) AS cnt;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.016.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.016.query.sqlpp
new file mode 100644
index 0000000..5e7caad
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.016.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;
+FROM query_index("test", "ds1", "ds1_dept") as v SELECT COUNT(*) AS cnt;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.017.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.017.query.sqlpp
new file mode 100644
index 0000000..12e1311
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.017.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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 v.age, COUNT(*) AS cnt FROM ds1 as v GROUP BY v.age ORDER BY v.age;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.018.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.018.query.sqlpp
new file mode 100644
index 0000000..7015ff7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.018.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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 v.SK0 AS age, COUNT(*) AS cnt FROM query_index("test", "ds1", "ds1_age") as v GROUP BY v.SK0 ORDER BY v.SK0;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.019.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.019.query.sqlpp
new file mode 100644
index 0000000..dc94475
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.019.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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 v.SK0 AS age, COUNT(*) AS cnt FROM query_index("test", "ds1", "ds1_age_dept") as v GROUP BY v.SK0 ORDER BY v.SK0;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.020.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.020.query.sqlpp
new file mode 100644
index 0000000..4b2379d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.020.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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 v.SK1 AS age, COUNT(*) AS cnt FROM query_index("test", "ds1", "ds1_dept_age") as v GROUP BY v.SK1 ORDER BY v.SK1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.021.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.021.query.sqlpp
new file mode 100644
index 0000000..ebb2269d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.021.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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 v.age, v.dept, COUNT(*) AS cnt FROM ds1 as v GROUP BY v.age, v.dept ORDER BY v.age, v.dept;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.022.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.022.query.sqlpp
new file mode 100644
index 0000000..b535ee8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.022.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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 v.SK0 AS age, v.SK1 AS dept, COUNT(*) AS cnt FROM query_index("test", "ds1", "ds1_age_dept") as v
+GROUP BY v.SK0, v.SK1
+ORDER BY v.SK0, v.SK1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.023.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.023.query.sqlpp
new file mode 100644
index 0000000..87d3aad
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.023.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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 v.SK0 AS age, v.SK1 AS dept, COUNT(*) AS cnt FROM query_index("test", "ds1", "ds1_age_dept") as v
+GROUP BY v.SK1, v.SK0
+ORDER BY v.SK1, v.SK0;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.024.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.024.query.sqlpp
new file mode 100644
index 0000000..8f6255f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.024.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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;
+SET `compiler.sort.parallel` "false";
+EXPLAIN FROM query_index("test", "ds1", "ds1_age") as v SELECT VALUE v ORDER BY v.PK0;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.025.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.025.query.sqlpp
new file mode 100644
index 0000000..55f2b70
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.025.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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;
+SET `compiler.sort.parallel` "false";
+EXPLAIN FROM query_index("test", "ds1", "ds1_age_dept") as v SELECT VALUE v ORDER BY v.PK0;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.026.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.026.query.sqlpp
new file mode 100644
index 0000000..94b5f60
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.026.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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;
+SET `compiler.sort.parallel` "false";
+EXPLAIN SELECT v.SK0 AS age, v.SK1 AS dept, COUNT(*) AS cnt FROM query_index("test", "ds1", "ds1_age_dept") as v
+ GROUP BY v.SK1, v.SK0
+ ORDER BY v.SK1, v.SK0;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.027.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.027.query.sqlpp
new file mode 100644
index 0000000..7cc70b5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.027.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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;
+
+EXPLAIN FROM query_index("test", "ds1", "ds1_dept") as v SELECT COUNT(*) AS cnt;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.028.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.028.query.sqlpp
new file mode 100644
index 0000000..e6008a6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.028.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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;
+SET `compiler.sort.parallel` "false";
+EXPLAIN SELECT v.SK0 AS age, COUNT(*) AS cnt FROM query_index("test", "ds1", "ds1_age") as v GROUP BY v.SK0 ORDER BY v.SK0;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.029.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.029.query.sqlpp
new file mode 100644
index 0000000..361c57f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.029.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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;
+SET `compiler.sort.parallel` "false";
+EXPLAIN SELECT v.SK0 AS age, COUNT(*) AS cnt FROM query_index("test", "ds1", "ds1_age_dept") as v GROUP BY v.SK0 ORDER BY v.SK0;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.030.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.030.query.sqlpp
new file mode 100644
index 0000000..bcae31e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.030.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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;
+SET `compiler.sort.parallel` "false";
+EXPLAIN SELECT v.SK1 AS age, COUNT(*) AS cnt FROM query_index("test", "ds1", "ds1_dept_age") as v GROUP BY v.SK1 ORDER BY v.SK1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.031.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.031.query.sqlpp
new file mode 100644
index 0000000..4b2dc99
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.031.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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;
+SET `compiler.sort.parallel` "false";
+EXPLAIN SELECT v.SK0 AS age, v.SK1 AS dept, COUNT(*) AS cnt FROM query_index("test", "ds1", "ds1_age_dept") as v
+ GROUP BY v.SK0, v.SK1
+ ORDER BY v.SK0, v.SK1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.999.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.999.ddl.sqlpp
new file mode 100644
index 0000000..36b2bab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.999.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/query-ASTERIXDB-3116/query-ASTERIXDB-3116.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/query-ASTERIXDB-3116/query-ASTERIXDB-3116.1.query.sqlpp
new file mode 100644
index 0000000..1e7c329
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/query-ASTERIXDB-3116/query-ASTERIXDB-3116.1.query.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.
+ */
+
+/*
+ * Description : Test when statement parameter is used in a case expression inside an aggregate function
+ * Expected Res : Success
+ * Date : Feb 2023
+ */
+
+// requesttype=application/json
+
+// param args:json=[1]
+
+select value sum(case when t.y > $1 then t.x else 0 end)
+from [
+ { "x": 10, "y": 1 },
+ { "x": 20, "y": 2 },
+ { "x": 15, "y": 3 }
+] t
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/query-ASTERIXDB-3116/query-ASTERIXDB-3116.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/query-ASTERIXDB-3116/query-ASTERIXDB-3116.2.query.sqlpp
new file mode 100644
index 0000000..93b4281
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/query-ASTERIXDB-3116/query-ASTERIXDB-3116.2.query.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description : Test when statement parameter is used in a case expression inside an aggregate function
+ * Expected Res : Success
+ * Date : Feb 2023
+ */
+
+// requesttype=application/json
+
+// param $y1:json=1
+// param $x1:json=5
+
+select value sum(case when t.y > $y1 then t.x else $x1 end)
+from [
+ { "x": 10, "y": 1 },
+ { "x": 20, "y": 2 },
+ { "x": 15, "y": 3 }
+] t
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/invalid-unicode/test.000.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/invalid-unicode/test.000.query.sqlpp
new file mode 100644
index 0000000..a533d7e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/invalid-unicode/test.000.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.
+ */
+
+// param max-warnings:json=1000
+
+[
+ string_length("\uDEAD x \uDEAD"),
+ string_to_codepoint("\uDEAD x \uDEAD"),
+ trim("\uDEAD x \uDEAD"),
+ ltrim("\uDEAD x \uDEAD"),
+ rtrim("\uDEAD x \uDEAD"),
+ trim("\uDEAD x \uDEAD", "x"),
+ ltrim("\uDEAD x \uDEAD", "x"),
+ rtrim("\uDEAD x \uDEAD", "x"),
+ reverse("\uDEAD x \uDEAD"),
+ position("\uDEAD x \uDEAD", "x"),
+ position1("\uDEAD x \uDEAD", "x")
+];
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.01.ddl.sqlpp
new file mode 100644
index 0000000..e3c9e55
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.01.ddl.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE OR REPLACE FUNCTION `fun0` (data) { data + 1 };
+CREATE OR REPLACE FUNCTION `fun1` (data) { ( SELECT RAW data ) };
+CREATE OR REPLACE FUNCTION `fun2` (data) { ( SELECT RAW (SELECT RAW data) ) };
+CREATE OR REPLACE FUNCTION `fun3` (data) { ( SELECT data ) };
+CREATE OR REPLACE FUNCTION `fun4` (data) { ( SELECT (SELECT data) AS x) };
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.02.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.02.query.sqlpp
new file mode 100644
index 0000000..0a298b2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.02.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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;
+
+FROM [1,2,3] a
+LET test = fun0(a)
+SELECT a, test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.03.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.03.query.sqlpp
new file mode 100644
index 0000000..d4c60c9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.03.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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;
+
+FROM [1,2,3] a
+LET test = fun1(a)
+SELECT a, test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.04.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.04.query.sqlpp
new file mode 100644
index 0000000..7dc96f0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.04.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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;
+
+FROM [1,2,3] a
+LET test = fun2(a)
+SELECT a, test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.05.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.05.query.sqlpp
new file mode 100644
index 0000000..4e57970
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.05.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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;
+
+FROM [1,2,3] a
+LET test = fun3(a)
+SELECT a, test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.06.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.06.query.sqlpp
new file mode 100644
index 0000000..fc0402b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.06.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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;
+
+FROM [1,2,3] a
+LET test = fun4(a)
+SELECT a, test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.07.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.07.query.sqlpp
new file mode 100644
index 0000000..60faeb8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.07.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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;
+
+FROM [1,2,3] a
+SELECT a, fun0(a);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.08.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.08.query.sqlpp
new file mode 100644
index 0000000..6d470c7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.08.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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;
+
+FROM [1,2,3] a
+SELECT a, fun1(a);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.09.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.09.query.sqlpp
new file mode 100644
index 0000000..7c96370
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.09.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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;
+
+FROM [1,2,3] a
+SELECT a, fun2(a);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.10.query.sqlpp
new file mode 100644
index 0000000..975b708
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.10.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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;
+
+FROM [1,2,3] a
+SELECT a, fun3(a);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.11.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.11.query.sqlpp
new file mode 100644
index 0000000..fc50baa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.11.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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;
+
+FROM [1,2,3] a
+SELECT a, fun4(a);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.12.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.12.query.sqlpp
new file mode 100644
index 0000000..0c3dc54
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.12.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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;
+
+FROM [1,2,3] a
+LET test = (SELECT RAW a)
+SELECT a, test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.13.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.13.query.sqlpp
new file mode 100644
index 0000000..54e82fd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.13.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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;
+
+FROM [1,2,3] a
+SELECT a, (SELECT RAW a);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.14.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.14.query.sqlpp
new file mode 100644
index 0000000..4bc8a67
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.14.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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;
+
+FROM [1,2,3] a
+LET test = ( SELECT RAW (SELECT RAW a) )
+SELECT a, test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.15.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.15.query.sqlpp
new file mode 100644
index 0000000..cfa7a65
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.15.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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;
+
+FROM [1,2,3] a
+SELECT a, ( SELECT RAW (SELECT RAW a) );
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.16.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.16.query.sqlpp
new file mode 100644
index 0000000..978b6cb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.16.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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;
+
+FROM [1,2,3] a
+LET test = ( (LET x = 6 SELECT RAW a+x) )
+SELECT a, test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.17.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.17.query.sqlpp
new file mode 100644
index 0000000..392da09
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.17.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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;
+
+FROM [1,2,3] a
+LET test = ( (LET x = random() SELECT RAW a+x) )
+SELECT a, floor(test[0]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.99.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.99.ddl.sqlpp
new file mode 100644
index 0000000..36b2bab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/select_element/select_element.99.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q18/q18.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q18/q18.4.query.sqlpp
new file mode 100644
index 0000000..1f7c7d5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q18/q18.4.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.
+ */
+
+// see ASTERIXDB-3043
+SET `compiler.joinmemory` "160KB";
+
+USE tpcds;
+
+SELECT count (*) AS cnt
+FROM customer c, customer_demographics cd2, customer_address ca
+WHERE
+ c.c_current_cdemo_sk /*+ indexnl */ = cd2.cd_demo_sk
+ AND c.c_current_addr_sk /*+ indexnl */ = ca.ca_address_sk
+ AND c.c_birth_month in [4,5];
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q18/q18.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q18/q18.5.query.sqlpp
new file mode 100644
index 0000000..3a7ae0b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q18/q18.5.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.
+ */
+
+// see ASTERIXDB-3043
+SET `compiler.joinmemory` "160KB";
+
+USE tpcds;
+
+SELECT count (*) AS cnt
+FROM customer c, customer_demographics cd2, customer_address ca
+WHERE
+ c.c_current_cdemo_sk = cd2.cd_demo_sk
+ AND c.c_current_addr_sk = ca.ca_address_sk
+ AND c.c_birth_month in [4,5];
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/inline-in-expr/inline-in-expr.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/inline-in-expr/inline-in-expr.01.ddl.sqlpp
new file mode 100644
index 0000000..d930fc9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/inline-in-expr/inline-in-expr.01.ddl.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description : Test inlining of UDFs
+ */
+
+drop dataverse experiments if exists;
+create dataverse experiments;
+use experiments;
+
+create function fun1(...) {
+ args[0] + args[1]
+};
+
+create function fun2(...) {
+ args[0] - args[1]
+};
+
+CREATE TYPE openType AS {id: int};
+CREATE DATASET ds(openType) primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/inline-in-expr/inline-in-expr.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/inline-in-expr/inline-in-expr.02.update.sqlpp
new file mode 100644
index 0000000..22cfcc1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/inline-in-expr/inline-in-expr.02.update.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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 experiments;
+
+UPSERT INTO ds {"id": 1, "a": 1, "b": 2, "c": [10,20,30,40], "d": [100,200,300,400]};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/inline-in-expr/inline-in-expr.03.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/inline-in-expr/inline-in-expr.03.query.sqlpp
new file mode 100644
index 0000000..5ae85ea
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/inline-in-expr/inline-in-expr.03.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 experiments;
+
+SELECT -fun1(ds.a, ds.b) AS x1,
+ds.c[fun1(ds.a, ds.b)] AS x2,
+ds.d[-fun2(ds.a, ds.b)] AS x3
+FROM ds;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-pushdown/view-pushdown.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-pushdown/view-pushdown.01.ddl.sqlpp
new file mode 100644
index 0000000..a500927
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-pushdown/view-pushdown.01.ddl.sqlpp
@@ -0,0 +1,98 @@
+/*
+ * 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;
+
+// ************** Types **************
+
+CREATE TYPE OpenType AS {
+};
+
+CREATE TYPE TypeWithKnownField AS {
+ id: int,
+ review: string
+};
+
+// ************** Datasets **************
+
+CREATE External Dataset ExternalDataset(OpenType) USING localfs (
+ ("path" = "asterix_nc1://data/json/single-line/20-records.json"),
+ ("format" = "json")
+);
+
+CREATE DATASET DatasetWithKnownField(TypeWithKnownField)
+PRIMARY KEY id;
+
+// ************** Views **************
+
+CREATE OR REPLACE VIEW ExternalView (
+ uid uuid,
+ id int,
+ year int,
+ quarter int,
+ review string
+)
+DEFAULT NULL
+PRIMARY KEY (uid) NOT ENFORCED
+AS
+SELECT
+ uuid() uid,
+ d.id,
+ d.year,
+ d.quarter,
+ d.review
+FROM ExternalDataset d;
+
+
+CREATE OR REPLACE VIEW ViewWithKnownField (
+ id int,
+ year int,
+ quarter int,
+ review string
+)
+DEFAULT NULL
+PRIMARY KEY (id) NOT ENFORCED
+AS
+SELECT
+ d.id,
+ d.year,
+ d.quarter,
+ d.review
+FROM DatasetWithKnownField d;
+
+
+CREATE OR REPLACE VIEW ViewWithKnownFieldAndUuidKey (
+ uid uuid,
+ id int,
+ year int,
+ quarter int,
+ review string
+)
+DEFAULT NULL
+PRIMARY KEY (uid) NOT ENFORCED
+AS
+SELECT
+ uuid() uid,
+ d.id,
+ d.year,
+ d.quarter,
+ d.review
+FROM DatasetWithKnownField d;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-pushdown/view-pushdown.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-pushdown/view-pushdown.02.update.sqlpp
new file mode 100644
index 0000000..29b78b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-pushdown/view-pushdown.02.update.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;
+
+LOAD DATASET DatasetWithKnownField USING localfs (
+ ("path" = "asterix_nc1://data/json/single-line/20-records.json"),
+ ("format" = "json")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-pushdown/view-pushdown.03.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-pushdown/view-pushdown.03.query.sqlpp
new file mode 100644
index 0000000..e87ab52
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-pushdown/view-pushdown.03.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+
+SELECT x.id, x.review
+FROM (
+ SELECT v.id, v.review
+ FROM ExternalView v
+ WHERE v.review = "good"
+ AND year IS NOT UNKNOWN
+ AND quarter IS NOT UNKNOWN
+ LIMIT 3
+) x
+ORDER BY x.id
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-pushdown/view-pushdown.04.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-pushdown/view-pushdown.04.query.sqlpp
new file mode 100644
index 0000000..ae30b48
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-pushdown/view-pushdown.04.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+EXPLAIN
+SELECT x.id, x.review
+FROM (
+ SELECT v.id, v.review
+ FROM ExternalView v
+ WHERE v.review = "good"
+ AND year IS NOT UNKNOWN
+ AND quarter IS NOT UNKNOWN
+ LIMIT 3
+) x
+ORDER BY x.id
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-pushdown/view-pushdown.05.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-pushdown/view-pushdown.05.query.sqlpp
new file mode 100644
index 0000000..e66cd7d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-pushdown/view-pushdown.05.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 v.id, v.review
+FROM ExternalView v
+LIMIT 3
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-pushdown/view-pushdown.06.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-pushdown/view-pushdown.06.query.sqlpp
new file mode 100644
index 0000000..6b272ce
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-pushdown/view-pushdown.06.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;
+
+EXPLAIN
+SELECT v.id, v.review
+FROM ExternalView v
+LIMIT 3
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-pushdown/view-pushdown.07.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-pushdown/view-pushdown.07.query.sqlpp
new file mode 100644
index 0000000..e703f5d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-pushdown/view-pushdown.07.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+
+SELECT x.id, x.review
+FROM (
+ SELECT v.id, v.review
+ FROM ViewWithKnownField v
+ WHERE v.review = "good"
+ AND year IS NOT UNKNOWN
+ AND quarter IS NOT UNKNOWN
+ LIMIT 3
+) x
+ORDER BY x.id
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-pushdown/view-pushdown.08.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-pushdown/view-pushdown.08.query.sqlpp
new file mode 100644
index 0000000..0661cc4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-pushdown/view-pushdown.08.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+EXPLAIN
+SELECT x.id, x.review
+FROM (
+ SELECT v.id, v.review
+ FROM ViewWithKnownField v
+ WHERE v.review = "good"
+ AND year IS NOT UNKNOWN
+ AND quarter IS NOT UNKNOWN
+ LIMIT 3
+) x
+ORDER BY x.id
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-pushdown/view-pushdown.09.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-pushdown/view-pushdown.09.query.sqlpp
new file mode 100644
index 0000000..676227e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-pushdown/view-pushdown.09.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.
+ */
+
+USE test;
+
+
+SELECT v.id, v.review
+FROM ViewWithKnownField v
+ORDER BY v.id
+LIMIT 3
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-pushdown/view-pushdown.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-pushdown/view-pushdown.10.query.sqlpp
new file mode 100644
index 0000000..aec8c67
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-pushdown/view-pushdown.10.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.
+ */
+
+USE test;
+
+EXPLAIN
+SELECT v.id, v.review
+FROM ViewWithKnownField v
+ORDER BY v.id
+LIMIT 3
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-pushdown/view-pushdown.11.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-pushdown/view-pushdown.11.query.sqlpp
new file mode 100644
index 0000000..acf950e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-pushdown/view-pushdown.11.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+
+SELECT x.id, x.review
+FROM (
+ SELECT v.id, v.review
+ FROM ViewWithKnownFieldAndUuidKey v
+ WHERE v.review = "good"
+ AND year IS NOT UNKNOWN
+ AND quarter IS NOT UNKNOWN
+ LIMIT 3
+) x
+ORDER BY x.id
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-pushdown/view-pushdown.12.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-pushdown/view-pushdown.12.query.sqlpp
new file mode 100644
index 0000000..9feb85c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-pushdown/view-pushdown.12.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+EXPLAIN
+SELECT x.id, x.review
+FROM (
+ SELECT v.id, v.review
+ FROM ViewWithKnownFieldAndUuidKey v
+ WHERE v.review = "good"
+ AND year IS NOT UNKNOWN
+ AND quarter IS NOT UNKNOWN
+ LIMIT 3
+) x
+ORDER BY x.id
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-pushdown/view-pushdown.13.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-pushdown/view-pushdown.13.query.sqlpp
new file mode 100644
index 0000000..4ff1f89
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-pushdown/view-pushdown.13.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.
+ */
+
+USE test;
+
+
+SELECT v.id, v.review
+FROM ViewWithKnownFieldAndUuidKey v
+ORDER BY v.id
+LIMIT 3
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-pushdown/view-pushdown.14.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-pushdown/view-pushdown.14.query.sqlpp
new file mode 100644
index 0000000..3d451bc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-pushdown/view-pushdown.14.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.
+ */
+
+USE test;
+
+EXPLAIN
+SELECT v.id, v.review
+FROM ViewWithKnownFieldAndUuidKey v
+ORDER BY v.id
+LIMIT 3
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/broadcast-join-hint-warning/broadcast-join-hint-warning.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/broadcast-join-hint-warning/broadcast-join-hint-warning.1.ddl.sqlpp
new file mode 100644
index 0000000..79ede1c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/broadcast-join-hint-warning/broadcast-join-hint-warning.1.ddl.sqlpp
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description : Test warnings for enhanced broadcast join hint
+* Expected Res : Warning, ignore hint
+* Date : 11/20/2022
+*/
+
+DROP dataverse tpch IF EXISTS;
+CREATE dataverse tpch;
+
+USE tpch;
+
+CREATE TYPE OrderType AS {
+ o_orderkey : integer
+};
+
+CREATE TYPE CustomerType AS {
+ c_custkey : integer
+};
+
+CREATE TYPE NationType AS {
+ n_nationkey : integer
+};
+
+CREATE DATASET orders(OrderType) PRIMARY KEY o_orderkey;
+
+CREATE DATASET customer(CustomerType) PRIMARY KEY c_custkey;
+
+CREATE DATASET nation(NationType) PRIMARY KEY n_nationkey;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/broadcast-join-hint-warning/broadcast-join-hint-warning.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/broadcast-join-hint-warning/broadcast-join-hint-warning.2.query.sqlpp
new file mode 100644
index 0000000..5c3d855
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/broadcast-join-hint-warning/broadcast-join-hint-warning.2.query.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description : Test warnings for enhanced broadcast join hint
+* Expected Res : Warning, ignore hint
+* Date : 11/20/2022
+*/
+// requesttype=application/json
+// param max-warnings:json=10
+
+use tpch;
+
+SELECT COUNT(*)
+FROM orders o,
+ (
+ SELECT *
+ FROM customer c,
+ nation n
+ WHERE c.c_nationkey = n.n_nationkey) cn
+WHERE cn.c_custkey /*+ hash-bcas */ = o.o_custkey;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/broadcast-join-hint-warning/broadcast-join-hint-warning.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/broadcast-join-hint-warning/broadcast-join-hint-warning.3.query.sqlpp
new file mode 100644
index 0000000..2639913
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/broadcast-join-hint-warning/broadcast-join-hint-warning.3.query.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description : Test warnings for enhanced broadcast join hint
+* Expected Res : Warning, ignore hint
+* Date : 11/20/2022
+*/
+// requesttype=application/json
+// param max-warnings:json=10
+
+use tpch;
+
+SELECT COUNT(*)
+FROM orders o,
+ (
+ SELECT *
+ FROM customer c,
+ nation n
+ WHERE c.c_nationkey = n.n_nationkey) cn
+WHERE cn.c_custkey /*+ hash-bcast () */ = o.o_custkey;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/broadcast-join-hint-warning/broadcast-join-hint-warning.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/broadcast-join-hint-warning/broadcast-join-hint-warning.4.query.sqlpp
new file mode 100644
index 0000000..73149a8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/broadcast-join-hint-warning/broadcast-join-hint-warning.4.query.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description : Test warnings for enhanced broadcast join hint
+* Expected Res : Warning, ignore hint
+* Date : 11/20/2022
+*/
+// requesttype=application/json
+// param max-warnings:json=10
+
+use tpch;
+
+SELECT COUNT(*)
+FROM orders o,
+ (
+ SELECT *
+ FROM customer c,
+ nation n
+ WHERE c.c_nationkey = n.n_nationkey) cn
+WHERE cn.c_custkey /*+ hash-bcast (cn) */ = o.o_custkey;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/cardinality-hint-warning/cardinality-hint-warning.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/cardinality-hint-warning/cardinality-hint-warning.01.ddl.sqlpp
new file mode 100644
index 0000000..e4727dc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/cardinality-hint-warning/cardinality-hint-warning.01.ddl.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.
+ */
+/*
+* Description : Test warnings for cardinality hints
+* Expected Res : Warning, ignore hint
+* Date : 03/09/2023
+*/
+
+DROP dataverse tpch IF EXISTS;
+CREATE dataverse tpch;
+
+USE tpch;
+
+
+CREATE TYPE OrderType AS {
+ o_orderkey : integer
+};
+
+CREATE TYPE CustomerType AS {
+ c_custkey : integer
+};
+
+CREATE TYPE NationType AS {
+ n_nationkey : integer,
+ n_name : string
+};
+
+CREATE DATASET orders(OrderType) PRIMARY KEY o_orderkey;
+
+CREATE DATASET customer(CustomerType) PRIMARY KEY c_custkey;
+
+CREATE DATASET nation(NationType) PRIMARY KEY n_nationkey;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/cardinality-hint-warning/cardinality-hint-warning.02.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/cardinality-hint-warning/cardinality-hint-warning.02.query.sqlpp
new file mode 100644
index 0000000..0ed9c00
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/cardinality-hint-warning/cardinality-hint-warning.02.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description : Test warnings for cardinality hints
+* Expected Res : Warning, ignore hint
+* Date : 03/09/2023
+*/
+// requesttype=application/json
+// param max-warnings:json=10
+
+use tpch;
+
+SELECT count(*)
+FROM customer c, nation n
+WHERE c.c_nationkey = n.n_nationkey and n.n_name /*+ selectivity */ = 'UNITED STATES';
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/cardinality-hint-warning/cardinality-hint-warning.03.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/cardinality-hint-warning/cardinality-hint-warning.03.query.sqlpp
new file mode 100644
index 0000000..f3838eb9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/cardinality-hint-warning/cardinality-hint-warning.03.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description : Test warnings for cardinality hints
+* Expected Res : Warning, ignore hint
+* Date : 03/09/2023
+*/
+// requesttype=application/json
+// param max-warnings:json=10
+
+use tpch;
+
+SELECT count(*)
+FROM customer c, nation n
+WHERE c.c_nationkey = n.n_nationkey and n.n_name /*+ selectivity 1 */ = 'UNITED STATES';
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/cardinality-hint-warning/cardinality-hint-warning.04.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/cardinality-hint-warning/cardinality-hint-warning.04.query.sqlpp
new file mode 100644
index 0000000..b40ff68
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/cardinality-hint-warning/cardinality-hint-warning.04.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description : Test warnings for cardinality hints
+* Expected Res : No warning, honor hint
+* Date : 03/09/2023
+*/
+// requesttype=application/json
+// param max-warnings:json=10
+
+use tpch;
+
+SELECT count(*)
+FROM customer c, nation n
+WHERE c.c_nationkey = n.n_nationkey and n.n_name /*+ selectivity 0.1 */ = 'UNITED STATES';
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/cardinality-hint-warning/cardinality-hint-warning.05.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/cardinality-hint-warning/cardinality-hint-warning.05.query.sqlpp
new file mode 100644
index 0000000..ddce731
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/cardinality-hint-warning/cardinality-hint-warning.05.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description : Test warnings for cardinality hints
+* Expected Res : Warning, ignore hint
+* Date : 03/09/2023
+*/
+// requesttype=application/json
+// param max-warnings:json=10
+
+use tpch;
+
+SELECT count(*)
+FROM customer c, nation n
+WHERE c.c_nationkey = n.n_nationkey and n.n_name /*+ selectivity -0.1 */ = 'UNITED STATES';
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/cardinality-hint-warning/cardinality-hint-warning.06.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/cardinality-hint-warning/cardinality-hint-warning.06.query.sqlpp
new file mode 100644
index 0000000..659988d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/cardinality-hint-warning/cardinality-hint-warning.06.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description : Test warnings for cardinality hints
+* Expected Res : Warning, ignore hint
+* Date : 03/09/2023
+*/
+// requesttype=application/json
+// param max-warnings:json=10
+
+use tpch;
+
+SELECT count(*)
+FROM customer c, nation n
+WHERE c.c_nationkey = n.n_nationkey and n.n_name /*+ selectivity 0.0 */ = 'UNITED STATES';
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/cardinality-hint-warning/cardinality-hint-warning.07.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/cardinality-hint-warning/cardinality-hint-warning.07.query.sqlpp
new file mode 100644
index 0000000..8c196df
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/cardinality-hint-warning/cardinality-hint-warning.07.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description : Test warnings for cardinality hints
+* Expected Res : Warning, ignore hint
+* Date : 03/09/2023
+*/
+// requesttype=application/json
+// param max-warnings:json=10
+
+use tpch;
+
+SELECT count(*)
+FROM customer c, nation n
+WHERE c.c_nationkey = n.n_nationkey and n.n_name /*+ selectivity 1.0 */ = 'UNITED STATES';
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/cardinality-hint-warning/cardinality-hint-warning.08.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/cardinality-hint-warning/cardinality-hint-warning.08.query.sqlpp
new file mode 100644
index 0000000..0d98c86
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/cardinality-hint-warning/cardinality-hint-warning.08.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description : Test warnings for cardinality hints
+* Expected Res : Warning, ignore hint
+* Date : 03/09/2023
+*/
+// requesttype=application/json
+// param max-warnings:json=10
+
+use tpch;
+
+SELECT count(*)
+FROM customer c, nation n
+WHERE c.c_nationkey /*+ productivity */ = n.n_nationkey;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/cardinality-hint-warning/cardinality-hint-warning.09.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/cardinality-hint-warning/cardinality-hint-warning.09.query.sqlpp
new file mode 100644
index 0000000..21d4735
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/cardinality-hint-warning/cardinality-hint-warning.09.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description : Test warnings for enhanced hash join hint
+* Expected Res : Warning, ignore hint
+* Date : 03/09/2023
+*/
+// requesttype=application/json
+// param max-warnings:json=10
+
+use tpch;
+
+SELECT count(*)
+FROM customer c, nation n
+WHERE c.c_nationkey /*+ productivity 100.0 */ = n.n_nationkey;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/cardinality-hint-warning/cardinality-hint-warning.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/cardinality-hint-warning/cardinality-hint-warning.10.query.sqlpp
new file mode 100644
index 0000000..626490a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/cardinality-hint-warning/cardinality-hint-warning.10.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description : Test warnings for cardinality hints
+* Expected Res : Warning, ignore hint
+* Date : 03/09/2023
+*/
+// requesttype=application/json
+// param max-warnings:json=10
+
+use tpch;
+
+SELECT count(*)
+FROM customer c, nation n
+WHERE c.c_nationkey /*+ productivity n */ = n.n_nationkey;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/cardinality-hint-warning/cardinality-hint-warning.11.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/cardinality-hint-warning/cardinality-hint-warning.11.query.sqlpp
new file mode 100644
index 0000000..6a6a440
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/cardinality-hint-warning/cardinality-hint-warning.11.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description : Test warnings for cardinality hints
+* Expected Res : Warning, ignore hint
+* Date : 03/09/2023
+*/
+// requesttype=application/json
+// param max-warnings:json=10
+
+use tpch;
+
+SELECT count(*)
+FROM customer c, nation n
+WHERE c.c_nationkey /*+ productivity n 100 */ = n.n_nationkey;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/cardinality-hint-warning/cardinality-hint-warning.12.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/cardinality-hint-warning/cardinality-hint-warning.12.query.sqlpp
new file mode 100644
index 0000000..725b0ea
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/cardinality-hint-warning/cardinality-hint-warning.12.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description : Test warnings for cardinality hints
+* Expected Res : No warning, honor hint
+* Date : 03/09/2023
+*/
+// requesttype=application/json
+// param max-warnings:json=10
+
+use tpch;
+
+SELECT count(*)
+FROM customer c, nation n
+WHERE c.c_nationkey /*+ productivity n 100.0 */ = n.n_nationkey;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/cardinality-hint-warning/cardinality-hint-warning.13.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/cardinality-hint-warning/cardinality-hint-warning.13.query.sqlpp
new file mode 100644
index 0000000..6cc8eff
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/cardinality-hint-warning/cardinality-hint-warning.13.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description : Test warnings for cardinality hints
+* Expected Res : Warning, ignore hint
+* Date : 03/09/2023
+*/
+// requesttype=application/json
+// param max-warnings:json=10
+
+use tpch;
+
+SELECT count(*)
+FROM customer c, nation n
+WHERE c.c_nationkey /*+ productivity n -10.0 */ = n.n_nationkey;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/cardinality-hint-warning/cardinality-hint-warning.14.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/cardinality-hint-warning/cardinality-hint-warning.14.query.sqlpp
new file mode 100644
index 0000000..dd56076
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/cardinality-hint-warning/cardinality-hint-warning.14.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description : Test warnings for cardinality hints
+* Expected Res : Warning, ignore hint
+* Date : 03/09/2023
+*/
+// requesttype=application/json
+// param max-warnings:json=10
+
+use tpch;
+
+SELECT count(*)
+FROM customer c, nation n
+WHERE c.c_nationkey /*+ productivity n 0.0 */ = n.n_nationkey;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/hashjoin-hint-warning/hashjoin-hint-warning.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/hashjoin-hint-warning/hashjoin-hint-warning.01.ddl.sqlpp
new file mode 100644
index 0000000..b64fa28
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/hashjoin-hint-warning/hashjoin-hint-warning.01.ddl.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.
+ */
+/*
+* Description : Test warnings for enhanced hash join hint
+* Expected Res : Warning, ignore hint
+* Date : 11/20/2022
+*/
+
+DROP dataverse tpch IF EXISTS;
+CREATE dataverse tpch;
+
+USE tpch;
+
+
+CREATE TYPE OrderType AS {
+ o_orderkey : integer
+};
+
+CREATE TYPE CustomerType AS {
+ c_custkey : integer
+};
+
+CREATE TYPE NationType AS {
+ n_nationkey : integer,
+ n_name : string
+};
+
+CREATE DATASET orders(OrderType) PRIMARY KEY o_orderkey;
+
+CREATE DATASET customer(CustomerType) PRIMARY KEY c_custkey;
+
+CREATE DATASET nation(NationType) PRIMARY KEY n_nationkey;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/hashjoin-hint-warning/hashjoin-hint-warning.02.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/hashjoin-hint-warning/hashjoin-hint-warning.02.query.sqlpp
new file mode 100644
index 0000000..2134749
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/hashjoin-hint-warning/hashjoin-hint-warning.02.query.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description : Test warnings for enhanced hash join hint
+* Expected Res : Warning, ignore hint
+* Date : 11/20/2022
+*/
+// requesttype=application/json
+// param max-warnings:json=10
+
+use tpch;
+
+SELECT COUNT(*)
+FROM orders o,
+ (
+ SELECT *
+ FROM customer c,
+ nation n
+ WHERE c.c_nationkey = n.n_nationkey) cn
+WHERE cn.c_custkey /*+ hashjoin build */ = o.o_custkey;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/hashjoin-hint-warning/hashjoin-hint-warning.03.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/hashjoin-hint-warning/hashjoin-hint-warning.03.query.sqlpp
new file mode 100644
index 0000000..218c395
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/hashjoin-hint-warning/hashjoin-hint-warning.03.query.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description : Test warnings for enhanced hash join hint
+* Expected Res : Warning, ignore hint
+* Date : 11/20/2022
+*/
+// requesttype=application/json
+// param max-warnings:json=10
+
+use tpch;
+
+SELECT COUNT(*)
+FROM orders o,
+ (
+ SELECT *
+ FROM customer c,
+ nation n
+ WHERE c.c_nationkey = n.n_nationkey) cn
+WHERE cn.c_custkey /*+ hashjon build */ = o.o_custkey;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/hashjoin-hint-warning/hashjoin-hint-warning.04.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/hashjoin-hint-warning/hashjoin-hint-warning.04.query.sqlpp
new file mode 100644
index 0000000..e447779
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/hashjoin-hint-warning/hashjoin-hint-warning.04.query.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description : Test warnings for enhanced hash join hint
+* Expected Res : Warning, ignore hint
+* Date : 11/20/2022
+*/
+// requesttype=application/json
+// param max-warnings:json=10
+
+use tpch;
+
+SELECT COUNT(*)
+FROM orders o,
+ (
+ SELECT *
+ FROM customer c,
+ nation n
+ WHERE c.c_nationkey = n.n_nationkey) cn
+WHERE cn.c_custkey /*+ hashjoin buil */ = o.o_custkey;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/hashjoin-hint-warning/hashjoin-hint-warning.05.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/hashjoin-hint-warning/hashjoin-hint-warning.05.query.sqlpp
new file mode 100644
index 0000000..447f68f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/hashjoin-hint-warning/hashjoin-hint-warning.05.query.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description : Test warnings for enhanced hash join hint
+* Expected Res : Warning, ignore hint
+* Date : 11/20/2022
+*/
+// requesttype=application/json
+// param max-warnings:json=10
+
+use tpch;
+
+SELECT COUNT(*)
+FROM orders o,
+ (
+ SELECT *
+ FROM customer c,
+ nation n
+ WHERE c.c_nationkey = n.n_nationkey) cn
+WHERE cn.c_custkey /*+ hashjoin build () */ = o.o_custkey;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/hashjoin-hint-warning/hashjoin-hint-warning.06.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/hashjoin-hint-warning/hashjoin-hint-warning.06.query.sqlpp
new file mode 100644
index 0000000..6ec7e4a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/hashjoin-hint-warning/hashjoin-hint-warning.06.query.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description : Test warnings for enhanced hash join hint
+* Expected Res : Warning, ignore hint
+* Date : 11/20/2022
+*/
+// requesttype=application/json
+// param max-warnings:json=10
+
+use tpch;
+
+SELECT COUNT(*)
+FROM orders o,
+ (
+ SELECT *
+ FROM customer c,
+ nation n
+ WHERE c.c_nationkey = n.n_nationkey) cn
+WHERE cn.c_custkey /*+ hashjoin build (cn) */ = o.o_custkey;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/hashjoin-hint-warning/hashjoin-hint-warning.07.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/hashjoin-hint-warning/hashjoin-hint-warning.07.query.sqlpp
new file mode 100644
index 0000000..bc5da6d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/hashjoin-hint-warning/hashjoin-hint-warning.07.query.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description : Test warnings for enhanced hash join hint
+* Expected Res : Warning, ignore hint
+* Date : 11/20/2022
+*/
+// requesttype=application/json
+// param max-warnings:json=10
+
+use tpch;
+
+SELECT COUNT(*)
+FROM orders o,
+ (
+ SELECT *
+ FROM customer c,
+ nation n
+ WHERE c.c_nationkey = n.n_nationkey) cn
+WHERE cn.c_custkey /*+ hashjoin probe */ = o.o_custkey;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/hashjoin-hint-warning/hashjoin-hint-warning.08.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/hashjoin-hint-warning/hashjoin-hint-warning.08.query.sqlpp
new file mode 100644
index 0000000..a2aad7d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/hashjoin-hint-warning/hashjoin-hint-warning.08.query.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description : Test warnings for enhanced hash join hint
+* Expected Res : Warning, ignore hint
+* Date : 11/20/2022
+*/
+// requesttype=application/json
+// param max-warnings:json=10
+
+use tpch;
+
+SELECT COUNT(*)
+FROM orders o,
+ (
+ SELECT *
+ FROM customer c,
+ nation n
+ WHERE c.c_nationkey = n.n_nationkey) cn
+WHERE cn.c_custkey /*+ hashjon probe */ = o.o_custkey;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/hashjoin-hint-warning/hashjoin-hint-warning.09.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/hashjoin-hint-warning/hashjoin-hint-warning.09.query.sqlpp
new file mode 100644
index 0000000..e4d7fb0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/hashjoin-hint-warning/hashjoin-hint-warning.09.query.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description : Test warnings for enhanced hash join hint
+* Expected Res : Warning, ignore hint
+* Date : 11/20/2022
+*/
+// requesttype=application/json
+// param max-warnings:json=10
+
+use tpch;
+
+SELECT COUNT(*)
+FROM orders o,
+ (
+ SELECT *
+ FROM customer c,
+ nation n
+ WHERE c.c_nationkey = n.n_nationkey) cn
+WHERE cn.c_custkey /*+ hashjoin prob */ = o.o_custkey;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/hashjoin-hint-warning/hashjoin-hint-warning.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/hashjoin-hint-warning/hashjoin-hint-warning.10.query.sqlpp
new file mode 100644
index 0000000..f69bc62
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/hashjoin-hint-warning/hashjoin-hint-warning.10.query.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description : Test warnings for enhanced hash join hint
+* Expected Res : Warning, ignore hint
+* Date : 11/20/2022
+*/
+// requesttype=application/json
+// param max-warnings:json=10
+
+use tpch;
+
+SELECT COUNT(*)
+FROM orders o,
+ (
+ SELECT *
+ FROM customer c,
+ nation n
+ WHERE c.c_nationkey = n.n_nationkey) cn
+WHERE cn.c_custkey /*+ hashjoin probe () */ = o.o_custkey;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/hashjoin-hint-warning/hashjoin-hint-warning.11.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/hashjoin-hint-warning/hashjoin-hint-warning.11.query.sqlpp
new file mode 100644
index 0000000..6bd515c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/hashjoin-hint-warning/hashjoin-hint-warning.11.query.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description : Test warnings for enhanced hash join hint
+* Expected Res : Warning, ignore hint
+* Date : 11/20/2022
+*/
+// requesttype=application/json
+// param max-warnings:json=10
+
+use tpch;
+
+SELECT COUNT(*)
+FROM orders o,
+ (
+ SELECT *
+ FROM customer c,
+ nation n
+ WHERE c.c_nationkey = n.n_nationkey) cn
+WHERE cn.c_custkey /*+ hashjoin probe (cn) */ = o.o_custkey;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/hashjoin-hint-warning/hashjoin-hint-warning.12.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/hashjoin-hint-warning/hashjoin-hint-warning.12.query.sqlpp
new file mode 100644
index 0000000..f6aebcd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/hashjoin-hint-warning/hashjoin-hint-warning.12.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 : Test warnings for enhanced hash join hint
+* Expected Res : No warning, honor hint
+* Date : 11/20/2022
+*/
+// requesttype=application/json
+// param max-warnings:json=10
+
+use tpch;
+set `compiler.cbo` "true";
+SELECT COUNT(DISTINCT n.n_name) FROM customer c JOIN nation n
+ON c.c_nationkey /*+ hashjoin build(n) */ = n.n_nationkey;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/multiple-hints-warning/multiple-hints-warning.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/multiple-hints-warning/multiple-hints-warning.01.ddl.sqlpp
new file mode 100644
index 0000000..0a8d4b3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/multiple-hints-warning/multiple-hints-warning.01.ddl.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.
+ */
+/*
+* Description : Test warnings for multiple hints
+* Expected Res : Warning, ignore hint
+* Date : 11/20/2022
+*/
+
+DROP dataverse tpch IF EXISTS;
+CREATE dataverse tpch;
+
+USE tpch;
+
+
+CREATE TYPE OrderType AS {
+ o_orderkey : integer
+};
+
+CREATE TYPE CustomerType AS {
+ c_custkey : integer
+};
+
+CREATE TYPE NationType AS {
+ n_nationkey : integer,
+ n_name : string
+};
+
+CREATE DATASET orders(OrderType) PRIMARY KEY o_orderkey;
+
+CREATE DATASET customer(CustomerType) PRIMARY KEY c_custkey;
+
+CREATE DATASET nation(NationType) PRIMARY KEY n_nationkey;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/multiple-hints-warning/multiple-hints-warning.02.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/multiple-hints-warning/multiple-hints-warning.02.query.sqlpp
new file mode 100644
index 0000000..43d647b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/multiple-hints-warning/multiple-hints-warning.02.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 : Test warnings for multiple hints
+* Expected Res : Warning, ignore hint
+* Date : 11/20/2022
+*/
+// requesttype=application/json
+// param max-warnings:json=10
+
+use tpch;
+
+SELECT COUNT(*) FROM customer c JOIN nation n
+ON c.c_nationkey /*+ hashjoin build (n) */ /*+ productivity n 100 */ = n.n_nationkey;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/multiple-hints-warning/multiple-hints-warning.03.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/multiple-hints-warning/multiple-hints-warning.03.query.sqlpp
new file mode 100644
index 0000000..afc8b51
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/multiple-hints-warning/multiple-hints-warning.03.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 : Test warnings for multiple hints
+* Expected Res : Warning, ignore hint
+* Date : 11/20/2022
+*/
+// requesttype=application/json
+// param max-warnings:json=10
+
+use tpch;
+
+SELECT COUNT(*) FROM customer c JOIN nation n
+ON c.c_nationkey /*+ hashjoin build () */ /*+ productivity n 100.0 */ = n.n_nationkey;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/multiple-hints-warning/multiple-hints-warning.04.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/multiple-hints-warning/multiple-hints-warning.04.query.sqlpp
new file mode 100644
index 0000000..e5a74f3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/warnings/multiple-hints-warning/multiple-hints-warning.04.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 : Test warnings for multiple hints
+* Expected Res : Warning, ignore hint
+* Date : 11/20/2022
+*/
+// requesttype=application/json
+// param max-warnings:json=10
+
+use tpch;
+
+SELECT COUNT(*) FROM customer c JOIN nation n
+ON c.c_nationkey /*+ hashjoin build () */ /*+ productivity n 100 */ = n.n_nationkey;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/count_dataset/count_dataset.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/count_dataset/count_dataset.1.adm
deleted file mode 100644
index 9a0762a..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/count_dataset/count_dataset.1.adm
+++ /dev/null
@@ -1,26 +0,0 @@
-distribute result [$$25]
--- DISTRIBUTE_RESULT |UNPARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- aggregate [$$25] <- [agg-sql-sum($$28)]
- -- AGGREGATE |UNPARTITIONED|
- aggregate [$$28] <- [agg-sql-count(1)]
- -- AGGREGATE |PARTITIONED|
- exchange
- -- SORT_MERGE_EXCHANGE [$$26(ASC) ] |PARTITIONED|
- project ([$$26])
- -- STREAM_PROJECT |PARTITIONED|
- select (and(ge($$24, 1), le($$24, 10)))
- -- STREAM_SELECT |PARTITIONED|
- project ([$$26, $$24])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$24] <- [$$Tweet.getField(1)]
- -- ASSIGN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$26, $$Tweet] <- Twitter.Tweet
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/count_dataset/count_dataset.1.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/count_dataset/count_dataset.1.plan
new file mode 100644
index 0000000..58837ea
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/count_dataset/count_dataset.1.plan
@@ -0,0 +1,26 @@
+distribute result [$$25] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ aggregate [$$25] <- [agg-sql-sum($$28)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- AGGREGATE |UNPARTITIONED|
+ aggregate [$$28] <- [agg-sql-count(1)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- AGGREGATE |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SORT_MERGE_EXCHANGE [$$26(ASC) ] |PARTITIONED|
+ project ([$$26]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ select (and(ge($$24, 1), le($$24, 10))) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_SELECT |PARTITIONED|
+ project ([$$26, $$24]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$24] <- [$$Tweet.getField(1)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$26, $$Tweet] <- Twitter.Tweet [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/median/median.03.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/median/median.03.adm
new file mode 100644
index 0000000..6d291ab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/median/median.03.adm
@@ -0,0 +1 @@
+{ "med": 3.0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/median/median.04.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/median/median.04.adm
new file mode 100644
index 0000000..9506c5b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/median/median.04.adm
@@ -0,0 +1 @@
+{ "med": 2.5 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/median/median.05.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/median/median.05.adm
new file mode 100644
index 0000000..6d291ab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/median/median.05.adm
@@ -0,0 +1 @@
+{ "med": 3.0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/median/median.06.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/median/median.06.adm
new file mode 100644
index 0000000..9506c5b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/median/median.06.adm
@@ -0,0 +1 @@
+{ "med": 2.5 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/median/median.07.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/median/median.07.adm
new file mode 100644
index 0000000..dd133f3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/median/median.07.adm
@@ -0,0 +1 @@
+{ "med": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/median/median.08.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/median/median.08.adm
new file mode 100644
index 0000000..a4a8fa5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/median/median.08.adm
@@ -0,0 +1 @@
+{ "med": 9.0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/median/median.09.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/median/median.09.adm
new file mode 100644
index 0000000..8f23892
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/median/median.09.adm
@@ -0,0 +1 @@
+{ "med": 50.1 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/median/median.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/median/median.10.adm
new file mode 100644
index 0000000..d5f5227
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/median/median.10.adm
@@ -0,0 +1,15 @@
+{ "med": 50.0, "g": 0 }
+{ "med": 49.8, "g": 1 }
+{ "med": 50.0, "g": 2 }
+{ "med": 50.2, "g": 3 }
+{ "med": 50.3, "g": 4 }
+{ "med": 50.0, "g": 5 }
+{ "med": 50.1, "g": 6 }
+{ "med": 50.1, "g": 7 }
+{ "med": 50.2, "g": 8 }
+{ "med": 50.1, "g": 9 }
+{ "med": 50.3, "g": 10 }
+{ "med": 49.8, "g": 11 }
+{ "med": 49.9, "g": 12 }
+{ "med": 49.9, "g": 13 }
+{ "med": 50.2, "g": 14 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/median/median.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/median/median.11.adm
new file mode 100644
index 0000000..8f23892
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/median/median.11.adm
@@ -0,0 +1 @@
+{ "med": 50.1 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/count_dataset/count_dataset.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/count_dataset/count_dataset.1.adm
deleted file mode 100644
index f56f3a1..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/count_dataset/count_dataset.1.adm
+++ /dev/null
@@ -1,26 +0,0 @@
-distribute result [$$25]
--- DISTRIBUTE_RESULT |UNPARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- aggregate [$$25] <- [agg-sum($$28)]
- -- AGGREGATE |UNPARTITIONED|
- aggregate [$$28] <- [agg-count(1)]
- -- AGGREGATE |PARTITIONED|
- exchange
- -- SORT_MERGE_EXCHANGE [$$26(ASC) ] |PARTITIONED|
- project ([$$26])
- -- STREAM_PROJECT |PARTITIONED|
- select (and(ge($$24, 1), le($$24, 10)))
- -- STREAM_SELECT |PARTITIONED|
- project ([$$26, $$24])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$24] <- [$$Tweet.getField(1)]
- -- ASSIGN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$26, $$Tweet] <- Twitter.Tweet
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/count_dataset/count_dataset.1.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/count_dataset/count_dataset.1.plan
new file mode 100644
index 0000000..584763d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/count_dataset/count_dataset.1.plan
@@ -0,0 +1,26 @@
+distribute result [$$25] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ aggregate [$$25] <- [agg-sum($$28)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- AGGREGATE |UNPARTITIONED|
+ aggregate [$$28] <- [agg-count(1)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- AGGREGATE |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SORT_MERGE_EXCHANGE [$$26(ASC) ] |PARTITIONED|
+ project ([$$26]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ select (and(ge($$24, 1), le($$24, 10))) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_SELECT |PARTITIONED|
+ project ([$$26, $$24]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$24] <- [$$Tweet.getField(1)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$26, $$Tweet] <- Twitter.Tweet [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm
index c55c0bc..cb10dd2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm
@@ -10,7 +10,9 @@
"active\.suspend\.timeout" : 3600,
"azure.request.timeout" : 120,
"compiler\.arrayindex" : true,
+ "compiler.cbo" : false,
"compiler\.external\.field\.pushdown" : true,
+ "compiler.forcejoinorder" : false,
"compiler\.framesize" : 32768,
"compiler\.groupmemory" : 163840,
"compiler\.indexonly" : true,
@@ -18,6 +20,7 @@
"compiler\.joinmemory" : 262144,
"compiler\.min\.memory\.allocation" : true,
"compiler\.parallelism" : 0,
+ "compiler.queryplanshape" : "zigzag",
"compiler\.sort\.parallel" : false,
"compiler\.sort\.samples" : 100,
"compiler\.sortmemory" : 327680,
@@ -45,6 +48,7 @@
"replication\.strategy" : "none",
"replication\.timeout" : 120,
"ssl\.enabled" : false,
+ "storage.buffercache.pagesize" : 32768,
"storage.compression.block" : "snappy",
"storage.global.cleanup.timeout" : 600,
"storage.lsm.bloomfilter.falsepositiverate" : 0.01,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm
index 661daf3..59d595b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm
@@ -10,7 +10,9 @@
"active\.suspend\.timeout" : 3600,
"azure.request.timeout" : 120,
"compiler\.arrayindex" : true,
+ "compiler.cbo" : false,
"compiler\.external\.field\.pushdown" : true,
+ "compiler.forcejoinorder" : false,
"compiler\.framesize" : 32768,
"compiler\.groupmemory" : 163840,
"compiler\.indexonly" : true,
@@ -18,6 +20,7 @@
"compiler\.joinmemory" : 262144,
"compiler\.min\.memory\.allocation" : true,
"compiler\.parallelism" : -1,
+ "compiler.queryplanshape" : "zigzag",
"compiler\.sort\.parallel" : true,
"compiler\.sort\.samples" : 100,
"compiler\.sortmemory" : 327680,
@@ -45,6 +48,7 @@
"replication\.strategy" : "none",
"replication\.timeout" : 120,
"ssl\.enabled" : false,
+ "storage.buffercache.pagesize" : 32768,
"storage.compression.block" : "snappy",
"storage.global.cleanup.timeout" : 600,
"storage.lsm.bloomfilter.falsepositiverate" : 0.01,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm
index 1f0e865..56b413d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm
@@ -10,7 +10,9 @@
"active\.suspend\.timeout" : 3600,
"azure.request.timeout" : 120,
"compiler\.arrayindex" : true,
+ "compiler.cbo" : false,
"compiler\.external\.field\.pushdown" : true,
+ "compiler.forcejoinorder" : false,
"compiler\.framesize" : 32768,
"compiler\.groupmemory" : 163840,
"compiler\.indexonly" : true,
@@ -18,6 +20,7 @@
"compiler\.joinmemory" : 262144,
"compiler\.min\.memory\.allocation" : true,
"compiler\.parallelism" : 3,
+ "compiler.queryplanshape" : "zigzag",
"compiler\.sort\.parallel" : true,
"compiler\.sort\.samples" : 100,
"compiler\.sortmemory" : 327680,
@@ -45,6 +48,7 @@
"replication\.strategy" : "none",
"replication\.timeout" : 120,
"ssl\.enabled" : false,
+ "storage.buffercache.pagesize" : 32768,
"storage.compression.block" : "snappy",
"storage.global.cleanup.timeout" : 600,
"storage.lsm.bloomfilter.falsepositiverate" : 0.01,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly-2/compileonly-2.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly-2/compileonly-2.10.adm
new file mode 100644
index 0000000..bacb60c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly-2/compileonly-2.10.adm
@@ -0,0 +1 @@
+{ "cnt": 0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly-2/compileonly-2.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly-2/compileonly-2.2.adm
new file mode 100644
index 0000000..c1ff5e5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly-2/compileonly-2.2.adm
@@ -0,0 +1,6 @@
+{ "k": "Dataset", "n": "ds1" }
+{ "k": "Dataset", "n": "v1" }
+{ "k": "Datatype", "n": "t1" }
+{ "k": "Dataverse", "n": "test1" }
+{ "k": "Function", "n": "f1" }
+{ "k": "Synonym", "n": "s1" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly-2/compileonly-2.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly-2/compileonly-2.4.adm
new file mode 100644
index 0000000..c1ff5e5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly-2/compileonly-2.4.adm
@@ -0,0 +1,6 @@
+{ "k": "Dataset", "n": "ds1" }
+{ "k": "Dataset", "n": "v1" }
+{ "k": "Datatype", "n": "t1" }
+{ "k": "Dataverse", "n": "test1" }
+{ "k": "Function", "n": "f1" }
+{ "k": "Synonym", "n": "s1" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly-2/compileonly-2.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly-2/compileonly-2.6.adm
new file mode 100644
index 0000000..c1ff5e5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly-2/compileonly-2.6.adm
@@ -0,0 +1,6 @@
+{ "k": "Dataset", "n": "ds1" }
+{ "k": "Dataset", "n": "v1" }
+{ "k": "Datatype", "n": "t1" }
+{ "k": "Dataverse", "n": "test1" }
+{ "k": "Function", "n": "f1" }
+{ "k": "Synonym", "n": "s1" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly-2/compileonly-2.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly-2/compileonly-2.8.adm
new file mode 100644
index 0000000..c1ff5e5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly-2/compileonly-2.8.adm
@@ -0,0 +1,6 @@
+{ "k": "Dataset", "n": "ds1" }
+{ "k": "Dataset", "n": "v1" }
+{ "k": "Datatype", "n": "t1" }
+{ "k": "Dataverse", "n": "test1" }
+{ "k": "Function", "n": "f1" }
+{ "k": "Synonym", "n": "s1" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.1.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.1.regexjson
deleted file mode 100644
index 40a764c..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.1.regexjson
+++ /dev/null
@@ -1,9 +0,0 @@
-{
- "logicalPlan": {
- "operator":"distribute-result",
- "expressions":"R{.*}",
- "operatorId":"R{.*}",
- "execution-mode":"R{.*}",
- "inputs":"R{.*}"
- }
-}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.2.adm
deleted file mode 100644
index 1ced420..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.2.adm
+++ /dev/null
@@ -1 +0,0 @@
-{"statementCategory":"query","statementParameters":[1,2]}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.2.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.2.regexjson
new file mode 100644
index 0000000..95d29da
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.2.regexjson
@@ -0,0 +1,10 @@
+{
+ "logicalPlan": {
+ "operator":"distribute-result",
+ "expressions":"R{.*}",
+ "operatorId":"R{.*}",
+ "execution-mode":"R{.*}",
+ "optimizer-estimates":"R{.*}",
+ "inputs":"R{.*}"
+ }
+}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.3.adm
index 83b47f4..1ced420 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.3.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.3.adm
@@ -1 +1 @@
-{"statementCategory":"query","statementParameters":[1,2],"explainOnly":true}
\ No newline at end of file
+{"statementCategory":"query","statementParameters":[1,2]}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.4.adm
index 63c482a..83b47f4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.4.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.4.adm
@@ -1 +1 @@
-{"statementCategory":"update","statementParameters":[1,2]}
\ No newline at end of file
+{"statementCategory":"query","statementParameters":[1,2],"explainOnly":true}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.5.adm
new file mode 100644
index 0000000..63c482a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.5.adm
@@ -0,0 +1 @@
+{"statementCategory":"update","statementParameters":[1,2]}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/index-bad-fields-bulkload/index-bad-fields.006.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/index-bad-fields-bulkload/index-bad-fields.006.adm
new file mode 100644
index 0000000..917cc17
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/index-bad-fields-bulkload/index-bad-fields.006.adm
@@ -0,0 +1 @@
+{ "values": [ 10, 4 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/index-bad-fields/index-bad-fields.002.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/index-bad-fields/index-bad-fields.002.adm
new file mode 100644
index 0000000..746d306
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/index-bad-fields/index-bad-fields.002.adm
@@ -0,0 +1 @@
+{ "$1": 15 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/index-bad-fields/index-bad-fields.003.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/index-bad-fields/index-bad-fields.003.adm
new file mode 100644
index 0000000..a464380
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/index-bad-fields/index-bad-fields.003.adm
@@ -0,0 +1,10 @@
+{ "values": [ "hello", 1 ] }
+{ "values": [ "hello", 5 ] }
+{ "values": [ "hello", 6 ] }
+{ "values": [ "hello", 8 ] }
+{ "values": [ "hello", 10 ] }
+{ "values": [ "hello", 11 ] }
+{ "values": [ "hello", 12 ] }
+{ "values": [ "hello", 13 ] }
+{ "values": [ "hello", 14 ] }
+{ "values": [ "hello", 15 ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/index-bad-fields/index-bad-fields.004.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/index-bad-fields/index-bad-fields.004.adm
new file mode 100644
index 0000000..3fdbd81
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/index-bad-fields/index-bad-fields.004.adm
@@ -0,0 +1,2 @@
+{ "values": [ 1, 5 ] }
+{ "values": [ 10, 7 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/index-bad-fields/index-bad-fields.005.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/index-bad-fields/index-bad-fields.005.adm
new file mode 100644
index 0000000..6045d06
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/index-bad-fields/index-bad-fields.005.adm
@@ -0,0 +1 @@
+{ "values": [ 1, 10 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/index-bad-fields/index-bad-fields.006.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/index-bad-fields/index-bad-fields.006.adm
new file mode 100644
index 0000000..4c44dba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/index-bad-fields/index-bad-fields.006.adm
@@ -0,0 +1,2 @@
+{ "values": [ "aab", "10", 90, 11 ] }
+{ "values": [ "aab", "10", 93, 14 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/index-bad-fields/index-bad-fields.007.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/index-bad-fields/index-bad-fields.007.adm
new file mode 100644
index 0000000..a38bbc6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/index-bad-fields/index-bad-fields.007.adm
@@ -0,0 +1 @@
+{ "values": [ 100, "aab", "10", 100.1, 14 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/index-bad-fields/index-bad-fields.009.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/index-bad-fields/index-bad-fields.009.adm
new file mode 100644
index 0000000..a464380
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/index-bad-fields/index-bad-fields.009.adm
@@ -0,0 +1,10 @@
+{ "values": [ "hello", 1 ] }
+{ "values": [ "hello", 5 ] }
+{ "values": [ "hello", 6 ] }
+{ "values": [ "hello", 8 ] }
+{ "values": [ "hello", 10 ] }
+{ "values": [ "hello", 11 ] }
+{ "values": [ "hello", 12 ] }
+{ "values": [ "hello", 13 ] }
+{ "values": [ "hello", 14 ] }
+{ "values": [ "hello", 15 ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/index-bad-fields/index-bad-fields.010.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/index-bad-fields/index-bad-fields.010.adm
new file mode 100644
index 0000000..3fdbd81
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/index-bad-fields/index-bad-fields.010.adm
@@ -0,0 +1,2 @@
+{ "values": [ 1, 5 ] }
+{ "values": [ 10, 7 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/index-bad-fields/index-bad-fields.011.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/index-bad-fields/index-bad-fields.011.adm
new file mode 100644
index 0000000..6045d06
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/index-bad-fields/index-bad-fields.011.adm
@@ -0,0 +1 @@
+{ "values": [ 1, 10 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/index-bad-fields/index-bad-fields.012.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/index-bad-fields/index-bad-fields.012.adm
new file mode 100644
index 0000000..4c44dba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/index-bad-fields/index-bad-fields.012.adm
@@ -0,0 +1,2 @@
+{ "values": [ "aab", "10", 90, 11 ] }
+{ "values": [ "aab", "10", 93, 14 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/index-bad-fields/index-bad-fields.013.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/index-bad-fields/index-bad-fields.013.adm
new file mode 100644
index 0000000..a38bbc6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/index-bad-fields/index-bad-fields.013.adm
@@ -0,0 +1 @@
+{ "values": [ 100, "aab", "10", 100.1, 14 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/multiple-indexes/multiple-indexes.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/multiple-indexes/multiple-indexes.3.adm
new file mode 100644
index 0000000..9545720
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/multiple-indexes/multiple-indexes.3.adm
@@ -0,0 +1,4 @@
+{ "id": 1 }
+{ "id": 2 }
+{ "id": 3 }
+{ "id": 5 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/multiple-indexes/multiple-indexes.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/multiple-indexes/multiple-indexes.4.adm
new file mode 100644
index 0000000..9545720
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/multiple-indexes/multiple-indexes.4.adm
@@ -0,0 +1,4 @@
+{ "id": 1 }
+{ "id": 2 }
+{ "id": 3 }
+{ "id": 5 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/multiple-indexes/multiple-indexes.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/multiple-indexes/multiple-indexes.3.adm
new file mode 100644
index 0000000..9545720
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/multiple-indexes/multiple-indexes.3.adm
@@ -0,0 +1,4 @@
+{ "id": 1 }
+{ "id": 2 }
+{ "id": 3 }
+{ "id": 5 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/multiple-indexes/multiple-indexes.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/multiple-indexes/multiple-indexes.4.adm
new file mode 100644
index 0000000..9545720
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/multiple-indexes/multiple-indexes.4.adm
@@ -0,0 +1,4 @@
+{ "id": 1 }
+{ "id": 2 }
+{ "id": 3 }
+{ "id": 5 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_remove/array_remove.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_remove/array_remove.5.adm
deleted file mode 100644
index 3c38795..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_remove/array_remove.5.adm
+++ /dev/null
@@ -1,8 +0,0 @@
-distribute result [$$d]
--- DISTRIBUTE_RESULT |UNPARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- unnest $$d <- scan-collection(ordered-list-constructor({"id": 1, "t1": array-remove(ordered-list-constructor(1, 2, 3, ordered-list-constructor(9, 8), ordered-list-constructor("str1", "str2"), ordered-list-constructor(90, 100)), array: [ 9, 8 ], array: [ 90, 100 ])}, {"id": 2, "t2": cast(array: [ array: [ 5, 1, 2 ], array: [ 90, 100 ] ])}, {"id": 3, "t3": cast(array-remove(ordered-list-constructor({"id": 1, "age": 34}, {"id": 2, "age": 29}, {"id": 3, "age": 90}), {"id": 4, "age": 90}, {"id": 2, "age": 29}))}))
- -- UNNEST |UNPARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_remove/array_remove.5.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_remove/array_remove.5.plan
new file mode 100644
index 0000000..0100d55
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_remove/array_remove.5.plan
@@ -0,0 +1,8 @@
+distribute result [$$d] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ unnest $$d <- scan-collection(ordered-list-constructor({"id": 1, "t1": array-remove(ordered-list-constructor(1, 2, 3, ordered-list-constructor(9, 8), ordered-list-constructor("str1", "str2"), ordered-list-constructor(90, 100)), array: [ 9, 8 ], array: [ 90, 100 ])}, {"id": 2, "t2": cast(array: [ array: [ 5, 1, 2 ], array: [ 90, 100 ] ])}, {"id": 3, "t3": cast(array-remove(ordered-list-constructor({"id": 1, "age": 34}, {"id": 2, "age": 29}, {"id": 3, "age": 90}), {"id": 4, "age": 90}, {"id": 2, "age": 29}))})) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- UNNEST |UNPARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/arrays/arrays.020.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/arrays/arrays.020.adm
index 7b82038..dc424d3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/arrays/arrays.020.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/arrays/arrays.020.adm
@@ -1 +1 @@
-{ "t1": { "c": "[0,1] = [double('0'), float('1')]", "r": true }, "t2": { "c": "[-0, -1] = [float('-0'), -1]", "r": false }, "t3": { "c": "[double('INF')] > [0]", "r": true }, "t4": { "c": "[double('-INF')] < [0]", "r": true }, "t5": { "c": "[double('INF')] > [-0]", "r": true }, "t6": { "c": "[double('-INF')] < [-0]", "r": true }, "t7": { "c": "[double('INF')] > [double('-0')]", "r": true }, "t8": { "c": "[double('-INF')] < [double('-0')]", "r": true }, "t9": { "c": "[double('NaN')] > [0]", "r": true }, "t10": { "c": "[double('NaN')] < [0]", "r": false }, "t11": { "c": "[double('NaN')] > [-0]", "r": true }, "t12": { "c": "[double('NaN')] < [-0]", "r": false }, "t13": { "c": "[double('NaN')] > [double('-0')]", "r": true }, "t14": { "c": "[double('NaN')] < [double('-0')]", "r": false }, "t15": { "c": "[double('-INF')] < [double('INF')]", "r": true }, "t16": { "c": "[double('INF')] > [double('NaN')]", "r": false }, "t17": { "c": "[double('-INF')] < [double('NaN')]", "r": true } }
\ No newline at end of file
+{ "t1": { "c": "[0,1] = [double('0'), float('1')]", "r": true }, "t2": { "c": "[-0, -1] = [float('-0'), -1]", "r": true }, "t3": { "c": "[double('INF')] > [0]", "r": true }, "t4": { "c": "[double('-INF')] < [0]", "r": true }, "t5": { "c": "[double('INF')] > [-0]", "r": true }, "t6": { "c": "[double('-INF')] < [-0]", "r": true }, "t7": { "c": "[double('INF')] > [double('-0')]", "r": true }, "t8": { "c": "[double('-INF')] < [double('-0')]", "r": true }, "t9": { "c": "[double('NaN')] > [0]", "r": true }, "t10": { "c": "[double('NaN')] < [0]", "r": false }, "t11": { "c": "[double('NaN')] > [-0]", "r": true }, "t12": { "c": "[double('NaN')] < [-0]", "r": false }, "t13": { "c": "[double('NaN')] > [double('-0')]", "r": true }, "t14": { "c": "[double('NaN')] < [double('-0')]", "r": false }, "t15": { "c": "[double('-INF')] < [double('INF')]", "r": true }, "t16": { "c": "[double('INF')] > [double('NaN')]", "r": false }, "t17": { "c": "[double('-INF')] < [double('NaN')]", "r": true } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.11.adm
new file mode 100644
index 0000000..58f454b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.11.adm
@@ -0,0 +1 @@
+{ "metadata": { "DatasetName": "ds1", "IndexName": "sample_idx_2_ds1", "SampleCardinalityTarget": 4252, "SourceCardinality": 1100, "SourceAvgItemSize": true, "SampleSeed": 234 }, "stats": { "cnt": 1100, "min_pk": 1, "max_pk": 1100, "min_x": -1100, "max_x": -1 } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.14.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.14.adm
new file mode 100644
index 0000000..6ef756a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.14.adm
@@ -0,0 +1 @@
+{ "metadata": { "DatasetName": "ds1", "IndexName": "sample_idx_1_ds1", "SampleCardinalityTarget": 4252, "SourceCardinality": 4400, "SourceAvgItemSize": true, "SampleSeed": true }, "stats": { "cnt": 4246, "min_pk": true, "max_pk": true, "min_x": true, "max_x": true } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.16.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.16.adm
new file mode 100644
index 0000000..01eb5b7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.16.adm
@@ -0,0 +1 @@
+{ "metadata": { "DatasetName": "ds1", "IndexName": "sample_idx_2_ds1", "SampleCardinalityTarget": 17008, "SourceCardinality": 4400, "SourceAvgItemSize": true, "SampleSeed": 345 }, "stats": { "cnt": 4400, "min_pk": 1, "max_pk": 4400, "min_x": -4400, "max_x": -1 } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.19.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.19.adm
new file mode 100644
index 0000000..60b969f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.19.adm
@@ -0,0 +1 @@
+{ "metadata": { "DatasetName": "ds1", "IndexName": "sample_idx_1_ds1", "SampleCardinalityTarget": 17008, "SourceCardinality": 17100, "SourceAvgItemSize": true, "SampleSeed": true }, "stats": { "cnt": 16972, "min_pk": true, "max_pk": true, "min_x": true, "max_x": true } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.2.adm
new file mode 100644
index 0000000..e3cefee
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.2.adm
@@ -0,0 +1 @@
+{ "DatasetName": "ds1", "IndexName": "sample_idx_1_ds1", "SampleCardinalityTarget": 1063, "SourceCardinality": 0, "SourceAvgItemSize": 0, "SampleSeed": true }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.21.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.21.adm
new file mode 100644
index 0000000..49f9285
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.21.adm
@@ -0,0 +1 @@
+{ "metadata": { "DatasetName": "ds1", "IndexName": "sample_idx_2_ds1", "SampleCardinalityTarget": 17008, "SourceCardinality": 17100, "SourceAvgItemSize": true, "SampleSeed": true }, "stats": { "cnt": 16972, "min_pk": true, "max_pk": true, "min_x": true, "max_x": true } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.23.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.23.adm
new file mode 100644
index 0000000..bacb60c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.23.adm
@@ -0,0 +1 @@
+{ "cnt": 0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.25.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.25.adm
new file mode 100644
index 0000000..f86e66b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.25.adm
@@ -0,0 +1 @@
+{ "cnt": 1 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.27.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.27.adm
new file mode 100644
index 0000000..bacb60c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.27.adm
@@ -0,0 +1 @@
+{ "cnt": 0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.5.adm
new file mode 100644
index 0000000..605bb12
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.5.adm
@@ -0,0 +1 @@
+{ "metadata": { "DatasetName": "ds1", "IndexName": "sample_idx_2_ds1", "SampleCardinalityTarget": 1063, "SourceCardinality": 8, "SourceAvgItemSize": true, "SampleSeed": 123 }, "stats": { "cnt": 8, "min_pk": 1, "max_pk": 8, "min_x": -8, "max_x": -1 } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.7.adm
new file mode 100644
index 0000000..605bb12
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.7.adm
@@ -0,0 +1 @@
+{ "metadata": { "DatasetName": "ds1", "IndexName": "sample_idx_2_ds1", "SampleCardinalityTarget": 1063, "SourceCardinality": 8, "SourceAvgItemSize": true, "SampleSeed": 123 }, "stats": { "cnt": 8, "min_pk": 1, "max_pk": 8, "min_x": -8, "max_x": -1 } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.9.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.9.adm
new file mode 100644
index 0000000..0084d2b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.9.adm
@@ -0,0 +1 @@
+{ "metadata": { "DatasetName": "ds1", "IndexName": "sample_idx_1_ds1", "SampleCardinalityTarget": 1063, "SourceCardinality": 1100, "SourceAvgItemSize": true, "SampleSeed": true }, "stats": { "cnt": 1033, "min_pk": true, "max_pk": true, "min_x": true, "max_x": true } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-with-indexes/analyze-dataset-with-indexes.04.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-with-indexes/analyze-dataset-with-indexes.04.adm
new file mode 100644
index 0000000..aa355f4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-with-indexes/analyze-dataset-with-indexes.04.adm
@@ -0,0 +1,3 @@
+{ "DatasetName": "ds1", "IndexName": "sample_idx_1_ds1", "IndexStructure": "SAMPLE", "SampleCardinalityTarget": 1063, "SourceCardinality": 1000, "SourceAvgItemSize": 369, "IndexStats": [ { "IndexName": "ds1_idx3", "NumPages": 8 }, { "IndexName": "ds1_idx2", "NumPages": 8 }, { "IndexName": "ds1_idx1", "NumPages": 8 } ] }
+{ "DatasetName": "ds2", "IndexName": "sample_idx_1_ds2", "IndexStructure": "SAMPLE", "SampleCardinalityTarget": 1063, "SourceCardinality": 1000, "SourceAvgItemSize": 369, "IndexStats": [ { "IndexName": "ds2_idx1", "NumPages": 8 } ] }
+{ "DatasetName": "ds3", "IndexName": "sample_idx_1_ds3", "IndexStructure": "SAMPLE", "SampleCardinalityTarget": 1063, "SourceCardinality": 1000, "SourceAvgItemSize": 369 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-with-indexes/analyze-dataset-with-indexes.06.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-with-indexes/analyze-dataset-with-indexes.06.adm
new file mode 100644
index 0000000..e193cb2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-with-indexes/analyze-dataset-with-indexes.06.adm
@@ -0,0 +1,3 @@
+{ "DatasetName": "ds1", "IndexName": "sample_idx_2_ds1", "IndexStructure": "SAMPLE", "SampleCardinalityTarget": 1063, "SourceCardinality": 1000, "SourceAvgItemSize": 369, "IndexStats": [ { "IndexName": "ds1_idx2", "NumPages": 8 }, { "IndexName": "ds1_idx1", "NumPages": 8 } ] }
+{ "DatasetName": "ds2", "IndexName": "sample_idx_1_ds2", "IndexStructure": "SAMPLE", "SampleCardinalityTarget": 1063, "SourceCardinality": 1000, "SourceAvgItemSize": 369, "IndexStats": [ { "IndexName": "ds2_idx1", "NumPages": 8 } ] }
+{ "DatasetName": "ds3", "IndexName": "sample_idx_1_ds3", "IndexStructure": "SAMPLE", "SampleCardinalityTarget": 1063, "SourceCardinality": 1000, "SourceAvgItemSize": 369 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access/explain_field_access.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access/explain_field_access.1.adm
deleted file mode 100644
index 31ef021..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access/explain_field_access.1.adm
+++ /dev/null
@@ -1,42 +0,0 @@
-distribute result [$$47]
--- DISTRIBUTE_RESULT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$47])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$47] <- [{"deptId": $#1, "star_cost": $$50}]
- -- ASSIGN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- group by ([$#1 := $$55]) decor ([]) {
- aggregate [$$50] <- [agg-global-sql-sum($$54)]
- -- AGGREGATE |LOCAL|
- nested tuple source
- -- NESTED_TUPLE_SOURCE |LOCAL|
- }
- -- SORT_GROUP_BY[$$55] |PARTITIONED|
- exchange
- -- HASH_PARTITION_EXCHANGE [$$55] |PARTITIONED|
- group by ([$$55 := $$48]) decor ([]) {
- aggregate [$$54] <- [agg-local-sql-sum($$45)]
- -- AGGREGATE |LOCAL|
- nested tuple source
- -- NESTED_TUPLE_SOURCE |LOCAL|
- }
- -- SORT_GROUP_BY[$$48] |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$45, $$48])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$48, $$45] <- [substring($$e.getField("dept").getField("department_id"), 0), $$e.getField("salary")]
- -- ASSIGN |PARTITIONED|
- project ([$$e])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$49, $$e] <- gby.Employee
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access/explain_field_access.1.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access/explain_field_access.1.plan
new file mode 100644
index 0000000..d3c9b7d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access/explain_field_access.1.plan
@@ -0,0 +1,42 @@
+distribute result [$$47] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$47]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$47] <- [{"deptId": $#1, "star_cost": $$50}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ group by ([$#1 := $$55]) decor ([]) {
+ aggregate [$$50] <- [agg-global-sql-sum($$54)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- AGGREGATE |LOCAL|
+ nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ } [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SORT_GROUP_BY[$$55] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- HASH_PARTITION_EXCHANGE [$$55] |PARTITIONED|
+ group by ([$$55 := $$48]) decor ([]) {
+ aggregate [$$54] <- [agg-local-sql-sum($$45)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- AGGREGATE |LOCAL|
+ nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ } [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SORT_GROUP_BY[$$48] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$45, $$48]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$48, $$45] <- [substring($$e.getField("dept").getField("department_id"), 0), $$e.getField("salary")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$e]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$49, $$e] <- gby.Employee [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access_closed/explain_field_access_closed.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access_closed/explain_field_access_closed.1.adm
deleted file mode 100644
index bbdbb57..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access_closed/explain_field_access_closed.1.adm
+++ /dev/null
@@ -1,42 +0,0 @@
-distribute result [$$46]
--- DISTRIBUTE_RESULT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$46])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$46] <- [{"deptId": $#1, "star_cost": $$49}]
- -- ASSIGN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- group by ([$#1 := $$53]) decor ([]) {
- aggregate [$$49] <- [agg-global-sql-sum($$52)]
- -- AGGREGATE |LOCAL|
- nested tuple source
- -- NESTED_TUPLE_SOURCE |LOCAL|
- }
- -- SORT_GROUP_BY[$$53] |PARTITIONED|
- exchange
- -- HASH_PARTITION_EXCHANGE [$$53] |PARTITIONED|
- group by ([$$53 := $$47]) decor ([]) {
- aggregate [$$52] <- [agg-local-sql-sum($$44)]
- -- AGGREGATE |LOCAL|
- nested tuple source
- -- NESTED_TUPLE_SOURCE |LOCAL|
- }
- -- SORT_GROUP_BY[$$47] |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$44, $$47])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$47, $$44] <- [substring($$e.getField(1), 0), $$e.getField(2)]
- -- ASSIGN |PARTITIONED|
- project ([$$e])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$48, $$e] <- gby.Employee
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access_closed/explain_field_access_closed.1.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access_closed/explain_field_access_closed.1.plan
new file mode 100644
index 0000000..579fe54
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access_closed/explain_field_access_closed.1.plan
@@ -0,0 +1,42 @@
+distribute result [$$46] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$46]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$46] <- [{"deptId": $#1, "star_cost": $$49}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ group by ([$#1 := $$53]) decor ([]) {
+ aggregate [$$49] <- [agg-global-sql-sum($$52)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- AGGREGATE |LOCAL|
+ nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ } [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SORT_GROUP_BY[$$53] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- HASH_PARTITION_EXCHANGE [$$53] |PARTITIONED|
+ group by ([$$53 := $$47]) decor ([]) {
+ aggregate [$$52] <- [agg-local-sql-sum($$44)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- AGGREGATE |LOCAL|
+ nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ } [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SORT_GROUP_BY[$$47] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$44, $$47]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$47, $$44] <- [substring($$e.getField(1), 0), $$e.getField(2)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$e]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$48, $$e] <- gby.Employee [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_object_constructor-01/explain_object_constructor-01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_object_constructor-01/explain_object_constructor-01.1.adm
deleted file mode 100644
index 9a536a6..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_object_constructor-01/explain_object_constructor-01.1.adm
+++ /dev/null
@@ -1,8 +0,0 @@
-distribute result [$$2]
--- DISTRIBUTE_RESULT |UNPARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- assign [$$2] <- [{ a: "1", b: 1 }]
- -- ASSIGN |UNPARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_object_constructor-01/explain_object_constructor-01.1.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_object_constructor-01/explain_object_constructor-01.1.plan
new file mode 100644
index 0000000..2261f13
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_object_constructor-01/explain_object_constructor-01.1.plan
@@ -0,0 +1,8 @@
+distribute result [$$2] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ assign [$$2] <- [{ "a": "1", "b": 1 }] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |UNPARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_object_constructor-02/explain_object_constructor-02.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_object_constructor-02/explain_object_constructor-02.1.adm
deleted file mode 100644
index c637d87..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_object_constructor-02/explain_object_constructor-02.1.adm
+++ /dev/null
@@ -1,8 +0,0 @@
-distribute result [$$2]
--- DISTRIBUTE_RESULT |UNPARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- assign [$$2] <- [{ a: "1" }]
- -- ASSIGN |UNPARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_object_constructor-02/explain_object_constructor-02.1.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_object_constructor-02/explain_object_constructor-02.1.plan
new file mode 100644
index 0000000..1c7aaf6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_object_constructor-02/explain_object_constructor-02.1.plan
@@ -0,0 +1,8 @@
+distribute result [$$2] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ assign [$$2] <- [{ "a": "1" }] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |UNPARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_object_constructor-03/explain_object_constructor-03.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_object_constructor-03/explain_object_constructor-03.1.adm
deleted file mode 100644
index 83a9af1..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_object_constructor-03/explain_object_constructor-03.1.adm
+++ /dev/null
@@ -1,8 +0,0 @@
-distribute result [$$2]
--- DISTRIBUTE_RESULT |UNPARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- assign [$$2] <- [{ }]
- -- ASSIGN |UNPARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_object_constructor-03/explain_object_constructor-03.1.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_object_constructor-03/explain_object_constructor-03.1.plan
new file mode 100644
index 0000000..044388f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_object_constructor-03/explain_object_constructor-03.1.plan
@@ -0,0 +1,8 @@
+distribute result [$$2] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ assign [$$2] <- [{ }] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |UNPARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_same_datasource_function_different_arguments/test.020.unorderedtxt b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_same_datasource_function_different_arguments/test.020.unorderedtxt
new file mode 100644
index 0000000..24e0e67
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_same_datasource_function_different_arguments/test.020.unorderedtxt
@@ -0,0 +1,2 @@
+{ "intIndex": [ { "values": [ 15, 1 ] } ] }
+{ "stringIndex": [ { "values": [ "foo", 1 ] } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_same_datasource_function_different_arguments/test.030.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_same_datasource_function_different_arguments/test.030.plan
new file mode 100644
index 0000000..825f43a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_same_datasource_function_different_arguments/test.030.plan
@@ -0,0 +1,38 @@
+distribute result [$$11] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ union ($$16, $$17, $$11) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- UNION_ALL |UNPARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ project ([$$16]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |UNPARTITIONED|
+ assign [$$16] <- [cast({"intIndex": $$13})] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |UNPARTITIONED|
+ aggregate [$$13] <- [listify($$12)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- AGGREGATE |UNPARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$12] <- asterix.dump-index. [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ project ([$$17]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |UNPARTITIONED|
+ assign [$$17] <- [cast({"stringIndex": $$15})] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |UNPARTITIONED|
+ aggregate [$$15] <- [listify($$14)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- AGGREGATE |UNPARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$14] <- asterix.dump-index. [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_simple/explain_simple.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_simple/explain_simple.1.adm
deleted file mode 100644
index ad872a1..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_simple/explain_simple.1.adm
+++ /dev/null
@@ -1,8 +0,0 @@
-distribute result [$$2]
--- DISTRIBUTE_RESULT |UNPARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- assign [$$2] <- [2]
- -- ASSIGN |UNPARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_simple/explain_simple.1.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_simple/explain_simple.1.plan
new file mode 100644
index 0000000..0e8ada1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_simple/explain_simple.1.plan
@@ -0,0 +1,8 @@
+distribute result [$$2] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ assign [$$2] <- [2] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |UNPARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/json/json/external_dataset.008.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/json/json/external_dataset.008.adm
new file mode 100644
index 0000000..7660e7e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/json/json/external_dataset.008.adm
@@ -0,0 +1,25 @@
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3 ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3 ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3 ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3 ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3 ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/json/json/external_dataset.009.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/json/json/external_dataset.009.adm
new file mode 100644
index 0000000..7643986
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/json/json/external_dataset.009.adm
@@ -0,0 +1,25 @@
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ], "nested": { "id": 1 } }
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ], "nested": { "id": 1 } }
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ], "nested": { "id": 1 } }
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ], "nested": { "id": 1 } }
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ], "nested": { "id": 1 } }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ], "nested": { "id": 1 }, "nested2": [ { "id": 1 } ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ], "nested": { "id": 1 }, "nested2": [ { "id": 1 } ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ], "nested": { "id": 1 }, "nested2": [ { "id": 1 } ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ], "nested": { "id": 1 }, "nested2": [ { "id": 1 } ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ], "nested": { "id": 1 }, "nested2": [ { "id": 1 } ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ { "nested": { "array": [ 1, 2 ] } } ] } } ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ { "nested": { "array": [ 1, 2 ] } } ] } } ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ { "nested": { "array": [ 1, 2 ] } } ] } } ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ { "nested": { "array": [ 1, 2 ] } } ] } } ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ { "nested": { "array": [ 1, 2 ] } } ] } } ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3, { "nested1": { "id": 1, "nested2": { "id": 2, "nested3": [ { "nested4": null } ] } } } ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3, { "nested1": { "id": 1, "nested2": { "id": 2, "nested3": [ { "nested4": null } ] } } } ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3, { "nested1": { "id": 1, "nested2": { "id": 2, "nested3": [ { "nested4": null } ] } } } ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3, { "nested1": { "id": 1, "nested2": { "id": 2, "nested3": [ { "nested4": null } ] } } } ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3, { "nested1": { "id": 1, "nested2": { "id": 2, "nested3": [ { "nested4": null } ] } } } ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ 1, 2 ] } } ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ 1, 2 ] } } ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ 1, 2 ] } } ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ 1, 2 ] } } ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ 1, 2 ] } } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/json/json/external_dataset.010.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/json/json/external_dataset.010.adm
new file mode 100644
index 0000000..86babba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/json/json/external_dataset.010.adm
@@ -0,0 +1 @@
+50128
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/json/json/external_dataset.011.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/json/json/external_dataset.011.plan
new file mode 100644
index 0000000..6ec7195
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/json/json/external_dataset.011.plan
@@ -0,0 +1,48 @@
+distribute result [$$48] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ project ([$$48]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- STREAM_PROJECT |UNPARTITIONED|
+ assign [$$48] <- [{"cnt": $$51}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- ASSIGN |UNPARTITIONED|
+ aggregate [$$51] <- [agg-sql-sum($$53)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- AGGREGATE |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ aggregate [$$53] <- [agg-sql-count($$49)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- AGGREGATE |PARTITIONED|
+ project ([$$49]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ join (eq($$49, $$50)) [cardinality: 1000000.0, op-cost: 2000000.0, total-cost: 6000000.0]
+ -- HYBRID_HASH_JOIN [$$49][$$50] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
+ -- HASH_PARTITION_EXCHANGE [$$49] |PARTITIONED|
+ project ([$$49]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$49] <- [$$a.getField("quarter")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$a] <- test.test6 [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
+ -- HASH_PARTITION_EXCHANGE [$$50] |PARTITIONED|
+ project ([$$50]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$50] <- [$$b.getField("quarter")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$b] <- test.test6 [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/json/json/external_dataset.012.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/json/json/external_dataset.012.adm
new file mode 100644
index 0000000..6cd35ab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/json/json/external_dataset.012.adm
@@ -0,0 +1 @@
+{ "cnt": 15600 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/json/json/external_dataset.013.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/json/json/external_dataset.013.plan
new file mode 100644
index 0000000..22d5bd4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/json/json/external_dataset.013.plan
@@ -0,0 +1,62 @@
+distribute result [$$47] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ project ([$$47]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- STREAM_PROJECT |UNPARTITIONED|
+ assign [$$47] <- [{"cnt": $$50}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- ASSIGN |UNPARTITIONED|
+ aggregate [$$50] <- [agg-sql-sum($$52)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- AGGREGATE |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ aggregate [$$52] <- [agg-sql-count($$48)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- AGGREGATE |PARTITIONED|
+ project ([$$48]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ join (eq($$48, $$49)) [cardinality: 1000000.0, op-cost: 2000000.0, total-cost: 6000000.0]
+ -- HYBRID_HASH_JOIN [$$48][$$49] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$48]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$48] <- [$$49] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ replicate [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- REPLICATE |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
+ -- HASH_PARTITION_EXCHANGE [$$49] |PARTITIONED|
+ project ([$$49]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$49] <- [$$b.getField("quarter")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$b] <- test.test6 [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ replicate [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- REPLICATE |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
+ -- HASH_PARTITION_EXCHANGE [$$49] |PARTITIONED|
+ project ([$$49]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$49] <- [$$b.getField("quarter")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$b] <- test.test6 [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/json/json/external_dataset.014.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/json/json/external_dataset.014.adm
new file mode 100644
index 0000000..8006c78
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/json/json/external_dataset.014.adm
@@ -0,0 +1 @@
+{ "cnt": 1800 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.03.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.03.adm
deleted file mode 100644
index f0fad1e..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.03.adm
+++ /dev/null
@@ -1,28 +0,0 @@
-distribute result [$$17]
--- DISTRIBUTE_RESULT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$17])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$17] <- [{"display_url": $$20}]
- -- ASSIGN |PARTITIONED|
- project ([$$20])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- SORT_MERGE_EXCHANGE [$$19(ASC) ] |PARTITIONED|
- order (ASC, $$19)
- -- STABLE_SORT [$$19(ASC)] |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$20, $$19])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$20, $$19] <- [get-item($$p.getField("entities").getField("urls"), 0).getField("display_url"), $$p.getField("id")]
- -- ASSIGN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$p] <- test.ParquetDataset
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.03.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.03.plan
new file mode 100644
index 0000000..6c19f9e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.03.plan
@@ -0,0 +1,28 @@
+distribute result [$$17] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$17]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$17] <- [{"display_url": $$20}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$20]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SORT_MERGE_EXCHANGE [$$19(ASC) ] |PARTITIONED|
+ order (ASC, $$19) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STABLE_SORT [$$19(ASC)] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$20, $$19]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$20, $$19] <- [get-item($$p.getField("entities").getField("urls"), 0).getField("display_url"), $$p.getField("id")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p] <- test.ParquetDataset [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.05.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.05.adm
deleted file mode 100644
index 2a24b09..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.05.adm
+++ /dev/null
@@ -1,28 +0,0 @@
-distribute result [$$17]
--- DISTRIBUTE_RESULT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$17])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$17] <- [{"display_url": $$20}]
- -- ASSIGN |PARTITIONED|
- project ([$$20])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- SORT_MERGE_EXCHANGE [$$19(ASC) ] |PARTITIONED|
- order (ASC, $$19)
- -- STABLE_SORT [$$19(ASC)] |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$20, $$19])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$20, $$19] <- [get-item($$p.getField("entities").getField("urls"), 0).getField("display_url"), $$p.getField("id")]
- -- ASSIGN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$p] <- test.ParquetDataset project ({entities:{urls:[{display_url:any}]},id:any})
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.05.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.05.plan
new file mode 100644
index 0000000..dc2410a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.05.plan
@@ -0,0 +1,28 @@
+distribute result [$$17] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$17]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$17] <- [{"display_url": $$20}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$20]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SORT_MERGE_EXCHANGE [$$19(ASC) ] |PARTITIONED|
+ order (ASC, $$19) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STABLE_SORT [$$19(ASC)] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$20, $$19]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$20, $$19] <- [get-item($$p.getField("entities").getField("urls"), 0).getField("display_url"), $$p.getField("id")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p] <- test.ParquetDataset project ({entities:{urls:[{display_url:any}]},id:any}) [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.07.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.07.adm
deleted file mode 100644
index 1ed4eed..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.07.adm
+++ /dev/null
@@ -1,34 +0,0 @@
-distribute result [$$21]
--- DISTRIBUTE_RESULT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$21])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$21] <- [{"display_url": $$25}]
- -- ASSIGN |PARTITIONED|
- project ([$$25])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- SORT_MERGE_EXCHANGE [$$24(ASC) ] |PARTITIONED|
- order (ASC, $$24)
- -- STABLE_SORT [$$24(ASC)] |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$25, $$24])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$25] <- [array-star($$23).getField("display_url")]
- -- ASSIGN |PARTITIONED|
- select (not(is-missing($$23)))
- -- STREAM_SELECT |PARTITIONED|
- project ([$$23, $$24])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$23, $$24] <- [$$p.getField("entities").getField("urls"), $$p.getField("id")]
- -- ASSIGN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$p] <- test.ParquetDataset
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.07.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.07.plan
new file mode 100644
index 0000000..2ecb7fe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.07.plan
@@ -0,0 +1,34 @@
+distribute result [$$21] [cardinality: 2.1, op-cost: 0.0, total-cost: 2.1]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 2.1]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$21]) [cardinality: 2.1, op-cost: 0.0, total-cost: 2.1]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$21] <- [{"display_url": $$25}] [cardinality: 2.1, op-cost: 0.0, total-cost: 2.1]
+ -- ASSIGN |PARTITIONED|
+ project ([$$25]) [cardinality: 2.1, op-cost: 0.0, total-cost: 2.1]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 2.1]
+ -- SORT_MERGE_EXCHANGE [$$24(ASC) ] |PARTITIONED|
+ order (ASC, $$24) [cardinality: 2.1, op-cost: 0.0, total-cost: 2.1]
+ -- STABLE_SORT [$$24(ASC)] |PARTITIONED|
+ exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 2.1]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$25, $$24]) [cardinality: 2.1, op-cost: 0.0, total-cost: 2.1]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$25] <- [array-star($$23).getField("display_url")] [cardinality: 2.1, op-cost: 0.0, total-cost: 2.1]
+ -- ASSIGN |PARTITIONED|
+ select (not(is-missing($$23))) [cardinality: 2.1, op-cost: 0.0, total-cost: 2.1]
+ -- STREAM_SELECT |PARTITIONED|
+ project ([$$23, $$24]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$23, $$24] <- [$$p.getField("entities").getField("urls"), $$p.getField("id")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p] <- test.ParquetDataset [cardinality: 1000000.0, op-cost: 2.1, total-cost: 2.1]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.09.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.09.adm
deleted file mode 100644
index 2302d35..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.09.adm
+++ /dev/null
@@ -1,34 +0,0 @@
-distribute result [$$21]
--- DISTRIBUTE_RESULT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$21])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$21] <- [{"display_url": $$25}]
- -- ASSIGN |PARTITIONED|
- project ([$$25])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- SORT_MERGE_EXCHANGE [$$24(ASC) ] |PARTITIONED|
- order (ASC, $$24)
- -- STABLE_SORT [$$24(ASC)] |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$25, $$24])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$25] <- [array-star($$23).getField("display_url")]
- -- ASSIGN |PARTITIONED|
- select (not(is-missing($$23)))
- -- STREAM_SELECT |PARTITIONED|
- project ([$$23, $$24])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$23, $$24] <- [$$p.getField("entities").getField("urls"), $$p.getField("id")]
- -- ASSIGN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$p] <- test.ParquetDataset project ({entities:{urls:[{display_url:any}]},id:any})
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.09.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.09.plan
new file mode 100644
index 0000000..8dfb99e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.09.plan
@@ -0,0 +1,34 @@
+distribute result [$$21] [cardinality: 2.1, op-cost: 0.0, total-cost: 2.1]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 2.1]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$21]) [cardinality: 2.1, op-cost: 0.0, total-cost: 2.1]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$21] <- [{"display_url": $$25}] [cardinality: 2.1, op-cost: 0.0, total-cost: 2.1]
+ -- ASSIGN |PARTITIONED|
+ project ([$$25]) [cardinality: 2.1, op-cost: 0.0, total-cost: 2.1]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 2.1]
+ -- SORT_MERGE_EXCHANGE [$$24(ASC) ] |PARTITIONED|
+ order (ASC, $$24) [cardinality: 2.1, op-cost: 0.0, total-cost: 2.1]
+ -- STABLE_SORT [$$24(ASC)] |PARTITIONED|
+ exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 2.1]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$25, $$24]) [cardinality: 2.1, op-cost: 0.0, total-cost: 2.1]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$25] <- [array-star($$23).getField("display_url")] [cardinality: 2.1, op-cost: 0.0, total-cost: 2.1]
+ -- ASSIGN |PARTITIONED|
+ select (not(is-missing($$23))) [cardinality: 2.1, op-cost: 0.0, total-cost: 2.1]
+ -- STREAM_SELECT |PARTITIONED|
+ project ([$$23, $$24]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$23, $$24] <- [$$p.getField("entities").getField("urls"), $$p.getField("id")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p] <- test.ParquetDataset project ({entities:{urls:[{display_url:any}]},id:any}) [cardinality: 1000000.0, op-cost: 2.1, total-cost: 2.1]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.11.adm
deleted file mode 100644
index a9ee166..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.11.adm
+++ /dev/null
@@ -1,32 +0,0 @@
-distribute result [$$26]
--- DISTRIBUTE_RESULT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$26])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$26] <- [{"display_url": $$urls.getField("display_url")}]
- -- ASSIGN |PARTITIONED|
- project ([$$urls])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- SORT_MERGE_EXCHANGE [$$29(ASC) ] |PARTITIONED|
- order (ASC, $$29)
- -- STABLE_SORT [$$29(ASC)] |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$urls, $$29])
- -- STREAM_PROJECT |PARTITIONED|
- unnest $$urls <- scan-collection($$28)
- -- UNNEST |PARTITIONED|
- project ([$$28, $$29])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$28, $$29] <- [$$p.getField("entities").getField("urls"), $$p.getField("id")]
- -- ASSIGN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$p] <- test.ParquetDataset
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.11.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.11.plan
new file mode 100644
index 0000000..6c38ed4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.11.plan
@@ -0,0 +1,32 @@
+distribute result [$$26] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$26]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$26] <- [{"display_url": $$urls.getField("display_url")}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$urls]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SORT_MERGE_EXCHANGE [$$29(ASC) ] |PARTITIONED|
+ order (ASC, $$29) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STABLE_SORT [$$29(ASC)] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$urls, $$29]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ unnest $$urls <- scan-collection($$28) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- UNNEST |PARTITIONED|
+ project ([$$28, $$29]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$28, $$29] <- [$$p.getField("entities").getField("urls"), $$p.getField("id")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p] <- test.ParquetDataset [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.13.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.13.adm
deleted file mode 100644
index a29ef6c..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.13.adm
+++ /dev/null
@@ -1,32 +0,0 @@
-distribute result [$$26]
--- DISTRIBUTE_RESULT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$26])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$26] <- [{"display_url": $$urls.getField("display_url")}]
- -- ASSIGN |PARTITIONED|
- project ([$$urls])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- SORT_MERGE_EXCHANGE [$$29(ASC) ] |PARTITIONED|
- order (ASC, $$29)
- -- STABLE_SORT [$$29(ASC)] |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$urls, $$29])
- -- STREAM_PROJECT |PARTITIONED|
- unnest $$urls <- scan-collection($$28)
- -- UNNEST |PARTITIONED|
- project ([$$28, $$29])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$28, $$29] <- [$$p.getField("entities").getField("urls"), $$p.getField("id")]
- -- ASSIGN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$p] <- test.ParquetDataset project ({entities:{urls:[{display_url:any}]},id:any})
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.13.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.13.plan
new file mode 100644
index 0000000..27149e6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.13.plan
@@ -0,0 +1,32 @@
+distribute result [$$26] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$26]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$26] <- [{"display_url": $$urls.getField("display_url")}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$urls]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SORT_MERGE_EXCHANGE [$$29(ASC) ] |PARTITIONED|
+ order (ASC, $$29) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STABLE_SORT [$$29(ASC)] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$urls, $$29]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ unnest $$urls <- scan-collection($$28) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- UNNEST |PARTITIONED|
+ project ([$$28, $$29]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$28, $$29] <- [$$p.getField("entities").getField("urls"), $$p.getField("id")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p] <- test.ParquetDataset project ({entities:{urls:[{display_url:any}]},id:any}) [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.15.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.15.adm
deleted file mode 100644
index de214d0..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.15.adm
+++ /dev/null
@@ -1,39 +0,0 @@
-distribute result [$$46]
--- DISTRIBUTE_RESULT |UNPARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- aggregate [$$46] <- [agg-sql-sum($$52)]
- -- AGGREGATE |UNPARTITIONED|
- exchange
- -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
- aggregate [$$52] <- [agg-sql-count(1)]
- -- AGGREGATE |PARTITIONED|
- select ($$39)
- -- STREAM_SELECT |PARTITIONED|
- project ([$$39])
- -- STREAM_PROJECT |PARTITIONED|
- subplan {
- aggregate [$$39] <- [empty-stream()]
- -- AGGREGATE |LOCAL|
- select (not(if-missing-or-null(eq($$48, "string"), FALSE)))
- -- STREAM_SELECT |LOCAL|
- assign [$$48] <- [$$ht.getField("display_url")]
- -- ASSIGN |LOCAL|
- unnest $$ht <- scan-collection($$47)
- -- UNNEST |LOCAL|
- nested tuple source
- -- NESTED_TUPLE_SOURCE |LOCAL|
- }
- -- SUBPLAN |PARTITIONED|
- project ([$$47])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$47] <- [$$p.getField("entities").getField("urls")]
- -- ASSIGN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$p] <- test.ParquetDataset
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.15.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.15.plan
new file mode 100644
index 0000000..5551e5a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.15.plan
@@ -0,0 +1,39 @@
+distribute result [$$46] [cardinality: 2.1, op-cost: 0.0, total-cost: 2.1]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 2.1]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ aggregate [$$46] <- [agg-sql-sum($$52)] [cardinality: 2.1, op-cost: 0.0, total-cost: 2.1]
+ -- AGGREGATE |UNPARTITIONED|
+ exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 2.1]
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ aggregate [$$52] <- [agg-sql-count(1)] [cardinality: 2.1, op-cost: 0.0, total-cost: 2.1]
+ -- AGGREGATE |PARTITIONED|
+ select ($$39) [cardinality: 2.1, op-cost: 0.0, total-cost: 2.1]
+ -- STREAM_SELECT |PARTITIONED|
+ project ([$$39]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- STREAM_PROJECT |PARTITIONED|
+ subplan {
+ aggregate [$$39] <- [empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- AGGREGATE |LOCAL|
+ select (not(if-missing-or-null(eq($$48, "string"), false))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_SELECT |LOCAL|
+ assign [$$48] <- [$$ht.getField("display_url")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |LOCAL|
+ unnest $$ht <- scan-collection($$47) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- UNNEST |LOCAL|
+ nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ } [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- SUBPLAN |PARTITIONED|
+ project ([$$47]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$47] <- [$$p.getField("entities").getField("urls")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p] <- test.ParquetDataset [cardinality: 1000000.0, op-cost: 2.1, total-cost: 2.1]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.17.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.17.adm
deleted file mode 100644
index 9f122c6..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.17.adm
+++ /dev/null
@@ -1,39 +0,0 @@
-distribute result [$$46]
--- DISTRIBUTE_RESULT |UNPARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- aggregate [$$46] <- [agg-sql-sum($$52)]
- -- AGGREGATE |UNPARTITIONED|
- exchange
- -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
- aggregate [$$52] <- [agg-sql-count(1)]
- -- AGGREGATE |PARTITIONED|
- select ($$39)
- -- STREAM_SELECT |PARTITIONED|
- project ([$$39])
- -- STREAM_PROJECT |PARTITIONED|
- subplan {
- aggregate [$$39] <- [empty-stream()]
- -- AGGREGATE |LOCAL|
- select (not(if-missing-or-null(eq($$48, "string"), FALSE)))
- -- STREAM_SELECT |LOCAL|
- assign [$$48] <- [$$ht.getField("display_url")]
- -- ASSIGN |LOCAL|
- unnest $$ht <- scan-collection($$47)
- -- UNNEST |LOCAL|
- nested tuple source
- -- NESTED_TUPLE_SOURCE |LOCAL|
- }
- -- SUBPLAN |PARTITIONED|
- project ([$$47])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$47] <- [$$p.getField("entities").getField("urls")]
- -- ASSIGN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$p] <- test.ParquetDataset project ({entities:{urls:[{display_url:any}]}})
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.17.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.17.plan
new file mode 100644
index 0000000..76fdcc8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.17.plan
@@ -0,0 +1,39 @@
+distribute result [$$46] [cardinality: 2.1, op-cost: 0.0, total-cost: 2.1]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 2.1]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ aggregate [$$46] <- [agg-sql-sum($$52)] [cardinality: 2.1, op-cost: 0.0, total-cost: 2.1]
+ -- AGGREGATE |UNPARTITIONED|
+ exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 2.1]
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ aggregate [$$52] <- [agg-sql-count(1)] [cardinality: 2.1, op-cost: 0.0, total-cost: 2.1]
+ -- AGGREGATE |PARTITIONED|
+ select ($$39) [cardinality: 2.1, op-cost: 0.0, total-cost: 2.1]
+ -- STREAM_SELECT |PARTITIONED|
+ project ([$$39]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- STREAM_PROJECT |PARTITIONED|
+ subplan {
+ aggregate [$$39] <- [empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- AGGREGATE |LOCAL|
+ select (not(if-missing-or-null(eq($$48, "string"), false))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_SELECT |LOCAL|
+ assign [$$48] <- [$$ht.getField("display_url")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |LOCAL|
+ unnest $$ht <- scan-collection($$47) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- UNNEST |LOCAL|
+ nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ } [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- SUBPLAN |PARTITIONED|
+ project ([$$47]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$47] <- [$$p.getField("entities").getField("urls")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p] <- test.ParquetDataset project ({entities:{urls:[{display_url:any}]}}) [cardinality: 1000000.0, op-cost: 2.1, total-cost: 2.1]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.19.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.19.adm
deleted file mode 100644
index 37849ca..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.19.adm
+++ /dev/null
@@ -1,28 +0,0 @@
-distribute result [$$18]
--- DISTRIBUTE_RESULT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$18])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$18] <- [get-item(get-item(get-item($$21, 0), 0), 0)]
- -- ASSIGN |PARTITIONED|
- project ([$$21])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- SORT_MERGE_EXCHANGE [$$20(ASC) ] |PARTITIONED|
- order (ASC, $$20)
- -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$21, $$20])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$21, $$20] <- [$$p.getField("place").getField("bounding_box").getField("coordinates"), $$p.getField("id")]
- -- ASSIGN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$p] <- test.ParquetDataset
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.19.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.19.plan
new file mode 100644
index 0000000..6c14c8c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.19.plan
@@ -0,0 +1,28 @@
+distribute result [$$18] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$18]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$18] <- [get-item(get-item(get-item($$21, 0), 0), 0)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$21]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SORT_MERGE_EXCHANGE [$$20(ASC) ] |PARTITIONED|
+ order (ASC, $$20) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$21, $$20]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$21, $$20] <- [$$p.getField("place").getField("bounding_box").getField("coordinates"), $$p.getField("id")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p] <- test.ParquetDataset [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.21.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.21.adm
deleted file mode 100644
index ee3a128..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.21.adm
+++ /dev/null
@@ -1,28 +0,0 @@
-distribute result [$$18]
--- DISTRIBUTE_RESULT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$18])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$18] <- [get-item(get-item(get-item($$21, 0), 0), 0)]
- -- ASSIGN |PARTITIONED|
- project ([$$21])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- SORT_MERGE_EXCHANGE [$$20(ASC) ] |PARTITIONED|
- order (ASC, $$20)
- -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$21, $$20])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$21, $$20] <- [$$p.getField("place").getField("bounding_box").getField("coordinates"), $$p.getField("id")]
- -- ASSIGN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$p] <- test.ParquetDataset project ({place:{bounding_box:{coordinates:[[[any]]]}},id:any})
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.21.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.21.plan
new file mode 100644
index 0000000..a22adfa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.21.plan
@@ -0,0 +1,28 @@
+distribute result [$$18] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$18]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$18] <- [get-item(get-item(get-item($$21, 0), 0), 0)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$21]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SORT_MERGE_EXCHANGE [$$20(ASC) ] |PARTITIONED|
+ order (ASC, $$20) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$21, $$20]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$21, $$20] <- [$$p.getField("place").getField("bounding_box").getField("coordinates"), $$p.getField("id")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p] <- test.ParquetDataset project ({place:{bounding_box:{coordinates:[[[any]]]}},id:any}) [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.03.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.03.adm
deleted file mode 100644
index 5260a0a..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.03.adm
+++ /dev/null
@@ -1,22 +0,0 @@
-distribute result [$$p]
--- DISTRIBUTE_RESULT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$p])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- SORT_MERGE_EXCHANGE [$$13(ASC) ] |PARTITIONED|
- order (ASC, $$13)
- -- STABLE_SORT [$$13(ASC)] |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- assign [$$13] <- [$$p.getField("id")]
- -- ASSIGN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$p] <- test.ParquetDataset
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.03.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.03.plan
new file mode 100644
index 0000000..a8803e7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.03.plan
@@ -0,0 +1,22 @@
+distribute result [$$p] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$p]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SORT_MERGE_EXCHANGE [$$13(ASC) ] |PARTITIONED|
+ order (ASC, $$13) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STABLE_SORT [$$13(ASC)] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ assign [$$13] <- [$$p.getField("id")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p] <- test.ParquetDataset [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.05.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.05.adm
deleted file mode 100644
index 5260a0a..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.05.adm
+++ /dev/null
@@ -1,22 +0,0 @@
-distribute result [$$p]
--- DISTRIBUTE_RESULT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$p])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- SORT_MERGE_EXCHANGE [$$13(ASC) ] |PARTITIONED|
- order (ASC, $$13)
- -- STABLE_SORT [$$13(ASC)] |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- assign [$$13] <- [$$p.getField("id")]
- -- ASSIGN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$p] <- test.ParquetDataset
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.05.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.05.plan
new file mode 100644
index 0000000..a8803e7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.05.plan
@@ -0,0 +1,22 @@
+distribute result [$$p] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$p]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SORT_MERGE_EXCHANGE [$$13(ASC) ] |PARTITIONED|
+ order (ASC, $$13) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STABLE_SORT [$$13(ASC)] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ assign [$$13] <- [$$p.getField("id")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p] <- test.ParquetDataset [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.07.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.07.adm
deleted file mode 100644
index 48d64c1..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.07.adm
+++ /dev/null
@@ -1,46 +0,0 @@
-distribute result [$$28]
--- DISTRIBUTE_RESULT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$28])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$28] <- [{"p1": $$p1, "id": $$30}]
- -- ASSIGN |PARTITIONED|
- exchange
- -- SORT_MERGE_EXCHANGE [$$30(ASC) ] |PARTITIONED|
- order (ASC, $$30)
- -- STABLE_SORT [$$30(ASC)] |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$p1, $$30])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- join (eq($$31, $$30))
- -- HYBRID_HASH_JOIN [$$31][$$30] |PARTITIONED|
- exchange
- -- HASH_PARTITION_EXCHANGE [$$31] |PARTITIONED|
- assign [$$31] <- [$$p1.getField("id")]
- -- ASSIGN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$p1] <- test.ParquetDataset
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- exchange
- -- HASH_PARTITION_EXCHANGE [$$30] |PARTITIONED|
- project ([$$30])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$30] <- [$$p2.getField("id")]
- -- ASSIGN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$p2] <- test.ParquetDataset2
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.07.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.07.plan
new file mode 100644
index 0000000..e555057
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.07.plan
@@ -0,0 +1,46 @@
+distribute result [$$28] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$28]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$28] <- [{"p1": $$p1, "id": $$30}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- SORT_MERGE_EXCHANGE [$$30(ASC) ] |PARTITIONED|
+ order (ASC, $$30) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- STABLE_SORT [$$30(ASC)] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$p1, $$30]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ join (eq($$31, $$30)) [cardinality: 1000000.0, op-cost: 2000000.0, total-cost: 6000000.0]
+ -- HYBRID_HASH_JOIN [$$31][$$30] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
+ -- HASH_PARTITION_EXCHANGE [$$31] |PARTITIONED|
+ assign [$$31] <- [$$p1.getField("id")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p1] <- test.ParquetDataset [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
+ -- HASH_PARTITION_EXCHANGE [$$30] |PARTITIONED|
+ project ([$$30]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$30] <- [$$p2.getField("id")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p2] <- test.ParquetDataset2 [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.09.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.09.adm
deleted file mode 100644
index 2e7bd2b..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.09.adm
+++ /dev/null
@@ -1,46 +0,0 @@
-distribute result [$$28]
--- DISTRIBUTE_RESULT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$28])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$28] <- [{"p1": $$p1, "id": $$30}]
- -- ASSIGN |PARTITIONED|
- exchange
- -- SORT_MERGE_EXCHANGE [$$30(ASC) ] |PARTITIONED|
- order (ASC, $$30)
- -- STABLE_SORT [$$30(ASC)] |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$p1, $$30])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- join (eq($$31, $$30))
- -- HYBRID_HASH_JOIN [$$31][$$30] |PARTITIONED|
- exchange
- -- HASH_PARTITION_EXCHANGE [$$31] |PARTITIONED|
- assign [$$31] <- [$$p1.getField("id")]
- -- ASSIGN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$p1] <- test.ParquetDataset
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- exchange
- -- HASH_PARTITION_EXCHANGE [$$30] |PARTITIONED|
- project ([$$30])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$30] <- [$$p2.getField("id")]
- -- ASSIGN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$p2] <- test.ParquetDataset2 project ({id:any})
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.09.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.09.plan
new file mode 100644
index 0000000..534738a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.09.plan
@@ -0,0 +1,46 @@
+distribute result [$$28] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$28]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$28] <- [{"p1": $$p1, "id": $$30}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- SORT_MERGE_EXCHANGE [$$30(ASC) ] |PARTITIONED|
+ order (ASC, $$30) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- STABLE_SORT [$$30(ASC)] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$p1, $$30]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ join (eq($$31, $$30)) [cardinality: 1000000.0, op-cost: 2000000.0, total-cost: 6000000.0]
+ -- HYBRID_HASH_JOIN [$$31][$$30] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
+ -- HASH_PARTITION_EXCHANGE [$$31] |PARTITIONED|
+ assign [$$31] <- [$$p1.getField("id")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p1] <- test.ParquetDataset [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
+ -- HASH_PARTITION_EXCHANGE [$$30] |PARTITIONED|
+ project ([$$30]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$30] <- [$$p2.getField("id")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p2] <- test.ParquetDataset2 project ({id:any}) [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.11.adm
deleted file mode 100644
index 8eb02fe..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.11.adm
+++ /dev/null
@@ -1,50 +0,0 @@
-distribute result [$$29]
--- DISTRIBUTE_RESULT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$29])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$29] <- [{"age": $$34, "name": $$35}]
- -- ASSIGN |PARTITIONED|
- project ([$$34, $$35])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- SORT_MERGE_EXCHANGE [$$32(ASC) ] |PARTITIONED|
- order (ASC, $$32)
- -- STABLE_SORT [$$32(ASC)] |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$34, $$35, $$32])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- join (eq($$31, $$32))
- -- HYBRID_HASH_JOIN [$$31][$$32] |PARTITIONED|
- exchange
- -- HASH_PARTITION_EXCHANGE [$$31] |PARTITIONED|
- project ([$$34, $$31])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$34, $$31] <- [$$p1.getField("age"), $$p1.getField("id")]
- -- ASSIGN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$p1] <- test.ParquetDataset
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- exchange
- -- HASH_PARTITION_EXCHANGE [$$32] |PARTITIONED|
- project ([$$35, $$32])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$35, $$32] <- [$$p2.getField("name"), $$p2.getField("id")]
- -- ASSIGN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$p2] <- test.ParquetDataset3
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.11.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.11.plan
new file mode 100644
index 0000000..1060512
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.11.plan
@@ -0,0 +1,50 @@
+distribute result [$$29] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$29]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$29] <- [{"age": $$34, "name": $$35}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$34, $$35]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- SORT_MERGE_EXCHANGE [$$32(ASC) ] |PARTITIONED|
+ order (ASC, $$32) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- STABLE_SORT [$$32(ASC)] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$34, $$35, $$32]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ join (eq($$31, $$32)) [cardinality: 1000000.0, op-cost: 2000000.0, total-cost: 6000000.0]
+ -- HYBRID_HASH_JOIN [$$31][$$32] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
+ -- HASH_PARTITION_EXCHANGE [$$31] |PARTITIONED|
+ project ([$$34, $$31]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$34, $$31] <- [$$p1.getField("age"), $$p1.getField("id")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p1] <- test.ParquetDataset [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
+ -- HASH_PARTITION_EXCHANGE [$$32] |PARTITIONED|
+ project ([$$35, $$32]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$35, $$32] <- [$$p2.getField("name"), $$p2.getField("id")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p2] <- test.ParquetDataset3 [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.13.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.13.adm
deleted file mode 100644
index 12e1b6b..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.13.adm
+++ /dev/null
@@ -1,50 +0,0 @@
-distribute result [$$29]
--- DISTRIBUTE_RESULT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$29])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$29] <- [{"age": $$34, "name": $$35}]
- -- ASSIGN |PARTITIONED|
- project ([$$34, $$35])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- SORT_MERGE_EXCHANGE [$$32(ASC) ] |PARTITIONED|
- order (ASC, $$32)
- -- STABLE_SORT [$$32(ASC)] |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$34, $$35, $$32])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- join (eq($$31, $$32))
- -- HYBRID_HASH_JOIN [$$31][$$32] |PARTITIONED|
- exchange
- -- HASH_PARTITION_EXCHANGE [$$31] |PARTITIONED|
- project ([$$34, $$31])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$34, $$31] <- [$$p1.getField("age"), $$p1.getField("id")]
- -- ASSIGN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$p1] <- test.ParquetDataset project ({id:any,age:any})
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- exchange
- -- HASH_PARTITION_EXCHANGE [$$32] |PARTITIONED|
- project ([$$35, $$32])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$35, $$32] <- [$$p2.getField("name"), $$p2.getField("id")]
- -- ASSIGN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$p2] <- test.ParquetDataset3 project ({name:any,id:any})
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.13.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.13.plan
new file mode 100644
index 0000000..90e058c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.13.plan
@@ -0,0 +1,50 @@
+distribute result [$$29] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$29]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$29] <- [{"age": $$34, "name": $$35}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$34, $$35]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- SORT_MERGE_EXCHANGE [$$32(ASC) ] |PARTITIONED|
+ order (ASC, $$32) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- STABLE_SORT [$$32(ASC)] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$34, $$35, $$32]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ join (eq($$31, $$32)) [cardinality: 1000000.0, op-cost: 2000000.0, total-cost: 6000000.0]
+ -- HYBRID_HASH_JOIN [$$31][$$32] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
+ -- HASH_PARTITION_EXCHANGE [$$31] |PARTITIONED|
+ project ([$$34, $$31]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$34, $$31] <- [$$p1.getField("age"), $$p1.getField("id")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p1] <- test.ParquetDataset project ({id:any,age:any}) [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
+ -- HASH_PARTITION_EXCHANGE [$$32] |PARTITIONED|
+ project ([$$35, $$32]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$35, $$32] <- [$$p2.getField("name"), $$p2.getField("id")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p2] <- test.ParquetDataset3 project ({name:any,id:any}) [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.15.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.15.adm
deleted file mode 100644
index dc8c103..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.15.adm
+++ /dev/null
@@ -1,32 +0,0 @@
-distribute result [$$17]
--- DISTRIBUTE_RESULT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$17])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$17] <- [{"id": $$21, "name": $$22}]
- -- ASSIGN |PARTITIONED|
- project ([$$21, $$22])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- SORT_MERGE_EXCHANGE [$$20(ASC) ] |PARTITIONED|
- order (ASC, $$20)
- -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$21, $$22, $$20])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$22, $$21] <- [$$19.getField("name"), $$19.getField("id")]
- -- ASSIGN |PARTITIONED|
- project ([$$19, $$20])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$19, $$20] <- [$$p.getField("user"), $$p.getField("id")]
- -- ASSIGN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$p] <- test.ParquetDataset4
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.15.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.15.plan
new file mode 100644
index 0000000..f90c114
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.15.plan
@@ -0,0 +1,32 @@
+distribute result [$$17] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$17]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$17] <- [{"id": $$21, "name": $$22}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$21, $$22]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SORT_MERGE_EXCHANGE [$$20(ASC) ] |PARTITIONED|
+ order (ASC, $$20) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$21, $$22, $$20]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$22, $$21] <- [$$19.getField("name"), $$19.getField("id")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$19, $$20]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$19, $$20] <- [$$p.getField("user"), $$p.getField("id")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p] <- test.ParquetDataset4 [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.17.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.17.adm
deleted file mode 100644
index 1ffc9a7..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.17.adm
+++ /dev/null
@@ -1,32 +0,0 @@
-distribute result [$$17]
--- DISTRIBUTE_RESULT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$17])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$17] <- [{"id": $$21, "name": $$22}]
- -- ASSIGN |PARTITIONED|
- project ([$$21, $$22])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- SORT_MERGE_EXCHANGE [$$20(ASC) ] |PARTITIONED|
- order (ASC, $$20)
- -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$21, $$22, $$20])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$22, $$21] <- [$$19.getField("name"), $$19.getField("id")]
- -- ASSIGN |PARTITIONED|
- project ([$$19, $$20])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$19, $$20] <- [$$p.getField("user"), $$p.getField("id")]
- -- ASSIGN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$p] <- test.ParquetDataset4 project ({id:any,user:{name:any,id:any}})
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.17.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.17.plan
new file mode 100644
index 0000000..98ee3ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.17.plan
@@ -0,0 +1,32 @@
+distribute result [$$17] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$17]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$17] <- [{"id": $$21, "name": $$22}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$21, $$22]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SORT_MERGE_EXCHANGE [$$20(ASC) ] |PARTITIONED|
+ order (ASC, $$20) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$21, $$22, $$20]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$22, $$21] <- [$$19.getField("name"), $$19.getField("id")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$19, $$20]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$19, $$20] <- [$$p.getField("user"), $$p.getField("id")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p] <- test.ParquetDataset4 project ({id:any,user:{name:any,id:any}}) [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.19.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.19.adm
deleted file mode 100644
index 229e53a..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.19.adm
+++ /dev/null
@@ -1,18 +0,0 @@
-distribute result [$$31]
--- DISTRIBUTE_RESULT |UNPARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- aggregate [$$31] <- [agg-sql-sum($$32)]
- -- AGGREGATE |UNPARTITIONED|
- exchange
- -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
- aggregate [$$32] <- [agg-sql-count(1)]
- -- AGGREGATE |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$p] <- test.ParquetDataset4
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.19.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.19.plan
new file mode 100644
index 0000000..6c9771b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.19.plan
@@ -0,0 +1,18 @@
+distribute result [$$31] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ aggregate [$$31] <- [agg-sql-sum($$32)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- AGGREGATE |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ aggregate [$$32] <- [agg-sql-count(1)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- AGGREGATE |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p] <- test.ParquetDataset4 [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.21.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.21.adm
deleted file mode 100644
index 6937d90..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.21.adm
+++ /dev/null
@@ -1,18 +0,0 @@
-distribute result [$$31]
--- DISTRIBUTE_RESULT |UNPARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- aggregate [$$31] <- [agg-sql-sum($$32)]
- -- AGGREGATE |UNPARTITIONED|
- exchange
- -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
- aggregate [$$32] <- [agg-sql-count(1)]
- -- AGGREGATE |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$p] <- test.ParquetDataset4 project ({})
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.21.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.21.plan
new file mode 100644
index 0000000..7d3c4ce
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.21.plan
@@ -0,0 +1,18 @@
+distribute result [$$31] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ aggregate [$$31] <- [agg-sql-sum($$32)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- AGGREGATE |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ aggregate [$$32] <- [agg-sql-count(1)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- AGGREGATE |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p] <- test.ParquetDataset4 project ({}) [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.22.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.22.adm
deleted file mode 100644
index 1ffc9a7..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.22.adm
+++ /dev/null
@@ -1,32 +0,0 @@
-distribute result [$$17]
--- DISTRIBUTE_RESULT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$17])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$17] <- [{"id": $$21, "name": $$22}]
- -- ASSIGN |PARTITIONED|
- project ([$$21, $$22])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- SORT_MERGE_EXCHANGE [$$20(ASC) ] |PARTITIONED|
- order (ASC, $$20)
- -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$21, $$22, $$20])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$22, $$21] <- [$$19.getField("name"), $$19.getField("id")]
- -- ASSIGN |PARTITIONED|
- project ([$$19, $$20])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$19, $$20] <- [$$p.getField("user"), $$p.getField("id")]
- -- ASSIGN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$p] <- test.ParquetDataset4 project ({id:any,user:{name:any,id:any}})
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.22.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.22.plan
new file mode 100644
index 0000000..98ee3ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.22.plan
@@ -0,0 +1,32 @@
+distribute result [$$17] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$17]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$17] <- [{"id": $$21, "name": $$22}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$21, $$22]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SORT_MERGE_EXCHANGE [$$20(ASC) ] |PARTITIONED|
+ order (ASC, $$20) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$21, $$22, $$20]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$22, $$21] <- [$$19.getField("name"), $$19.getField("id")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$19, $$20]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$19, $$20] <- [$$p.getField("user"), $$p.getField("id")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p] <- test.ParquetDataset4 project ({id:any,user:{name:any,id:any}}) [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.03.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.03.adm
deleted file mode 100644
index 4a602ef..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.03.adm
+++ /dev/null
@@ -1,30 +0,0 @@
-distribute result [$$20]
--- DISTRIBUTE_RESULT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$20])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$20] <- [switch-case(TRUE, is-array($$22), $$24, $$25)]
- -- ASSIGN |PARTITIONED|
- project ([$$22, $$24, $$25])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- SORT_MERGE_EXCHANGE [$$23(ASC) ] |PARTITIONED|
- order (ASC, $$23)
- -- STABLE_SORT [$$23(ASC)] |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- assign [$$24, $$25] <- [array-star($$22).getField("text"), $$22.getField("text")]
- -- ASSIGN |PARTITIONED|
- project ([$$22, $$23])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$22, $$23] <- [$$p.getField("arrayOrObject"), $$p.getField("id")]
- -- ASSIGN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$p] <- test.ParquetDataset
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.03.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.03.plan
new file mode 100644
index 0000000..0b83b8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.03.plan
@@ -0,0 +1,30 @@
+distribute result [$$20] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$20]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$20] <- [switch-case(true, is-array($$22), $$24, $$25)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$22, $$24, $$25]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SORT_MERGE_EXCHANGE [$$23(ASC) ] |PARTITIONED|
+ order (ASC, $$23) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STABLE_SORT [$$23(ASC)] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ assign [$$24, $$25] <- [array-star($$22).getField("text"), $$22.getField("text")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$22, $$23]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$22, $$23] <- [$$p.getField("arrayOrObject"), $$p.getField("id")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p] <- test.ParquetDataset [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.05.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.05.adm
deleted file mode 100644
index 56dd2ee..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.05.adm
+++ /dev/null
@@ -1,30 +0,0 @@
-distribute result [$$20]
--- DISTRIBUTE_RESULT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$20])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$20] <- [switch-case(TRUE, is-array($$22), $$24, $$25)]
- -- ASSIGN |PARTITIONED|
- project ([$$22, $$24, $$25])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- SORT_MERGE_EXCHANGE [$$23(ASC) ] |PARTITIONED|
- order (ASC, $$23)
- -- STABLE_SORT [$$23(ASC)] |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- assign [$$24, $$25] <- [array-star($$22).getField("text"), $$22.getField("text")]
- -- ASSIGN |PARTITIONED|
- project ([$$22, $$23])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$22, $$23] <- [$$p.getField("arrayOrObject"), $$p.getField("id")]
- -- ASSIGN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$p] <- test.ParquetDataset project ({arrayOrObject:<[{text:any}],{text:any}>,id:any})
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.05.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.05.plan
new file mode 100644
index 0000000..db3e0aa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.05.plan
@@ -0,0 +1,30 @@
+distribute result [$$20] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$20]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$20] <- [switch-case(true, is-array($$22), $$24, $$25)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$22, $$24, $$25]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SORT_MERGE_EXCHANGE [$$23(ASC) ] |PARTITIONED|
+ order (ASC, $$23) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STABLE_SORT [$$23(ASC)] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ assign [$$24, $$25] <- [array-star($$22).getField("text"), $$22.getField("text")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$22, $$23]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$22, $$23] <- [$$p.getField("arrayOrObject"), $$p.getField("id")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p] <- test.ParquetDataset project ({arrayOrObject:<[{text:any}],{text:any}>,id:any}) [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/object-concat/object-concat.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/object-concat/object-concat.3.adm
deleted file mode 100644
index 365218f..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/object-concat/object-concat.3.adm
+++ /dev/null
@@ -1,24 +0,0 @@
-distribute result [$$16]
--- DISTRIBUTE_RESULT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$16])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- SORT_MERGE_EXCHANGE [$$18(ASC) ] |PARTITIONED|
- order (ASC, $$18)
- -- STABLE_SORT [$$18(ASC)] |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$16, $$18])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$16, $$18] <- [object-concat($$p.getField("coordinates"), $$p.getField("user")).getField("name"), $$p.getField("id")]
- -- ASSIGN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$p] <- test.ParquetDataset project ({coordinates:any,id:any,user:any})
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/object-concat/object-concat.3.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/object-concat/object-concat.3.plan
new file mode 100644
index 0000000..60a60ab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/object-concat/object-concat.3.plan
@@ -0,0 +1,24 @@
+distribute result [$$16] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$16]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SORT_MERGE_EXCHANGE [$$18(ASC) ] |PARTITIONED|
+ order (ASC, $$18) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STABLE_SORT [$$18(ASC)] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$16, $$18]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$16, $$18] <- [object-concat($$p.getField("coordinates"), $$p.getField("user")).getField("name"), $$p.getField("id")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p] <- test.ParquetDataset project ({coordinates:any,id:any,user:any}) [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/object-concat/object-concat.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/object-concat/object-concat.5.adm
deleted file mode 100644
index 4533f63..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/object-concat/object-concat.5.adm
+++ /dev/null
@@ -1,28 +0,0 @@
-distribute result [$$15]
--- DISTRIBUTE_RESULT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$15])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$15] <- [object-concat($$18, $$19)]
- -- ASSIGN |PARTITIONED|
- project ([$$18, $$19])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- SORT_MERGE_EXCHANGE [$$17(ASC) ] |PARTITIONED|
- order (ASC, $$17)
- -- STABLE_SORT [$$17(ASC)] |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$18, $$19, $$17])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$19, $$18, $$17] <- [$$p.getField("user"), $$p.getField("coordinates"), $$p.getField("id")]
- -- ASSIGN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$p] <- test.ParquetDataset project ({coordinates:any,id:any,user:any})
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/object-concat/object-concat.5.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/object-concat/object-concat.5.plan
new file mode 100644
index 0000000..1d87deb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/object-concat/object-concat.5.plan
@@ -0,0 +1,28 @@
+distribute result [$$15] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$15]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$15] <- [object-concat($$18, $$19)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$18, $$19]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SORT_MERGE_EXCHANGE [$$17(ASC) ] |PARTITIONED|
+ order (ASC, $$17) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STABLE_SORT [$$17(ASC)] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$18, $$19, $$17]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$19, $$18, $$17] <- [$$p.getField("user"), $$p.getField("coordinates"), $$p.getField("id")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p] <- test.ParquetDataset project ({coordinates:any,id:any,user:any}) [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.02.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.02.adm
deleted file mode 100644
index b020ac9..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.02.adm
+++ /dev/null
@@ -1,50 +0,0 @@
-distribute result [$$48]
--- DISTRIBUTE_RESULT |UNPARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- project ([$$48])
- -- STREAM_PROJECT |UNPARTITIONED|
- assign [$$48] <- [{"$1": $$51}]
- -- ASSIGN |UNPARTITIONED|
- aggregate [$$51] <- [agg-global-sql-sum($$53)]
- -- AGGREGATE |UNPARTITIONED|
- exchange
- -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
- aggregate [$$53] <- [agg-local-sql-sum($$46)]
- -- AGGREGATE |PARTITIONED|
- project ([$$46])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$46] <- [object-length($$p1)]
- -- ASSIGN |PARTITIONED|
- project ([$$p1])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- join (eq($$49, $$50))
- -- HYBRID_HASH_JOIN [$$50][$$49] |PARTITIONED|
- exchange
- -- HASH_PARTITION_EXCHANGE [$$50] |PARTITIONED|
- assign [$$50] <- [$$p1.getField("id")]
- -- ASSIGN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$p1] <- test.ParquetDataset1
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- exchange
- -- HASH_PARTITION_EXCHANGE [$$49] |PARTITIONED|
- project ([$$49])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$49] <- [$$p2.getField("id")]
- -- ASSIGN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$p2] <- test.ParquetDataset2 project ({id:any})
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.02.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.02.plan
new file mode 100644
index 0000000..0a14ee0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.02.plan
@@ -0,0 +1,50 @@
+distribute result [$$48] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ project ([$$48]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- STREAM_PROJECT |UNPARTITIONED|
+ assign [$$48] <- [{"$1": $$51}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- ASSIGN |UNPARTITIONED|
+ aggregate [$$51] <- [agg-global-sql-sum($$53)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- AGGREGATE |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ aggregate [$$53] <- [agg-local-sql-sum($$46)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- AGGREGATE |PARTITIONED|
+ project ([$$46]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$46] <- [object-length($$p1)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$p1]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ join (eq($$49, $$50)) [cardinality: 1000000.0, op-cost: 2000000.0, total-cost: 6000000.0]
+ -- HYBRID_HASH_JOIN [$$50][$$49] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
+ -- HASH_PARTITION_EXCHANGE [$$50] |PARTITIONED|
+ assign [$$50] <- [$$p1.getField("id")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p1] <- test.ParquetDataset1 [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
+ -- HASH_PARTITION_EXCHANGE [$$49] |PARTITIONED|
+ project ([$$49]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$49] <- [$$p2.getField("id")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p2] <- test.ParquetDataset2 project ({id:any}) [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.03.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.03.adm
deleted file mode 100644
index 47348c2..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.03.adm
+++ /dev/null
@@ -1,22 +0,0 @@
-distribute result [$$p1]
--- DISTRIBUTE_RESULT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- distinct ([$$p1])
- -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- order (ASC, $$p1)
- -- STABLE_SORT [$$p1(ASC)] |PARTITIONED|
- exchange
- -- HASH_PARTITION_EXCHANGE [$$p1] |PARTITIONED|
- select (gt($$p1.getField("id"), 10))
- -- STREAM_SELECT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$p1] <- test.ParquetDataset1
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.03.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.03.plan
new file mode 100644
index 0000000..5990f6f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.03.plan
@@ -0,0 +1,22 @@
+distribute result [$$p1] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ distinct ([$$p1]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ order (ASC, $$p1) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STABLE_SORT [$$p1(ASC)] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- HASH_PARTITION_EXCHANGE [$$p1] |PARTITIONED|
+ select (gt($$p1.getField("id"), 10)) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_SELECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p1] <- test.ParquetDataset1 [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.04.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.04.adm
deleted file mode 100644
index d22cd15..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.04.adm
+++ /dev/null
@@ -1,50 +0,0 @@
-distribute result [$$65]
--- DISTRIBUTE_RESULT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$65])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$65] <- [{"text": $$text, "$1": $$68}]
- -- ASSIGN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- group by ([$$text := $$76]) decor ([]) {
- aggregate [$$68] <- [agg-global-sql-sum($$75)]
- -- AGGREGATE |LOCAL|
- nested tuple source
- -- NESTED_TUPLE_SOURCE |LOCAL|
- }
- -- SORT_GROUP_BY[$$76] |PARTITIONED|
- exchange
- -- HASH_PARTITION_EXCHANGE [$$76] |PARTITIONED|
- group by ([$$76 := $$67]) decor ([]) {
- aggregate [$$75] <- [agg-local-sql-sum(array-distinct($$70))]
- -- AGGREGATE |LOCAL|
- nested tuple source
- -- NESTED_TUPLE_SOURCE |LOCAL|
- }
- -- SORT_GROUP_BY[$$67] |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- select (eq(lowercase($$67), "string"))
- -- STREAM_SELECT |PARTITIONED|
- project ([$$70, $$67])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$67] <- [$$ht.getField("text")]
- -- ASSIGN |PARTITIONED|
- unnest $$ht <- scan-collection($$70)
- -- UNNEST |PARTITIONED|
- project ([$$70])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$70] <- [$$p1.getField("entities").getField("hashtags")]
- -- ASSIGN |PARTITIONED|
- select (gt($$p1.getField("id"), 10))
- -- STREAM_SELECT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$p1] <- test.ParquetDataset1 project ({entities:{hashtags:any},id:any})
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.04.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.04.plan
new file mode 100644
index 0000000..96a40a8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.04.plan
@@ -0,0 +1,50 @@
+distribute result [$$65] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$65]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$65] <- [{"text": $$text, "$1": $$68}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ group by ([$$text := $$76]) decor ([]) {
+ aggregate [$$68] <- [agg-global-sql-sum($$75)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- AGGREGATE |LOCAL|
+ nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ } [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SORT_GROUP_BY[$$76] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- HASH_PARTITION_EXCHANGE [$$76] |PARTITIONED|
+ group by ([$$76 := $$67]) decor ([]) {
+ aggregate [$$75] <- [agg-local-sql-sum(array-distinct($$70))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- AGGREGATE |LOCAL|
+ nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ } [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SORT_GROUP_BY[$$67] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ select (eq(lowercase($$67), "string")) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_SELECT |PARTITIONED|
+ project ([$$70, $$67]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$67] <- [$$ht.getField("text")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ unnest $$ht <- scan-collection($$70) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- UNNEST |PARTITIONED|
+ project ([$$70]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$70] <- [$$p1.getField("entities").getField("hashtags")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ select (gt($$p1.getField("id"), 10)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_SELECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p1] <- test.ParquetDataset1 project ({entities:{hashtags:any},id:any}) [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.05.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.05.adm
deleted file mode 100644
index 8ad73f9..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.05.adm
+++ /dev/null
@@ -1,52 +0,0 @@
-distribute result [$$64]
--- DISTRIBUTE_RESULT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$64])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$64] <- [{"text": $$text, "$1": $$67}]
- -- ASSIGN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- group by ([$$text := $$74]) decor ([]) {
- aggregate [$$67] <- [agg-global-sql-sum($$73)]
- -- AGGREGATE |LOCAL|
- nested tuple source
- -- NESTED_TUPLE_SOURCE |LOCAL|
- }
- -- SORT_GROUP_BY[$$74] |PARTITIONED|
- exchange
- -- HASH_PARTITION_EXCHANGE [$$74] |PARTITIONED|
- group by ([$$74 := $$66]) decor ([]) {
- aggregate [$$73] <- [agg-local-sql-sum(sql-sum($$70))]
- -- AGGREGATE |LOCAL|
- nested tuple source
- -- NESTED_TUPLE_SOURCE |LOCAL|
- }
- -- SORT_GROUP_BY[$$66] |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- select (eq(lowercase($$66), "string"))
- -- STREAM_SELECT |PARTITIONED|
- project ([$$70, $$66])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$70, $$66] <- [$$ht.getField("indices"), $$ht.getField("text")]
- -- ASSIGN |PARTITIONED|
- project ([$$ht])
- -- STREAM_PROJECT |PARTITIONED|
- unnest $$ht <- scan-collection($$69)
- -- UNNEST |PARTITIONED|
- project ([$$69])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$69] <- [$$p1.getField("entities").getField("hashtags")]
- -- ASSIGN |PARTITIONED|
- select (gt($$p1.getField("id"), 10))
- -- STREAM_SELECT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$p1] <- test.ParquetDataset1 project ({entities:{hashtags:[{indices:any,text:any}]},id:any})
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.05.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.05.plan
new file mode 100644
index 0000000..f03b6a1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.05.plan
@@ -0,0 +1,52 @@
+distribute result [$$64] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$64]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$64] <- [{"text": $$text, "$1": $$67}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ group by ([$$text := $$74]) decor ([]) {
+ aggregate [$$67] <- [agg-global-sql-sum($$73)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- AGGREGATE |LOCAL|
+ nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ } [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SORT_GROUP_BY[$$74] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- HASH_PARTITION_EXCHANGE [$$74] |PARTITIONED|
+ group by ([$$74 := $$66]) decor ([]) {
+ aggregate [$$73] <- [agg-local-sql-sum(sql-sum($$70))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- AGGREGATE |LOCAL|
+ nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ } [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SORT_GROUP_BY[$$66] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ select (eq(lowercase($$66), "string")) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_SELECT |PARTITIONED|
+ project ([$$70, $$66]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$70, $$66] <- [$$ht.getField("indices"), $$ht.getField("text")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$ht]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ unnest $$ht <- scan-collection($$69) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- UNNEST |PARTITIONED|
+ project ([$$69]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$69] <- [$$p1.getField("entities").getField("hashtags")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ select (gt($$p1.getField("id"), 10)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_SELECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p1] <- test.ParquetDataset1 project ({entities:{hashtags:[{indices:any,text:any}]},id:any}) [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.06.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.06.adm
deleted file mode 100644
index cac9db7..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.06.adm
+++ /dev/null
@@ -1,26 +0,0 @@
-distribute result [$$21]
--- DISTRIBUTE_RESULT |UNPARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- limit 10
- -- STREAM_LIMIT |UNPARTITIONED|
- exchange
- -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
- project ([$$21])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$21] <- [{"display_url": get-item($$22, 0).getField("display_url")}]
- -- ASSIGN |PARTITIONED|
- limit 10
- -- STREAM_LIMIT |PARTITIONED|
- project ([$$22])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$22] <- [$$p.getField("entities").getField("urls")]
- -- ASSIGN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$p] <- test.ParquetDataset1 condition (gt(sql-count($$p.getField("entities").getField("urls")), 10)) limit 10 project ({entities:{urls:any}})
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.06.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.06.plan
new file mode 100644
index 0000000..7a7a890
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.06.plan
@@ -0,0 +1,26 @@
+distribute result [$$21] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ limit 10 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_LIMIT |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ project ([$$21]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$21] <- [{"display_url": get-item($$22, 0).getField("display_url")}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ limit 10 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_LIMIT |PARTITIONED|
+ project ([$$22]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$22] <- [$$p.getField("entities").getField("urls")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p] <- test.ParquetDataset1 condition (gt(sql-count($$p.getField("entities").getField("urls")), 10)) limit 10 project ({entities:{urls:any}}) [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.07.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.07.adm
deleted file mode 100644
index 30d5ac3..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.07.adm
+++ /dev/null
@@ -1,113 +0,0 @@
-distribute result [$$94]
--- DISTRIBUTE_RESULT |UNPARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- limit 10
- -- STREAM_LIMIT |UNPARTITIONED|
- project ([$$94])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$94] <- [{"uname": $$uname, "cnt": $$96}]
- -- ASSIGN |PARTITIONED|
- exchange
- -- SORT_MERGE_EXCHANGE [$$96(DESC) ] |PARTITIONED|
- limit 10
- -- STREAM_LIMIT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- order (topK: 10) (DESC, $$96)
- -- STABLE_SORT [topK: 10] [$$96(DESC)] |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- group by ([$$uname := $$107]) decor ([]) {
- aggregate [$$96] <- [agg-sql-sum($$106)]
- -- AGGREGATE |LOCAL|
- nested tuple source
- -- NESTED_TUPLE_SOURCE |LOCAL|
- }
- -- SORT_GROUP_BY[$$107] |PARTITIONED|
- exchange
- -- HASH_PARTITION_EXCHANGE [$$107] |PARTITIONED|
- group by ([$$107 := $$95]) decor ([]) {
- aggregate [$$106] <- [agg-sql-count(1)]
- -- AGGREGATE |LOCAL|
- nested tuple source
- -- NESTED_TUPLE_SOURCE |LOCAL|
- }
- -- SORT_GROUP_BY[$$95] |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$95])
- -- STREAM_PROJECT |PARTITIONED|
- select ($$85)
- -- STREAM_SELECT |PARTITIONED|
- project ([$$85, $$95])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- group by ([$$105 := $$103]) decor ([$$95]) {
- aggregate [$$85] <- [non-empty-stream()]
- -- AGGREGATE |LOCAL|
- select (not(is-missing($$104)))
- -- STREAM_SELECT |LOCAL|
- nested tuple source
- -- NESTED_TUPLE_SOURCE |LOCAL|
- }
- -- PRE_CLUSTERED_GROUP_BY[$$103] |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- order (ASC, $$103)
- -- STABLE_SORT [$$103(ASC)] |PARTITIONED|
- exchange
- -- HASH_PARTITION_EXCHANGE [$$103] |PARTITIONED|
- project ([$$95, $$104, $$103])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- left outer join (eq($$97, $$81))
- -- HYBRID_HASH_JOIN [$$97][$$81] |PARTITIONED|
- exchange
- -- HASH_PARTITION_EXCHANGE [$$97] |PARTITIONED|
- running-aggregate [$$103] <- [create-query-uid()]
- -- RUNNING_AGGREGATE |PARTITIONED|
- project ([$$95, $$97])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$97] <- [$$ht1.getField("text")]
- -- ASSIGN |PARTITIONED|
- project ([$$95, $$ht1])
- -- STREAM_PROJECT |PARTITIONED|
- unnest $$ht1 <- scan-collection($$98)
- -- UNNEST |PARTITIONED|
- project ([$$98, $$95])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$98, $$95] <- [$$p1.getField("entities").getField("hashtags"), $$p1.getField("user").getField("name")]
- -- ASSIGN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$p1] <- test.ParquetDataset1 project ({entities:{hashtags:[{text:any}]},user:{name:any}})
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- exchange
- -- HASH_PARTITION_EXCHANGE [$$81] |PARTITIONED|
- project ([$$104, $$81])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$104, $$81] <- [TRUE, $$ht2.getField("text")]
- -- ASSIGN |PARTITIONED|
- project ([$$ht2])
- -- STREAM_PROJECT |PARTITIONED|
- unnest $$ht2 <- scan-collection($$99)
- -- UNNEST |PARTITIONED|
- project ([$$99])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$99] <- [$$p2.getField("entities").getField("hashtags")]
- -- ASSIGN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$p2] <- test.ParquetDataset2 project ({entities:{hashtags:[{text:any}]}})
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.07.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.07.plan
new file mode 100644
index 0000000..4818fb4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.07.plan
@@ -0,0 +1,113 @@
+distribute result [$$94] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ limit 10 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_LIMIT |UNPARTITIONED|
+ project ([$$94]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$94] <- [{"uname": $$uname, "cnt": $$96}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- SORT_MERGE_EXCHANGE [$$96(DESC) ] |PARTITIONED|
+ limit 10 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_LIMIT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ order (topK: 10) (DESC, $$96) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STABLE_SORT [topK: 10] [$$96(DESC)] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ group by ([$$uname := $$107]) decor ([]) {
+ aggregate [$$96] <- [agg-sql-sum($$106)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- AGGREGATE |LOCAL|
+ nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- SORT_GROUP_BY[$$107] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- HASH_PARTITION_EXCHANGE [$$107] |PARTITIONED|
+ group by ([$$107 := $$95]) decor ([]) {
+ aggregate [$$106] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- AGGREGATE |LOCAL|
+ nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- SORT_GROUP_BY[$$95] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$95]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ select ($$85) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_SELECT |PARTITIONED|
+ project ([$$85, $$95]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ group by ([$$105 := $$103]) decor ([$$95]) {
+ aggregate [$$85] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- AGGREGATE |LOCAL|
+ select (not(is-missing($$104))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_SELECT |LOCAL|
+ nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- PRE_CLUSTERED_GROUP_BY[$$103] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ order (ASC, $$103) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STABLE_SORT [$$103(ASC)] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- HASH_PARTITION_EXCHANGE [$$103] |PARTITIONED|
+ project ([$$95, $$104, $$103]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ left outer join (eq($$97, $$81)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- HYBRID_HASH_JOIN [$$97][$$81] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- HASH_PARTITION_EXCHANGE [$$97] |PARTITIONED|
+ running-aggregate [$$103] <- [create-query-uid()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- RUNNING_AGGREGATE |PARTITIONED|
+ project ([$$95, $$97]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$97] <- [$$ht1.getField("text")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$95, $$ht1]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ unnest $$ht1 <- scan-collection($$98) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- UNNEST |PARTITIONED|
+ project ([$$98, $$95]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$98, $$95] <- [$$p1.getField("entities").getField("hashtags"), $$p1.getField("user").getField("name")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p1] <- test.ParquetDataset1 project ({entities:{hashtags:[{text:any}]},user:{name:any}}) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- HASH_PARTITION_EXCHANGE [$$81] |PARTITIONED|
+ project ([$$104, $$81]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$104, $$81] <- [true, $$ht2.getField("text")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$ht2]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ unnest $$ht2 <- scan-collection($$99) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- UNNEST |PARTITIONED|
+ project ([$$99]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$99] <- [$$p2.getField("entities").getField("hashtags")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p2] <- test.ParquetDataset2 project ({entities:{hashtags:[{text:any}]}}) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/query-with-limit-plan/result.001.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/query-with-limit-plan/result.001.adm
deleted file mode 100644
index c7927c0..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/query-with-limit-plan/result.001.adm
+++ /dev/null
@@ -1,22 +0,0 @@
-distribute result [$$13]
--- DISTRIBUTE_RESULT |UNPARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- limit 3
- -- STREAM_LIMIT |UNPARTITIONED|
- exchange
- -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
- project ([$$13])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$13] <- [{"test": $$test}]
- -- ASSIGN |PARTITIONED|
- limit 3
- -- STREAM_LIMIT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$test] <- test.test limit 3
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/query-with-limit-plan/result.001.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/query-with-limit-plan/result.001.plan
new file mode 100644
index 0000000..db967ea
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/query-with-limit-plan/result.001.plan
@@ -0,0 +1,22 @@
+distribute result [$$13] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ limit 3 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_LIMIT |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ project ([$$13]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$13] <- [{"test": $$test}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ limit 3 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_LIMIT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$test] <- test.test limit 3 [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/deterministic/deterministic.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/deterministic/deterministic.4.adm
deleted file mode 100644
index 9b1b82f..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/deterministic/deterministic.4.adm
+++ /dev/null
@@ -1,8 +0,0 @@
-distribute result [$$1]
--- DISTRIBUTE_RESULT |UNPARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- assign [$$1] <- [{"default": getCapital_default("United States"), "deterministic": getCapital_deterministic("United States"), "not_deterministic": getCapital_not_deterministic("United States")}]
- -- ASSIGN |UNPARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/deterministic/deterministic.4.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/deterministic/deterministic.4.plan
new file mode 100644
index 0000000..08796cb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/deterministic/deterministic.4.plan
@@ -0,0 +1,8 @@
+distribute result [$$1] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ assign [$$1] <- [{"default": getCapital_default("United States"), "deterministic": getCapital_deterministic("United States"), "not_deterministic": getCapital_not_deterministic("United States")}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |UNPARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/type_validation/type_validation.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/type_validation/type_validation.1.adm
index f164f97..121262f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/type_validation/type_validation.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/type_validation/type_validation.1.adm
@@ -1 +1 @@
-"907 9.07 \"907\" 9.07 TRUE date: { 2013-01-01 } datetime: { 1989-09-07T12:13:14.039 } "
+"907 9.07 \"907\" 9.07 true date: { 2013-01-01 } datetime: { 1989-09-07T12:13:14.039 } "
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/global-aggregate/q06/q06.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/global-aggregate/q06/q06.3.adm
new file mode 100644
index 0000000..21c2ffa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/global-aggregate/q06/q06.3.adm
@@ -0,0 +1,10 @@
+{ "count": 0 }
+{ "count": 0 }
+{ "count": 0 }
+{ "count": 0 }
+{ "count": 0 }
+{ "count": 0 }
+{ "count": 0 }
+{ "count": 0 }
+{ "count": 0 }
+{ "count": 0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/group-by/sugar-01-negative/core-01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/group-by/sugar-01-negative/core-01.1.adm
new file mode 100644
index 0000000..cc15b26
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/group-by/sugar-01-negative/core-01.1.adm
@@ -0,0 +1 @@
+{ "avgpay": null, "workers": [ { "name": "Bill", "salary": 2000 }, { "name": "Fred", "salary": 3000 } ], "deptno": "K55" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/join/ASTERIXDB-3502/ASTERIXDB-3502.03.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/ASTERIXDB-3502/ASTERIXDB-3502.03.adm
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/ASTERIXDB-3502/ASTERIXDB-3502.03.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.03.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.03.adm
new file mode 100644
index 0000000..322cb10
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.03.adm
@@ -0,0 +1,58 @@
+{ "n_nationkey": 1, "s_nationkey": 1, "c_nationkey": 1 }
+{ "n_nationkey": 1, "s_nationkey": 1, "c_nationkey": 1 }
+{ "n_nationkey": 1, "s_nationkey": 1, "c_nationkey": 1 }
+{ "n_nationkey": 1, "s_nationkey": 1, "c_nationkey": 1 }
+{ "n_nationkey": 1, "s_nationkey": 1, "c_nationkey": 1 }
+{ "n_nationkey": 1, "s_nationkey": 1, "c_nationkey": 1 }
+{ "n_nationkey": 1, "s_nationkey": 1, "c_nationkey": 1 }
+{ "n_nationkey": 5, "s_nationkey": 5, "c_nationkey": 5 }
+{ "n_nationkey": 5, "s_nationkey": 5, "c_nationkey": 5 }
+{ "n_nationkey": 5, "s_nationkey": 5, "c_nationkey": 5 }
+{ "n_nationkey": 5, "s_nationkey": 5, "c_nationkey": 5 }
+{ "n_nationkey": 5, "s_nationkey": 5, "c_nationkey": 5 }
+{ "n_nationkey": 5, "s_nationkey": 5, "c_nationkey": 5 }
+{ "n_nationkey": 10, "s_nationkey": 10, "c_nationkey": 10 }
+{ "n_nationkey": 10, "s_nationkey": 10, "c_nationkey": 10 }
+{ "n_nationkey": 10, "s_nationkey": 10, "c_nationkey": 10 }
+{ "n_nationkey": 10, "s_nationkey": 10, "c_nationkey": 10 }
+{ "n_nationkey": 10, "s_nationkey": 10, "c_nationkey": 10 }
+{ "n_nationkey": 10, "s_nationkey": 10, "c_nationkey": 10 }
+{ "n_nationkey": 10, "s_nationkey": 10, "c_nationkey": 10 }
+{ "n_nationkey": 10, "s_nationkey": 10, "c_nationkey": 10 }
+{ "n_nationkey": 11, "s_nationkey": 11, "c_nationkey": 11 }
+{ "n_nationkey": 11, "s_nationkey": 11, "c_nationkey": 11 }
+{ "n_nationkey": 11, "s_nationkey": 11, "c_nationkey": 11 }
+{ "n_nationkey": 11, "s_nationkey": 11, "c_nationkey": 11 }
+{ "n_nationkey": 11, "s_nationkey": 11, "c_nationkey": 11 }
+{ "n_nationkey": 14, "s_nationkey": 14, "c_nationkey": 14 }
+{ "n_nationkey": 14, "s_nationkey": 14, "c_nationkey": 14 }
+{ "n_nationkey": 15, "s_nationkey": 15, "c_nationkey": 15 }
+{ "n_nationkey": 15, "s_nationkey": 15, "c_nationkey": 15 }
+{ "n_nationkey": 15, "s_nationkey": 15, "c_nationkey": 15 }
+{ "n_nationkey": 15, "s_nationkey": 15, "c_nationkey": 15 }
+{ "n_nationkey": 15, "s_nationkey": 15, "c_nationkey": 15 }
+{ "n_nationkey": 15, "s_nationkey": 15, "c_nationkey": 15 }
+{ "n_nationkey": 15, "s_nationkey": 15, "c_nationkey": 15 }
+{ "n_nationkey": 15, "s_nationkey": 15, "c_nationkey": 15 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 23, "s_nationkey": 23, "c_nationkey": 23 }
+{ "n_nationkey": 23, "s_nationkey": 23, "c_nationkey": 23 }
+{ "n_nationkey": 23, "s_nationkey": 23, "c_nationkey": 23 }
+{ "n_nationkey": 23, "s_nationkey": 23, "c_nationkey": 23 }
+{ "n_nationkey": 23, "s_nationkey": 23, "c_nationkey": 23 }
+{ "n_nationkey": 24, "s_nationkey": 24, "c_nationkey": 24 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.04.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.04.plan
new file mode 100644
index 0000000..99f36c1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.04.plan
@@ -0,0 +1,64 @@
+distribute result [$$48] [cardinality: 2.1, op-cost: 0.0, total-cost: 1.1E7]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 1.1E7]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$48]) [cardinality: 2.1, op-cost: 0.0, total-cost: 1.1E7]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$48] <- [{"n_nationkey": $$55, "s_nationkey": $$53, "c_nationkey": $$52}] [cardinality: 2.1, op-cost: 0.0, total-cost: 1.1E7]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 1.1E7]
+ -- SORT_MERGE_EXCHANGE [$$55(ASC), $$53(ASC), $$52(ASC) ] |PARTITIONED|
+ order (ASC, $$55) (ASC, $$53) (ASC, $$52) [cardinality: 2.1, op-cost: 0.0, total-cost: 1.1E7]
+ -- STABLE_SORT [$$55(ASC), $$53(ASC), $$52(ASC)] |PARTITIONED|
+ exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 1.1E7]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ join (eq($$52, $$55)) [cardinality: 2.1, op-cost: 2000000.0, total-cost: 1.1E7]
+ -- HYBRID_HASH_JOIN [$$55][$$52] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 7000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ join (eq($$53, $$55)) [cardinality: 1000000.0, op-cost: 2000000.0, total-cost: 6000000.0]
+ -- HYBRID_HASH_JOIN [$$55][$$53] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$55]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$55, $$n] <- tpch.Nation [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
+ -- HASH_PARTITION_EXCHANGE [$$53] |PARTITIONED|
+ project ([$$53]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$53] <- [$$s.getField(3)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$s]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$56, $$s] <- tpch.Supplier [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
+ -- HASH_PARTITION_EXCHANGE [$$52] |PARTITIONED|
+ project ([$$52]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$52] <- [$$c.getField(3)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$c]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$57, $$c] <- tpch.Customer [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.05.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.05.adm
new file mode 100644
index 0000000..92abec0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.05.adm
@@ -0,0 +1,3 @@
+{ "o_orderkey": 2, "l_orderkey": 2, "l_suppkey": 2 }
+{ "o_orderkey": 5, "l_orderkey": 5, "l_suppkey": 5 }
+{ "o_orderkey": 6, "l_orderkey": 6, "l_suppkey": 6 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.06.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.06.plan
new file mode 100644
index 0000000..3148a70
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.06.plan
@@ -0,0 +1,50 @@
+distribute result [$$34] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$34]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$34] <- [{"o_orderkey": $$41, "l_orderkey": $$42, "l_suppkey": $$40}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- SORT_MERGE_EXCHANGE [$$41(ASC), $$42(ASC), $$40(ASC) ] |PARTITIONED|
+ order (ASC, $$41) (ASC, $$42) (ASC, $$40) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- STABLE_SORT [$$41(ASC), $$42(ASC), $$40(ASC)] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$41, $$42, $$40]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ join (and(eq($$41, $$42), eq($$47, $$40))) [cardinality: 1000000.0, op-cost: 2000000.0, total-cost: 6000000.0]
+ -- HYBRID_HASH_JOIN [$$41, $$47][$$42, $$40] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ assign [$$47] <- [$$41] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$41]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$41, $$o] <- tpch.Orders [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
+ -- HASH_PARTITION_EXCHANGE [$$42] |PARTITIONED|
+ project ([$$42, $$40]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$40] <- [$$l.getField(2)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$42, $$l]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$42, $$43, $$l] <- tpch.LineItem [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.07.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.07.adm
new file mode 100644
index 0000000..55acc86
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.07.adm
@@ -0,0 +1,1500 @@
+{ "o_orderkey": 1 }
+{ "o_orderkey": 2, "l_orderkey": 2, "l_suppkey": 2 }
+{ "o_orderkey": 3 }
+{ "o_orderkey": 4 }
+{ "o_orderkey": 5, "l_orderkey": 5, "l_suppkey": 5 }
+{ "o_orderkey": 6, "l_orderkey": 6, "l_suppkey": 6 }
+{ "o_orderkey": 7 }
+{ "o_orderkey": 32 }
+{ "o_orderkey": 33 }
+{ "o_orderkey": 34 }
+{ "o_orderkey": 35 }
+{ "o_orderkey": 36 }
+{ "o_orderkey": 37 }
+{ "o_orderkey": 38 }
+{ "o_orderkey": 39 }
+{ "o_orderkey": 64 }
+{ "o_orderkey": 65 }
+{ "o_orderkey": 66 }
+{ "o_orderkey": 67 }
+{ "o_orderkey": 68 }
+{ "o_orderkey": 69 }
+{ "o_orderkey": 70 }
+{ "o_orderkey": 71 }
+{ "o_orderkey": 96 }
+{ "o_orderkey": 97 }
+{ "o_orderkey": 98 }
+{ "o_orderkey": 99 }
+{ "o_orderkey": 100 }
+{ "o_orderkey": 101 }
+{ "o_orderkey": 102 }
+{ "o_orderkey": 103 }
+{ "o_orderkey": 128 }
+{ "o_orderkey": 129 }
+{ "o_orderkey": 130 }
+{ "o_orderkey": 131 }
+{ "o_orderkey": 132 }
+{ "o_orderkey": 133 }
+{ "o_orderkey": 134 }
+{ "o_orderkey": 135 }
+{ "o_orderkey": 160 }
+{ "o_orderkey": 161 }
+{ "o_orderkey": 162 }
+{ "o_orderkey": 163 }
+{ "o_orderkey": 164 }
+{ "o_orderkey": 165 }
+{ "o_orderkey": 166 }
+{ "o_orderkey": 167 }
+{ "o_orderkey": 192 }
+{ "o_orderkey": 193 }
+{ "o_orderkey": 194 }
+{ "o_orderkey": 195 }
+{ "o_orderkey": 196 }
+{ "o_orderkey": 197 }
+{ "o_orderkey": 198 }
+{ "o_orderkey": 199 }
+{ "o_orderkey": 224 }
+{ "o_orderkey": 225 }
+{ "o_orderkey": 226 }
+{ "o_orderkey": 227 }
+{ "o_orderkey": 228 }
+{ "o_orderkey": 229 }
+{ "o_orderkey": 230 }
+{ "o_orderkey": 231 }
+{ "o_orderkey": 256 }
+{ "o_orderkey": 257 }
+{ "o_orderkey": 258 }
+{ "o_orderkey": 259 }
+{ "o_orderkey": 260 }
+{ "o_orderkey": 261 }
+{ "o_orderkey": 262 }
+{ "o_orderkey": 263 }
+{ "o_orderkey": 288 }
+{ "o_orderkey": 289 }
+{ "o_orderkey": 290 }
+{ "o_orderkey": 291 }
+{ "o_orderkey": 292 }
+{ "o_orderkey": 293 }
+{ "o_orderkey": 294 }
+{ "o_orderkey": 295 }
+{ "o_orderkey": 320 }
+{ "o_orderkey": 321 }
+{ "o_orderkey": 322 }
+{ "o_orderkey": 323 }
+{ "o_orderkey": 324 }
+{ "o_orderkey": 325 }
+{ "o_orderkey": 326 }
+{ "o_orderkey": 327 }
+{ "o_orderkey": 352 }
+{ "o_orderkey": 353 }
+{ "o_orderkey": 354 }
+{ "o_orderkey": 355 }
+{ "o_orderkey": 356 }
+{ "o_orderkey": 357 }
+{ "o_orderkey": 358 }
+{ "o_orderkey": 359 }
+{ "o_orderkey": 384 }
+{ "o_orderkey": 385 }
+{ "o_orderkey": 386 }
+{ "o_orderkey": 387 }
+{ "o_orderkey": 388 }
+{ "o_orderkey": 389 }
+{ "o_orderkey": 390 }
+{ "o_orderkey": 391 }
+{ "o_orderkey": 416 }
+{ "o_orderkey": 417 }
+{ "o_orderkey": 418 }
+{ "o_orderkey": 419 }
+{ "o_orderkey": 420 }
+{ "o_orderkey": 421 }
+{ "o_orderkey": 422 }
+{ "o_orderkey": 423 }
+{ "o_orderkey": 448 }
+{ "o_orderkey": 449 }
+{ "o_orderkey": 450 }
+{ "o_orderkey": 451 }
+{ "o_orderkey": 452 }
+{ "o_orderkey": 453 }
+{ "o_orderkey": 454 }
+{ "o_orderkey": 455 }
+{ "o_orderkey": 480 }
+{ "o_orderkey": 481 }
+{ "o_orderkey": 482 }
+{ "o_orderkey": 483 }
+{ "o_orderkey": 484 }
+{ "o_orderkey": 485 }
+{ "o_orderkey": 486 }
+{ "o_orderkey": 487 }
+{ "o_orderkey": 512 }
+{ "o_orderkey": 513 }
+{ "o_orderkey": 514 }
+{ "o_orderkey": 515 }
+{ "o_orderkey": 516 }
+{ "o_orderkey": 517 }
+{ "o_orderkey": 518 }
+{ "o_orderkey": 519 }
+{ "o_orderkey": 544 }
+{ "o_orderkey": 545 }
+{ "o_orderkey": 546 }
+{ "o_orderkey": 547 }
+{ "o_orderkey": 548 }
+{ "o_orderkey": 549 }
+{ "o_orderkey": 550 }
+{ "o_orderkey": 551 }
+{ "o_orderkey": 576 }
+{ "o_orderkey": 577 }
+{ "o_orderkey": 578 }
+{ "o_orderkey": 579 }
+{ "o_orderkey": 580 }
+{ "o_orderkey": 581 }
+{ "o_orderkey": 582 }
+{ "o_orderkey": 583 }
+{ "o_orderkey": 608 }
+{ "o_orderkey": 609 }
+{ "o_orderkey": 610 }
+{ "o_orderkey": 611 }
+{ "o_orderkey": 612 }
+{ "o_orderkey": 613 }
+{ "o_orderkey": 614 }
+{ "o_orderkey": 615 }
+{ "o_orderkey": 640 }
+{ "o_orderkey": 641 }
+{ "o_orderkey": 642 }
+{ "o_orderkey": 643 }
+{ "o_orderkey": 644 }
+{ "o_orderkey": 645 }
+{ "o_orderkey": 646 }
+{ "o_orderkey": 647 }
+{ "o_orderkey": 672 }
+{ "o_orderkey": 673 }
+{ "o_orderkey": 674 }
+{ "o_orderkey": 675 }
+{ "o_orderkey": 676 }
+{ "o_orderkey": 677 }
+{ "o_orderkey": 678 }
+{ "o_orderkey": 679 }
+{ "o_orderkey": 704 }
+{ "o_orderkey": 705 }
+{ "o_orderkey": 706 }
+{ "o_orderkey": 707 }
+{ "o_orderkey": 708 }
+{ "o_orderkey": 709 }
+{ "o_orderkey": 710 }
+{ "o_orderkey": 711 }
+{ "o_orderkey": 736 }
+{ "o_orderkey": 737 }
+{ "o_orderkey": 738 }
+{ "o_orderkey": 739 }
+{ "o_orderkey": 740 }
+{ "o_orderkey": 741 }
+{ "o_orderkey": 742 }
+{ "o_orderkey": 743 }
+{ "o_orderkey": 768 }
+{ "o_orderkey": 769 }
+{ "o_orderkey": 770 }
+{ "o_orderkey": 771 }
+{ "o_orderkey": 772 }
+{ "o_orderkey": 773 }
+{ "o_orderkey": 774 }
+{ "o_orderkey": 775 }
+{ "o_orderkey": 800 }
+{ "o_orderkey": 801 }
+{ "o_orderkey": 802 }
+{ "o_orderkey": 803 }
+{ "o_orderkey": 804 }
+{ "o_orderkey": 805 }
+{ "o_orderkey": 806 }
+{ "o_orderkey": 807 }
+{ "o_orderkey": 832 }
+{ "o_orderkey": 833 }
+{ "o_orderkey": 834 }
+{ "o_orderkey": 835 }
+{ "o_orderkey": 836 }
+{ "o_orderkey": 837 }
+{ "o_orderkey": 838 }
+{ "o_orderkey": 839 }
+{ "o_orderkey": 864 }
+{ "o_orderkey": 865 }
+{ "o_orderkey": 866 }
+{ "o_orderkey": 867 }
+{ "o_orderkey": 868 }
+{ "o_orderkey": 869 }
+{ "o_orderkey": 870 }
+{ "o_orderkey": 871 }
+{ "o_orderkey": 896 }
+{ "o_orderkey": 897 }
+{ "o_orderkey": 898 }
+{ "o_orderkey": 899 }
+{ "o_orderkey": 900 }
+{ "o_orderkey": 901 }
+{ "o_orderkey": 902 }
+{ "o_orderkey": 903 }
+{ "o_orderkey": 928 }
+{ "o_orderkey": 929 }
+{ "o_orderkey": 930 }
+{ "o_orderkey": 931 }
+{ "o_orderkey": 932 }
+{ "o_orderkey": 933 }
+{ "o_orderkey": 934 }
+{ "o_orderkey": 935 }
+{ "o_orderkey": 960 }
+{ "o_orderkey": 961 }
+{ "o_orderkey": 962 }
+{ "o_orderkey": 963 }
+{ "o_orderkey": 964 }
+{ "o_orderkey": 965 }
+{ "o_orderkey": 966 }
+{ "o_orderkey": 967 }
+{ "o_orderkey": 992 }
+{ "o_orderkey": 993 }
+{ "o_orderkey": 994 }
+{ "o_orderkey": 995 }
+{ "o_orderkey": 996 }
+{ "o_orderkey": 997 }
+{ "o_orderkey": 998 }
+{ "o_orderkey": 999 }
+{ "o_orderkey": 1024 }
+{ "o_orderkey": 1025 }
+{ "o_orderkey": 1026 }
+{ "o_orderkey": 1027 }
+{ "o_orderkey": 1028 }
+{ "o_orderkey": 1029 }
+{ "o_orderkey": 1030 }
+{ "o_orderkey": 1031 }
+{ "o_orderkey": 1056 }
+{ "o_orderkey": 1057 }
+{ "o_orderkey": 1058 }
+{ "o_orderkey": 1059 }
+{ "o_orderkey": 1060 }
+{ "o_orderkey": 1061 }
+{ "o_orderkey": 1062 }
+{ "o_orderkey": 1063 }
+{ "o_orderkey": 1088 }
+{ "o_orderkey": 1089 }
+{ "o_orderkey": 1090 }
+{ "o_orderkey": 1091 }
+{ "o_orderkey": 1092 }
+{ "o_orderkey": 1093 }
+{ "o_orderkey": 1094 }
+{ "o_orderkey": 1095 }
+{ "o_orderkey": 1120 }
+{ "o_orderkey": 1121 }
+{ "o_orderkey": 1122 }
+{ "o_orderkey": 1123 }
+{ "o_orderkey": 1124 }
+{ "o_orderkey": 1125 }
+{ "o_orderkey": 1126 }
+{ "o_orderkey": 1127 }
+{ "o_orderkey": 1152 }
+{ "o_orderkey": 1153 }
+{ "o_orderkey": 1154 }
+{ "o_orderkey": 1155 }
+{ "o_orderkey": 1156 }
+{ "o_orderkey": 1157 }
+{ "o_orderkey": 1158 }
+{ "o_orderkey": 1159 }
+{ "o_orderkey": 1184 }
+{ "o_orderkey": 1185 }
+{ "o_orderkey": 1186 }
+{ "o_orderkey": 1187 }
+{ "o_orderkey": 1188 }
+{ "o_orderkey": 1189 }
+{ "o_orderkey": 1190 }
+{ "o_orderkey": 1191 }
+{ "o_orderkey": 1216 }
+{ "o_orderkey": 1217 }
+{ "o_orderkey": 1218 }
+{ "o_orderkey": 1219 }
+{ "o_orderkey": 1220 }
+{ "o_orderkey": 1221 }
+{ "o_orderkey": 1222 }
+{ "o_orderkey": 1223 }
+{ "o_orderkey": 1248 }
+{ "o_orderkey": 1249 }
+{ "o_orderkey": 1250 }
+{ "o_orderkey": 1251 }
+{ "o_orderkey": 1252 }
+{ "o_orderkey": 1253 }
+{ "o_orderkey": 1254 }
+{ "o_orderkey": 1255 }
+{ "o_orderkey": 1280 }
+{ "o_orderkey": 1281 }
+{ "o_orderkey": 1282 }
+{ "o_orderkey": 1283 }
+{ "o_orderkey": 1284 }
+{ "o_orderkey": 1285 }
+{ "o_orderkey": 1286 }
+{ "o_orderkey": 1287 }
+{ "o_orderkey": 1312 }
+{ "o_orderkey": 1313 }
+{ "o_orderkey": 1314 }
+{ "o_orderkey": 1315 }
+{ "o_orderkey": 1316 }
+{ "o_orderkey": 1317 }
+{ "o_orderkey": 1318 }
+{ "o_orderkey": 1319 }
+{ "o_orderkey": 1344 }
+{ "o_orderkey": 1345 }
+{ "o_orderkey": 1346 }
+{ "o_orderkey": 1347 }
+{ "o_orderkey": 1348 }
+{ "o_orderkey": 1349 }
+{ "o_orderkey": 1350 }
+{ "o_orderkey": 1351 }
+{ "o_orderkey": 1376 }
+{ "o_orderkey": 1377 }
+{ "o_orderkey": 1378 }
+{ "o_orderkey": 1379 }
+{ "o_orderkey": 1380 }
+{ "o_orderkey": 1381 }
+{ "o_orderkey": 1382 }
+{ "o_orderkey": 1383 }
+{ "o_orderkey": 1408 }
+{ "o_orderkey": 1409 }
+{ "o_orderkey": 1410 }
+{ "o_orderkey": 1411 }
+{ "o_orderkey": 1412 }
+{ "o_orderkey": 1413 }
+{ "o_orderkey": 1414 }
+{ "o_orderkey": 1415 }
+{ "o_orderkey": 1440 }
+{ "o_orderkey": 1441 }
+{ "o_orderkey": 1442 }
+{ "o_orderkey": 1443 }
+{ "o_orderkey": 1444 }
+{ "o_orderkey": 1445 }
+{ "o_orderkey": 1446 }
+{ "o_orderkey": 1447 }
+{ "o_orderkey": 1472 }
+{ "o_orderkey": 1473 }
+{ "o_orderkey": 1474 }
+{ "o_orderkey": 1475 }
+{ "o_orderkey": 1476 }
+{ "o_orderkey": 1477 }
+{ "o_orderkey": 1478 }
+{ "o_orderkey": 1479 }
+{ "o_orderkey": 1504 }
+{ "o_orderkey": 1505 }
+{ "o_orderkey": 1506 }
+{ "o_orderkey": 1507 }
+{ "o_orderkey": 1508 }
+{ "o_orderkey": 1509 }
+{ "o_orderkey": 1510 }
+{ "o_orderkey": 1511 }
+{ "o_orderkey": 1536 }
+{ "o_orderkey": 1537 }
+{ "o_orderkey": 1538 }
+{ "o_orderkey": 1539 }
+{ "o_orderkey": 1540 }
+{ "o_orderkey": 1541 }
+{ "o_orderkey": 1542 }
+{ "o_orderkey": 1543 }
+{ "o_orderkey": 1568 }
+{ "o_orderkey": 1569 }
+{ "o_orderkey": 1570 }
+{ "o_orderkey": 1571 }
+{ "o_orderkey": 1572 }
+{ "o_orderkey": 1573 }
+{ "o_orderkey": 1574 }
+{ "o_orderkey": 1575 }
+{ "o_orderkey": 1600 }
+{ "o_orderkey": 1601 }
+{ "o_orderkey": 1602 }
+{ "o_orderkey": 1603 }
+{ "o_orderkey": 1604 }
+{ "o_orderkey": 1605 }
+{ "o_orderkey": 1606 }
+{ "o_orderkey": 1607 }
+{ "o_orderkey": 1632 }
+{ "o_orderkey": 1633 }
+{ "o_orderkey": 1634 }
+{ "o_orderkey": 1635 }
+{ "o_orderkey": 1636 }
+{ "o_orderkey": 1637 }
+{ "o_orderkey": 1638 }
+{ "o_orderkey": 1639 }
+{ "o_orderkey": 1664 }
+{ "o_orderkey": 1665 }
+{ "o_orderkey": 1666 }
+{ "o_orderkey": 1667 }
+{ "o_orderkey": 1668 }
+{ "o_orderkey": 1669 }
+{ "o_orderkey": 1670 }
+{ "o_orderkey": 1671 }
+{ "o_orderkey": 1696 }
+{ "o_orderkey": 1697 }
+{ "o_orderkey": 1698 }
+{ "o_orderkey": 1699 }
+{ "o_orderkey": 1700 }
+{ "o_orderkey": 1701 }
+{ "o_orderkey": 1702 }
+{ "o_orderkey": 1703 }
+{ "o_orderkey": 1728 }
+{ "o_orderkey": 1729 }
+{ "o_orderkey": 1730 }
+{ "o_orderkey": 1731 }
+{ "o_orderkey": 1732 }
+{ "o_orderkey": 1733 }
+{ "o_orderkey": 1734 }
+{ "o_orderkey": 1735 }
+{ "o_orderkey": 1760 }
+{ "o_orderkey": 1761 }
+{ "o_orderkey": 1762 }
+{ "o_orderkey": 1763 }
+{ "o_orderkey": 1764 }
+{ "o_orderkey": 1765 }
+{ "o_orderkey": 1766 }
+{ "o_orderkey": 1767 }
+{ "o_orderkey": 1792 }
+{ "o_orderkey": 1793 }
+{ "o_orderkey": 1794 }
+{ "o_orderkey": 1795 }
+{ "o_orderkey": 1796 }
+{ "o_orderkey": 1797 }
+{ "o_orderkey": 1798 }
+{ "o_orderkey": 1799 }
+{ "o_orderkey": 1824 }
+{ "o_orderkey": 1825 }
+{ "o_orderkey": 1826 }
+{ "o_orderkey": 1827 }
+{ "o_orderkey": 1828 }
+{ "o_orderkey": 1829 }
+{ "o_orderkey": 1830 }
+{ "o_orderkey": 1831 }
+{ "o_orderkey": 1856 }
+{ "o_orderkey": 1857 }
+{ "o_orderkey": 1858 }
+{ "o_orderkey": 1859 }
+{ "o_orderkey": 1860 }
+{ "o_orderkey": 1861 }
+{ "o_orderkey": 1862 }
+{ "o_orderkey": 1863 }
+{ "o_orderkey": 1888 }
+{ "o_orderkey": 1889 }
+{ "o_orderkey": 1890 }
+{ "o_orderkey": 1891 }
+{ "o_orderkey": 1892 }
+{ "o_orderkey": 1893 }
+{ "o_orderkey": 1894 }
+{ "o_orderkey": 1895 }
+{ "o_orderkey": 1920 }
+{ "o_orderkey": 1921 }
+{ "o_orderkey": 1922 }
+{ "o_orderkey": 1923 }
+{ "o_orderkey": 1924 }
+{ "o_orderkey": 1925 }
+{ "o_orderkey": 1926 }
+{ "o_orderkey": 1927 }
+{ "o_orderkey": 1952 }
+{ "o_orderkey": 1953 }
+{ "o_orderkey": 1954 }
+{ "o_orderkey": 1955 }
+{ "o_orderkey": 1956 }
+{ "o_orderkey": 1957 }
+{ "o_orderkey": 1958 }
+{ "o_orderkey": 1959 }
+{ "o_orderkey": 1984 }
+{ "o_orderkey": 1985 }
+{ "o_orderkey": 1986 }
+{ "o_orderkey": 1987 }
+{ "o_orderkey": 1988 }
+{ "o_orderkey": 1989 }
+{ "o_orderkey": 1990 }
+{ "o_orderkey": 1991 }
+{ "o_orderkey": 2016 }
+{ "o_orderkey": 2017 }
+{ "o_orderkey": 2018 }
+{ "o_orderkey": 2019 }
+{ "o_orderkey": 2020 }
+{ "o_orderkey": 2021 }
+{ "o_orderkey": 2022 }
+{ "o_orderkey": 2023 }
+{ "o_orderkey": 2048 }
+{ "o_orderkey": 2049 }
+{ "o_orderkey": 2050 }
+{ "o_orderkey": 2051 }
+{ "o_orderkey": 2052 }
+{ "o_orderkey": 2053 }
+{ "o_orderkey": 2054 }
+{ "o_orderkey": 2055 }
+{ "o_orderkey": 2080 }
+{ "o_orderkey": 2081 }
+{ "o_orderkey": 2082 }
+{ "o_orderkey": 2083 }
+{ "o_orderkey": 2084 }
+{ "o_orderkey": 2085 }
+{ "o_orderkey": 2086 }
+{ "o_orderkey": 2087 }
+{ "o_orderkey": 2112 }
+{ "o_orderkey": 2113 }
+{ "o_orderkey": 2114 }
+{ "o_orderkey": 2115 }
+{ "o_orderkey": 2116 }
+{ "o_orderkey": 2117 }
+{ "o_orderkey": 2118 }
+{ "o_orderkey": 2119 }
+{ "o_orderkey": 2144 }
+{ "o_orderkey": 2145 }
+{ "o_orderkey": 2146 }
+{ "o_orderkey": 2147 }
+{ "o_orderkey": 2148 }
+{ "o_orderkey": 2149 }
+{ "o_orderkey": 2150 }
+{ "o_orderkey": 2151 }
+{ "o_orderkey": 2176 }
+{ "o_orderkey": 2177 }
+{ "o_orderkey": 2178 }
+{ "o_orderkey": 2179 }
+{ "o_orderkey": 2180 }
+{ "o_orderkey": 2181 }
+{ "o_orderkey": 2182 }
+{ "o_orderkey": 2183 }
+{ "o_orderkey": 2208 }
+{ "o_orderkey": 2209 }
+{ "o_orderkey": 2210 }
+{ "o_orderkey": 2211 }
+{ "o_orderkey": 2212 }
+{ "o_orderkey": 2213 }
+{ "o_orderkey": 2214 }
+{ "o_orderkey": 2215 }
+{ "o_orderkey": 2240 }
+{ "o_orderkey": 2241 }
+{ "o_orderkey": 2242 }
+{ "o_orderkey": 2243 }
+{ "o_orderkey": 2244 }
+{ "o_orderkey": 2245 }
+{ "o_orderkey": 2246 }
+{ "o_orderkey": 2247 }
+{ "o_orderkey": 2272 }
+{ "o_orderkey": 2273 }
+{ "o_orderkey": 2274 }
+{ "o_orderkey": 2275 }
+{ "o_orderkey": 2276 }
+{ "o_orderkey": 2277 }
+{ "o_orderkey": 2278 }
+{ "o_orderkey": 2279 }
+{ "o_orderkey": 2304 }
+{ "o_orderkey": 2305 }
+{ "o_orderkey": 2306 }
+{ "o_orderkey": 2307 }
+{ "o_orderkey": 2308 }
+{ "o_orderkey": 2309 }
+{ "o_orderkey": 2310 }
+{ "o_orderkey": 2311 }
+{ "o_orderkey": 2336 }
+{ "o_orderkey": 2337 }
+{ "o_orderkey": 2338 }
+{ "o_orderkey": 2339 }
+{ "o_orderkey": 2340 }
+{ "o_orderkey": 2341 }
+{ "o_orderkey": 2342 }
+{ "o_orderkey": 2343 }
+{ "o_orderkey": 2368 }
+{ "o_orderkey": 2369 }
+{ "o_orderkey": 2370 }
+{ "o_orderkey": 2371 }
+{ "o_orderkey": 2372 }
+{ "o_orderkey": 2373 }
+{ "o_orderkey": 2374 }
+{ "o_orderkey": 2375 }
+{ "o_orderkey": 2400 }
+{ "o_orderkey": 2401 }
+{ "o_orderkey": 2402 }
+{ "o_orderkey": 2403 }
+{ "o_orderkey": 2404 }
+{ "o_orderkey": 2405 }
+{ "o_orderkey": 2406 }
+{ "o_orderkey": 2407 }
+{ "o_orderkey": 2432 }
+{ "o_orderkey": 2433 }
+{ "o_orderkey": 2434 }
+{ "o_orderkey": 2435 }
+{ "o_orderkey": 2436 }
+{ "o_orderkey": 2437 }
+{ "o_orderkey": 2438 }
+{ "o_orderkey": 2439 }
+{ "o_orderkey": 2464 }
+{ "o_orderkey": 2465 }
+{ "o_orderkey": 2466 }
+{ "o_orderkey": 2467 }
+{ "o_orderkey": 2468 }
+{ "o_orderkey": 2469 }
+{ "o_orderkey": 2470 }
+{ "o_orderkey": 2471 }
+{ "o_orderkey": 2496 }
+{ "o_orderkey": 2497 }
+{ "o_orderkey": 2498 }
+{ "o_orderkey": 2499 }
+{ "o_orderkey": 2500 }
+{ "o_orderkey": 2501 }
+{ "o_orderkey": 2502 }
+{ "o_orderkey": 2503 }
+{ "o_orderkey": 2528 }
+{ "o_orderkey": 2529 }
+{ "o_orderkey": 2530 }
+{ "o_orderkey": 2531 }
+{ "o_orderkey": 2532 }
+{ "o_orderkey": 2533 }
+{ "o_orderkey": 2534 }
+{ "o_orderkey": 2535 }
+{ "o_orderkey": 2560 }
+{ "o_orderkey": 2561 }
+{ "o_orderkey": 2562 }
+{ "o_orderkey": 2563 }
+{ "o_orderkey": 2564 }
+{ "o_orderkey": 2565 }
+{ "o_orderkey": 2566 }
+{ "o_orderkey": 2567 }
+{ "o_orderkey": 2592 }
+{ "o_orderkey": 2593 }
+{ "o_orderkey": 2594 }
+{ "o_orderkey": 2595 }
+{ "o_orderkey": 2596 }
+{ "o_orderkey": 2597 }
+{ "o_orderkey": 2598 }
+{ "o_orderkey": 2599 }
+{ "o_orderkey": 2624 }
+{ "o_orderkey": 2625 }
+{ "o_orderkey": 2626 }
+{ "o_orderkey": 2627 }
+{ "o_orderkey": 2628 }
+{ "o_orderkey": 2629 }
+{ "o_orderkey": 2630 }
+{ "o_orderkey": 2631 }
+{ "o_orderkey": 2656 }
+{ "o_orderkey": 2657 }
+{ "o_orderkey": 2658 }
+{ "o_orderkey": 2659 }
+{ "o_orderkey": 2660 }
+{ "o_orderkey": 2661 }
+{ "o_orderkey": 2662 }
+{ "o_orderkey": 2663 }
+{ "o_orderkey": 2688 }
+{ "o_orderkey": 2689 }
+{ "o_orderkey": 2690 }
+{ "o_orderkey": 2691 }
+{ "o_orderkey": 2692 }
+{ "o_orderkey": 2693 }
+{ "o_orderkey": 2694 }
+{ "o_orderkey": 2695 }
+{ "o_orderkey": 2720 }
+{ "o_orderkey": 2721 }
+{ "o_orderkey": 2722 }
+{ "o_orderkey": 2723 }
+{ "o_orderkey": 2724 }
+{ "o_orderkey": 2725 }
+{ "o_orderkey": 2726 }
+{ "o_orderkey": 2727 }
+{ "o_orderkey": 2752 }
+{ "o_orderkey": 2753 }
+{ "o_orderkey": 2754 }
+{ "o_orderkey": 2755 }
+{ "o_orderkey": 2756 }
+{ "o_orderkey": 2757 }
+{ "o_orderkey": 2758 }
+{ "o_orderkey": 2759 }
+{ "o_orderkey": 2784 }
+{ "o_orderkey": 2785 }
+{ "o_orderkey": 2786 }
+{ "o_orderkey": 2787 }
+{ "o_orderkey": 2788 }
+{ "o_orderkey": 2789 }
+{ "o_orderkey": 2790 }
+{ "o_orderkey": 2791 }
+{ "o_orderkey": 2816 }
+{ "o_orderkey": 2817 }
+{ "o_orderkey": 2818 }
+{ "o_orderkey": 2819 }
+{ "o_orderkey": 2820 }
+{ "o_orderkey": 2821 }
+{ "o_orderkey": 2822 }
+{ "o_orderkey": 2823 }
+{ "o_orderkey": 2848 }
+{ "o_orderkey": 2849 }
+{ "o_orderkey": 2850 }
+{ "o_orderkey": 2851 }
+{ "o_orderkey": 2852 }
+{ "o_orderkey": 2853 }
+{ "o_orderkey": 2854 }
+{ "o_orderkey": 2855 }
+{ "o_orderkey": 2880 }
+{ "o_orderkey": 2881 }
+{ "o_orderkey": 2882 }
+{ "o_orderkey": 2883 }
+{ "o_orderkey": 2884 }
+{ "o_orderkey": 2885 }
+{ "o_orderkey": 2886 }
+{ "o_orderkey": 2887 }
+{ "o_orderkey": 2912 }
+{ "o_orderkey": 2913 }
+{ "o_orderkey": 2914 }
+{ "o_orderkey": 2915 }
+{ "o_orderkey": 2916 }
+{ "o_orderkey": 2917 }
+{ "o_orderkey": 2918 }
+{ "o_orderkey": 2919 }
+{ "o_orderkey": 2944 }
+{ "o_orderkey": 2945 }
+{ "o_orderkey": 2946 }
+{ "o_orderkey": 2947 }
+{ "o_orderkey": 2948 }
+{ "o_orderkey": 2949 }
+{ "o_orderkey": 2950 }
+{ "o_orderkey": 2951 }
+{ "o_orderkey": 2976 }
+{ "o_orderkey": 2977 }
+{ "o_orderkey": 2978 }
+{ "o_orderkey": 2979 }
+{ "o_orderkey": 2980 }
+{ "o_orderkey": 2981 }
+{ "o_orderkey": 2982 }
+{ "o_orderkey": 2983 }
+{ "o_orderkey": 3008 }
+{ "o_orderkey": 3009 }
+{ "o_orderkey": 3010 }
+{ "o_orderkey": 3011 }
+{ "o_orderkey": 3012 }
+{ "o_orderkey": 3013 }
+{ "o_orderkey": 3014 }
+{ "o_orderkey": 3015 }
+{ "o_orderkey": 3040 }
+{ "o_orderkey": 3041 }
+{ "o_orderkey": 3042 }
+{ "o_orderkey": 3043 }
+{ "o_orderkey": 3044 }
+{ "o_orderkey": 3045 }
+{ "o_orderkey": 3046 }
+{ "o_orderkey": 3047 }
+{ "o_orderkey": 3072 }
+{ "o_orderkey": 3073 }
+{ "o_orderkey": 3074 }
+{ "o_orderkey": 3075 }
+{ "o_orderkey": 3076 }
+{ "o_orderkey": 3077 }
+{ "o_orderkey": 3078 }
+{ "o_orderkey": 3079 }
+{ "o_orderkey": 3104 }
+{ "o_orderkey": 3105 }
+{ "o_orderkey": 3106 }
+{ "o_orderkey": 3107 }
+{ "o_orderkey": 3108 }
+{ "o_orderkey": 3109 }
+{ "o_orderkey": 3110 }
+{ "o_orderkey": 3111 }
+{ "o_orderkey": 3136 }
+{ "o_orderkey": 3137 }
+{ "o_orderkey": 3138 }
+{ "o_orderkey": 3139 }
+{ "o_orderkey": 3140 }
+{ "o_orderkey": 3141 }
+{ "o_orderkey": 3142 }
+{ "o_orderkey": 3143 }
+{ "o_orderkey": 3168 }
+{ "o_orderkey": 3169 }
+{ "o_orderkey": 3170 }
+{ "o_orderkey": 3171 }
+{ "o_orderkey": 3172 }
+{ "o_orderkey": 3173 }
+{ "o_orderkey": 3174 }
+{ "o_orderkey": 3175 }
+{ "o_orderkey": 3200 }
+{ "o_orderkey": 3201 }
+{ "o_orderkey": 3202 }
+{ "o_orderkey": 3203 }
+{ "o_orderkey": 3204 }
+{ "o_orderkey": 3205 }
+{ "o_orderkey": 3206 }
+{ "o_orderkey": 3207 }
+{ "o_orderkey": 3232 }
+{ "o_orderkey": 3233 }
+{ "o_orderkey": 3234 }
+{ "o_orderkey": 3235 }
+{ "o_orderkey": 3236 }
+{ "o_orderkey": 3237 }
+{ "o_orderkey": 3238 }
+{ "o_orderkey": 3239 }
+{ "o_orderkey": 3264 }
+{ "o_orderkey": 3265 }
+{ "o_orderkey": 3266 }
+{ "o_orderkey": 3267 }
+{ "o_orderkey": 3268 }
+{ "o_orderkey": 3269 }
+{ "o_orderkey": 3270 }
+{ "o_orderkey": 3271 }
+{ "o_orderkey": 3296 }
+{ "o_orderkey": 3297 }
+{ "o_orderkey": 3298 }
+{ "o_orderkey": 3299 }
+{ "o_orderkey": 3300 }
+{ "o_orderkey": 3301 }
+{ "o_orderkey": 3302 }
+{ "o_orderkey": 3303 }
+{ "o_orderkey": 3328 }
+{ "o_orderkey": 3329 }
+{ "o_orderkey": 3330 }
+{ "o_orderkey": 3331 }
+{ "o_orderkey": 3332 }
+{ "o_orderkey": 3333 }
+{ "o_orderkey": 3334 }
+{ "o_orderkey": 3335 }
+{ "o_orderkey": 3360 }
+{ "o_orderkey": 3361 }
+{ "o_orderkey": 3362 }
+{ "o_orderkey": 3363 }
+{ "o_orderkey": 3364 }
+{ "o_orderkey": 3365 }
+{ "o_orderkey": 3366 }
+{ "o_orderkey": 3367 }
+{ "o_orderkey": 3392 }
+{ "o_orderkey": 3393 }
+{ "o_orderkey": 3394 }
+{ "o_orderkey": 3395 }
+{ "o_orderkey": 3396 }
+{ "o_orderkey": 3397 }
+{ "o_orderkey": 3398 }
+{ "o_orderkey": 3399 }
+{ "o_orderkey": 3424 }
+{ "o_orderkey": 3425 }
+{ "o_orderkey": 3426 }
+{ "o_orderkey": 3427 }
+{ "o_orderkey": 3428 }
+{ "o_orderkey": 3429 }
+{ "o_orderkey": 3430 }
+{ "o_orderkey": 3431 }
+{ "o_orderkey": 3456 }
+{ "o_orderkey": 3457 }
+{ "o_orderkey": 3458 }
+{ "o_orderkey": 3459 }
+{ "o_orderkey": 3460 }
+{ "o_orderkey": 3461 }
+{ "o_orderkey": 3462 }
+{ "o_orderkey": 3463 }
+{ "o_orderkey": 3488 }
+{ "o_orderkey": 3489 }
+{ "o_orderkey": 3490 }
+{ "o_orderkey": 3491 }
+{ "o_orderkey": 3492 }
+{ "o_orderkey": 3493 }
+{ "o_orderkey": 3494 }
+{ "o_orderkey": 3495 }
+{ "o_orderkey": 3520 }
+{ "o_orderkey": 3521 }
+{ "o_orderkey": 3522 }
+{ "o_orderkey": 3523 }
+{ "o_orderkey": 3524 }
+{ "o_orderkey": 3525 }
+{ "o_orderkey": 3526 }
+{ "o_orderkey": 3527 }
+{ "o_orderkey": 3552 }
+{ "o_orderkey": 3553 }
+{ "o_orderkey": 3554 }
+{ "o_orderkey": 3555 }
+{ "o_orderkey": 3556 }
+{ "o_orderkey": 3557 }
+{ "o_orderkey": 3558 }
+{ "o_orderkey": 3559 }
+{ "o_orderkey": 3584 }
+{ "o_orderkey": 3585 }
+{ "o_orderkey": 3586 }
+{ "o_orderkey": 3587 }
+{ "o_orderkey": 3588 }
+{ "o_orderkey": 3589 }
+{ "o_orderkey": 3590 }
+{ "o_orderkey": 3591 }
+{ "o_orderkey": 3616 }
+{ "o_orderkey": 3617 }
+{ "o_orderkey": 3618 }
+{ "o_orderkey": 3619 }
+{ "o_orderkey": 3620 }
+{ "o_orderkey": 3621 }
+{ "o_orderkey": 3622 }
+{ "o_orderkey": 3623 }
+{ "o_orderkey": 3648 }
+{ "o_orderkey": 3649 }
+{ "o_orderkey": 3650 }
+{ "o_orderkey": 3651 }
+{ "o_orderkey": 3652 }
+{ "o_orderkey": 3653 }
+{ "o_orderkey": 3654 }
+{ "o_orderkey": 3655 }
+{ "o_orderkey": 3680 }
+{ "o_orderkey": 3681 }
+{ "o_orderkey": 3682 }
+{ "o_orderkey": 3683 }
+{ "o_orderkey": 3684 }
+{ "o_orderkey": 3685 }
+{ "o_orderkey": 3686 }
+{ "o_orderkey": 3687 }
+{ "o_orderkey": 3712 }
+{ "o_orderkey": 3713 }
+{ "o_orderkey": 3714 }
+{ "o_orderkey": 3715 }
+{ "o_orderkey": 3716 }
+{ "o_orderkey": 3717 }
+{ "o_orderkey": 3718 }
+{ "o_orderkey": 3719 }
+{ "o_orderkey": 3744 }
+{ "o_orderkey": 3745 }
+{ "o_orderkey": 3746 }
+{ "o_orderkey": 3747 }
+{ "o_orderkey": 3748 }
+{ "o_orderkey": 3749 }
+{ "o_orderkey": 3750 }
+{ "o_orderkey": 3751 }
+{ "o_orderkey": 3776 }
+{ "o_orderkey": 3777 }
+{ "o_orderkey": 3778 }
+{ "o_orderkey": 3779 }
+{ "o_orderkey": 3780 }
+{ "o_orderkey": 3781 }
+{ "o_orderkey": 3782 }
+{ "o_orderkey": 3783 }
+{ "o_orderkey": 3808 }
+{ "o_orderkey": 3809 }
+{ "o_orderkey": 3810 }
+{ "o_orderkey": 3811 }
+{ "o_orderkey": 3812 }
+{ "o_orderkey": 3813 }
+{ "o_orderkey": 3814 }
+{ "o_orderkey": 3815 }
+{ "o_orderkey": 3840 }
+{ "o_orderkey": 3841 }
+{ "o_orderkey": 3842 }
+{ "o_orderkey": 3843 }
+{ "o_orderkey": 3844 }
+{ "o_orderkey": 3845 }
+{ "o_orderkey": 3846 }
+{ "o_orderkey": 3847 }
+{ "o_orderkey": 3872 }
+{ "o_orderkey": 3873 }
+{ "o_orderkey": 3874 }
+{ "o_orderkey": 3875 }
+{ "o_orderkey": 3876 }
+{ "o_orderkey": 3877 }
+{ "o_orderkey": 3878 }
+{ "o_orderkey": 3879 }
+{ "o_orderkey": 3904 }
+{ "o_orderkey": 3905 }
+{ "o_orderkey": 3906 }
+{ "o_orderkey": 3907 }
+{ "o_orderkey": 3908 }
+{ "o_orderkey": 3909 }
+{ "o_orderkey": 3910 }
+{ "o_orderkey": 3911 }
+{ "o_orderkey": 3936 }
+{ "o_orderkey": 3937 }
+{ "o_orderkey": 3938 }
+{ "o_orderkey": 3939 }
+{ "o_orderkey": 3940 }
+{ "o_orderkey": 3941 }
+{ "o_orderkey": 3942 }
+{ "o_orderkey": 3943 }
+{ "o_orderkey": 3968 }
+{ "o_orderkey": 3969 }
+{ "o_orderkey": 3970 }
+{ "o_orderkey": 3971 }
+{ "o_orderkey": 3972 }
+{ "o_orderkey": 3973 }
+{ "o_orderkey": 3974 }
+{ "o_orderkey": 3975 }
+{ "o_orderkey": 4000 }
+{ "o_orderkey": 4001 }
+{ "o_orderkey": 4002 }
+{ "o_orderkey": 4003 }
+{ "o_orderkey": 4004 }
+{ "o_orderkey": 4005 }
+{ "o_orderkey": 4006 }
+{ "o_orderkey": 4007 }
+{ "o_orderkey": 4032 }
+{ "o_orderkey": 4033 }
+{ "o_orderkey": 4034 }
+{ "o_orderkey": 4035 }
+{ "o_orderkey": 4036 }
+{ "o_orderkey": 4037 }
+{ "o_orderkey": 4038 }
+{ "o_orderkey": 4039 }
+{ "o_orderkey": 4064 }
+{ "o_orderkey": 4065 }
+{ "o_orderkey": 4066 }
+{ "o_orderkey": 4067 }
+{ "o_orderkey": 4068 }
+{ "o_orderkey": 4069 }
+{ "o_orderkey": 4070 }
+{ "o_orderkey": 4071 }
+{ "o_orderkey": 4096 }
+{ "o_orderkey": 4097 }
+{ "o_orderkey": 4098 }
+{ "o_orderkey": 4099 }
+{ "o_orderkey": 4100 }
+{ "o_orderkey": 4101 }
+{ "o_orderkey": 4102 }
+{ "o_orderkey": 4103 }
+{ "o_orderkey": 4128 }
+{ "o_orderkey": 4129 }
+{ "o_orderkey": 4130 }
+{ "o_orderkey": 4131 }
+{ "o_orderkey": 4132 }
+{ "o_orderkey": 4133 }
+{ "o_orderkey": 4134 }
+{ "o_orderkey": 4135 }
+{ "o_orderkey": 4160 }
+{ "o_orderkey": 4161 }
+{ "o_orderkey": 4162 }
+{ "o_orderkey": 4163 }
+{ "o_orderkey": 4164 }
+{ "o_orderkey": 4165 }
+{ "o_orderkey": 4166 }
+{ "o_orderkey": 4167 }
+{ "o_orderkey": 4192 }
+{ "o_orderkey": 4193 }
+{ "o_orderkey": 4194 }
+{ "o_orderkey": 4195 }
+{ "o_orderkey": 4196 }
+{ "o_orderkey": 4197 }
+{ "o_orderkey": 4198 }
+{ "o_orderkey": 4199 }
+{ "o_orderkey": 4224 }
+{ "o_orderkey": 4225 }
+{ "o_orderkey": 4226 }
+{ "o_orderkey": 4227 }
+{ "o_orderkey": 4228 }
+{ "o_orderkey": 4229 }
+{ "o_orderkey": 4230 }
+{ "o_orderkey": 4231 }
+{ "o_orderkey": 4256 }
+{ "o_orderkey": 4257 }
+{ "o_orderkey": 4258 }
+{ "o_orderkey": 4259 }
+{ "o_orderkey": 4260 }
+{ "o_orderkey": 4261 }
+{ "o_orderkey": 4262 }
+{ "o_orderkey": 4263 }
+{ "o_orderkey": 4288 }
+{ "o_orderkey": 4289 }
+{ "o_orderkey": 4290 }
+{ "o_orderkey": 4291 }
+{ "o_orderkey": 4292 }
+{ "o_orderkey": 4293 }
+{ "o_orderkey": 4294 }
+{ "o_orderkey": 4295 }
+{ "o_orderkey": 4320 }
+{ "o_orderkey": 4321 }
+{ "o_orderkey": 4322 }
+{ "o_orderkey": 4323 }
+{ "o_orderkey": 4324 }
+{ "o_orderkey": 4325 }
+{ "o_orderkey": 4326 }
+{ "o_orderkey": 4327 }
+{ "o_orderkey": 4352 }
+{ "o_orderkey": 4353 }
+{ "o_orderkey": 4354 }
+{ "o_orderkey": 4355 }
+{ "o_orderkey": 4356 }
+{ "o_orderkey": 4357 }
+{ "o_orderkey": 4358 }
+{ "o_orderkey": 4359 }
+{ "o_orderkey": 4384 }
+{ "o_orderkey": 4385 }
+{ "o_orderkey": 4386 }
+{ "o_orderkey": 4387 }
+{ "o_orderkey": 4388 }
+{ "o_orderkey": 4389 }
+{ "o_orderkey": 4390 }
+{ "o_orderkey": 4391 }
+{ "o_orderkey": 4416 }
+{ "o_orderkey": 4417 }
+{ "o_orderkey": 4418 }
+{ "o_orderkey": 4419 }
+{ "o_orderkey": 4420 }
+{ "o_orderkey": 4421 }
+{ "o_orderkey": 4422 }
+{ "o_orderkey": 4423 }
+{ "o_orderkey": 4448 }
+{ "o_orderkey": 4449 }
+{ "o_orderkey": 4450 }
+{ "o_orderkey": 4451 }
+{ "o_orderkey": 4452 }
+{ "o_orderkey": 4453 }
+{ "o_orderkey": 4454 }
+{ "o_orderkey": 4455 }
+{ "o_orderkey": 4480 }
+{ "o_orderkey": 4481 }
+{ "o_orderkey": 4482 }
+{ "o_orderkey": 4483 }
+{ "o_orderkey": 4484 }
+{ "o_orderkey": 4485 }
+{ "o_orderkey": 4486 }
+{ "o_orderkey": 4487 }
+{ "o_orderkey": 4512 }
+{ "o_orderkey": 4513 }
+{ "o_orderkey": 4514 }
+{ "o_orderkey": 4515 }
+{ "o_orderkey": 4516 }
+{ "o_orderkey": 4517 }
+{ "o_orderkey": 4518 }
+{ "o_orderkey": 4519 }
+{ "o_orderkey": 4544 }
+{ "o_orderkey": 4545 }
+{ "o_orderkey": 4546 }
+{ "o_orderkey": 4547 }
+{ "o_orderkey": 4548 }
+{ "o_orderkey": 4549 }
+{ "o_orderkey": 4550 }
+{ "o_orderkey": 4551 }
+{ "o_orderkey": 4576 }
+{ "o_orderkey": 4577 }
+{ "o_orderkey": 4578 }
+{ "o_orderkey": 4579 }
+{ "o_orderkey": 4580 }
+{ "o_orderkey": 4581 }
+{ "o_orderkey": 4582 }
+{ "o_orderkey": 4583 }
+{ "o_orderkey": 4608 }
+{ "o_orderkey": 4609 }
+{ "o_orderkey": 4610 }
+{ "o_orderkey": 4611 }
+{ "o_orderkey": 4612 }
+{ "o_orderkey": 4613 }
+{ "o_orderkey": 4614 }
+{ "o_orderkey": 4615 }
+{ "o_orderkey": 4640 }
+{ "o_orderkey": 4641 }
+{ "o_orderkey": 4642 }
+{ "o_orderkey": 4643 }
+{ "o_orderkey": 4644 }
+{ "o_orderkey": 4645 }
+{ "o_orderkey": 4646 }
+{ "o_orderkey": 4647 }
+{ "o_orderkey": 4672 }
+{ "o_orderkey": 4673 }
+{ "o_orderkey": 4674 }
+{ "o_orderkey": 4675 }
+{ "o_orderkey": 4676 }
+{ "o_orderkey": 4677 }
+{ "o_orderkey": 4678 }
+{ "o_orderkey": 4679 }
+{ "o_orderkey": 4704 }
+{ "o_orderkey": 4705 }
+{ "o_orderkey": 4706 }
+{ "o_orderkey": 4707 }
+{ "o_orderkey": 4708 }
+{ "o_orderkey": 4709 }
+{ "o_orderkey": 4710 }
+{ "o_orderkey": 4711 }
+{ "o_orderkey": 4736 }
+{ "o_orderkey": 4737 }
+{ "o_orderkey": 4738 }
+{ "o_orderkey": 4739 }
+{ "o_orderkey": 4740 }
+{ "o_orderkey": 4741 }
+{ "o_orderkey": 4742 }
+{ "o_orderkey": 4743 }
+{ "o_orderkey": 4768 }
+{ "o_orderkey": 4769 }
+{ "o_orderkey": 4770 }
+{ "o_orderkey": 4771 }
+{ "o_orderkey": 4772 }
+{ "o_orderkey": 4773 }
+{ "o_orderkey": 4774 }
+{ "o_orderkey": 4775 }
+{ "o_orderkey": 4800 }
+{ "o_orderkey": 4801 }
+{ "o_orderkey": 4802 }
+{ "o_orderkey": 4803 }
+{ "o_orderkey": 4804 }
+{ "o_orderkey": 4805 }
+{ "o_orderkey": 4806 }
+{ "o_orderkey": 4807 }
+{ "o_orderkey": 4832 }
+{ "o_orderkey": 4833 }
+{ "o_orderkey": 4834 }
+{ "o_orderkey": 4835 }
+{ "o_orderkey": 4836 }
+{ "o_orderkey": 4837 }
+{ "o_orderkey": 4838 }
+{ "o_orderkey": 4839 }
+{ "o_orderkey": 4864 }
+{ "o_orderkey": 4865 }
+{ "o_orderkey": 4866 }
+{ "o_orderkey": 4867 }
+{ "o_orderkey": 4868 }
+{ "o_orderkey": 4869 }
+{ "o_orderkey": 4870 }
+{ "o_orderkey": 4871 }
+{ "o_orderkey": 4896 }
+{ "o_orderkey": 4897 }
+{ "o_orderkey": 4898 }
+{ "o_orderkey": 4899 }
+{ "o_orderkey": 4900 }
+{ "o_orderkey": 4901 }
+{ "o_orderkey": 4902 }
+{ "o_orderkey": 4903 }
+{ "o_orderkey": 4928 }
+{ "o_orderkey": 4929 }
+{ "o_orderkey": 4930 }
+{ "o_orderkey": 4931 }
+{ "o_orderkey": 4932 }
+{ "o_orderkey": 4933 }
+{ "o_orderkey": 4934 }
+{ "o_orderkey": 4935 }
+{ "o_orderkey": 4960 }
+{ "o_orderkey": 4961 }
+{ "o_orderkey": 4962 }
+{ "o_orderkey": 4963 }
+{ "o_orderkey": 4964 }
+{ "o_orderkey": 4965 }
+{ "o_orderkey": 4966 }
+{ "o_orderkey": 4967 }
+{ "o_orderkey": 4992 }
+{ "o_orderkey": 4993 }
+{ "o_orderkey": 4994 }
+{ "o_orderkey": 4995 }
+{ "o_orderkey": 4996 }
+{ "o_orderkey": 4997 }
+{ "o_orderkey": 4998 }
+{ "o_orderkey": 4999 }
+{ "o_orderkey": 5024 }
+{ "o_orderkey": 5025 }
+{ "o_orderkey": 5026 }
+{ "o_orderkey": 5027 }
+{ "o_orderkey": 5028 }
+{ "o_orderkey": 5029 }
+{ "o_orderkey": 5030 }
+{ "o_orderkey": 5031 }
+{ "o_orderkey": 5056 }
+{ "o_orderkey": 5057 }
+{ "o_orderkey": 5058 }
+{ "o_orderkey": 5059 }
+{ "o_orderkey": 5060 }
+{ "o_orderkey": 5061 }
+{ "o_orderkey": 5062 }
+{ "o_orderkey": 5063 }
+{ "o_orderkey": 5088 }
+{ "o_orderkey": 5089 }
+{ "o_orderkey": 5090 }
+{ "o_orderkey": 5091 }
+{ "o_orderkey": 5092 }
+{ "o_orderkey": 5093 }
+{ "o_orderkey": 5094 }
+{ "o_orderkey": 5095 }
+{ "o_orderkey": 5120 }
+{ "o_orderkey": 5121 }
+{ "o_orderkey": 5122 }
+{ "o_orderkey": 5123 }
+{ "o_orderkey": 5124 }
+{ "o_orderkey": 5125 }
+{ "o_orderkey": 5126 }
+{ "o_orderkey": 5127 }
+{ "o_orderkey": 5152 }
+{ "o_orderkey": 5153 }
+{ "o_orderkey": 5154 }
+{ "o_orderkey": 5155 }
+{ "o_orderkey": 5156 }
+{ "o_orderkey": 5157 }
+{ "o_orderkey": 5158 }
+{ "o_orderkey": 5159 }
+{ "o_orderkey": 5184 }
+{ "o_orderkey": 5185 }
+{ "o_orderkey": 5186 }
+{ "o_orderkey": 5187 }
+{ "o_orderkey": 5188 }
+{ "o_orderkey": 5189 }
+{ "o_orderkey": 5190 }
+{ "o_orderkey": 5191 }
+{ "o_orderkey": 5216 }
+{ "o_orderkey": 5217 }
+{ "o_orderkey": 5218 }
+{ "o_orderkey": 5219 }
+{ "o_orderkey": 5220 }
+{ "o_orderkey": 5221 }
+{ "o_orderkey": 5222 }
+{ "o_orderkey": 5223 }
+{ "o_orderkey": 5248 }
+{ "o_orderkey": 5249 }
+{ "o_orderkey": 5250 }
+{ "o_orderkey": 5251 }
+{ "o_orderkey": 5252 }
+{ "o_orderkey": 5253 }
+{ "o_orderkey": 5254 }
+{ "o_orderkey": 5255 }
+{ "o_orderkey": 5280 }
+{ "o_orderkey": 5281 }
+{ "o_orderkey": 5282 }
+{ "o_orderkey": 5283 }
+{ "o_orderkey": 5284 }
+{ "o_orderkey": 5285 }
+{ "o_orderkey": 5286 }
+{ "o_orderkey": 5287 }
+{ "o_orderkey": 5312 }
+{ "o_orderkey": 5313 }
+{ "o_orderkey": 5314 }
+{ "o_orderkey": 5315 }
+{ "o_orderkey": 5316 }
+{ "o_orderkey": 5317 }
+{ "o_orderkey": 5318 }
+{ "o_orderkey": 5319 }
+{ "o_orderkey": 5344 }
+{ "o_orderkey": 5345 }
+{ "o_orderkey": 5346 }
+{ "o_orderkey": 5347 }
+{ "o_orderkey": 5348 }
+{ "o_orderkey": 5349 }
+{ "o_orderkey": 5350 }
+{ "o_orderkey": 5351 }
+{ "o_orderkey": 5376 }
+{ "o_orderkey": 5377 }
+{ "o_orderkey": 5378 }
+{ "o_orderkey": 5379 }
+{ "o_orderkey": 5380 }
+{ "o_orderkey": 5381 }
+{ "o_orderkey": 5382 }
+{ "o_orderkey": 5383 }
+{ "o_orderkey": 5408 }
+{ "o_orderkey": 5409 }
+{ "o_orderkey": 5410 }
+{ "o_orderkey": 5411 }
+{ "o_orderkey": 5412 }
+{ "o_orderkey": 5413 }
+{ "o_orderkey": 5414 }
+{ "o_orderkey": 5415 }
+{ "o_orderkey": 5440 }
+{ "o_orderkey": 5441 }
+{ "o_orderkey": 5442 }
+{ "o_orderkey": 5443 }
+{ "o_orderkey": 5444 }
+{ "o_orderkey": 5445 }
+{ "o_orderkey": 5446 }
+{ "o_orderkey": 5447 }
+{ "o_orderkey": 5472 }
+{ "o_orderkey": 5473 }
+{ "o_orderkey": 5474 }
+{ "o_orderkey": 5475 }
+{ "o_orderkey": 5476 }
+{ "o_orderkey": 5477 }
+{ "o_orderkey": 5478 }
+{ "o_orderkey": 5479 }
+{ "o_orderkey": 5504 }
+{ "o_orderkey": 5505 }
+{ "o_orderkey": 5506 }
+{ "o_orderkey": 5507 }
+{ "o_orderkey": 5508 }
+{ "o_orderkey": 5509 }
+{ "o_orderkey": 5510 }
+{ "o_orderkey": 5511 }
+{ "o_orderkey": 5536 }
+{ "o_orderkey": 5537 }
+{ "o_orderkey": 5538 }
+{ "o_orderkey": 5539 }
+{ "o_orderkey": 5540 }
+{ "o_orderkey": 5541 }
+{ "o_orderkey": 5542 }
+{ "o_orderkey": 5543 }
+{ "o_orderkey": 5568 }
+{ "o_orderkey": 5569 }
+{ "o_orderkey": 5570 }
+{ "o_orderkey": 5571 }
+{ "o_orderkey": 5572 }
+{ "o_orderkey": 5573 }
+{ "o_orderkey": 5574 }
+{ "o_orderkey": 5575 }
+{ "o_orderkey": 5600 }
+{ "o_orderkey": 5601 }
+{ "o_orderkey": 5602 }
+{ "o_orderkey": 5603 }
+{ "o_orderkey": 5604 }
+{ "o_orderkey": 5605 }
+{ "o_orderkey": 5606 }
+{ "o_orderkey": 5607 }
+{ "o_orderkey": 5632 }
+{ "o_orderkey": 5633 }
+{ "o_orderkey": 5634 }
+{ "o_orderkey": 5635 }
+{ "o_orderkey": 5636 }
+{ "o_orderkey": 5637 }
+{ "o_orderkey": 5638 }
+{ "o_orderkey": 5639 }
+{ "o_orderkey": 5664 }
+{ "o_orderkey": 5665 }
+{ "o_orderkey": 5666 }
+{ "o_orderkey": 5667 }
+{ "o_orderkey": 5668 }
+{ "o_orderkey": 5669 }
+{ "o_orderkey": 5670 }
+{ "o_orderkey": 5671 }
+{ "o_orderkey": 5696 }
+{ "o_orderkey": 5697 }
+{ "o_orderkey": 5698 }
+{ "o_orderkey": 5699 }
+{ "o_orderkey": 5700 }
+{ "o_orderkey": 5701 }
+{ "o_orderkey": 5702 }
+{ "o_orderkey": 5703 }
+{ "o_orderkey": 5728 }
+{ "o_orderkey": 5729 }
+{ "o_orderkey": 5730 }
+{ "o_orderkey": 5731 }
+{ "o_orderkey": 5732 }
+{ "o_orderkey": 5733 }
+{ "o_orderkey": 5734 }
+{ "o_orderkey": 5735 }
+{ "o_orderkey": 5760 }
+{ "o_orderkey": 5761 }
+{ "o_orderkey": 5762 }
+{ "o_orderkey": 5763 }
+{ "o_orderkey": 5764 }
+{ "o_orderkey": 5765 }
+{ "o_orderkey": 5766 }
+{ "o_orderkey": 5767 }
+{ "o_orderkey": 5792 }
+{ "o_orderkey": 5793 }
+{ "o_orderkey": 5794 }
+{ "o_orderkey": 5795 }
+{ "o_orderkey": 5796 }
+{ "o_orderkey": 5797 }
+{ "o_orderkey": 5798 }
+{ "o_orderkey": 5799 }
+{ "o_orderkey": 5824 }
+{ "o_orderkey": 5825 }
+{ "o_orderkey": 5826 }
+{ "o_orderkey": 5827 }
+{ "o_orderkey": 5828 }
+{ "o_orderkey": 5829 }
+{ "o_orderkey": 5830 }
+{ "o_orderkey": 5831 }
+{ "o_orderkey": 5856 }
+{ "o_orderkey": 5857 }
+{ "o_orderkey": 5858 }
+{ "o_orderkey": 5859 }
+{ "o_orderkey": 5860 }
+{ "o_orderkey": 5861 }
+{ "o_orderkey": 5862 }
+{ "o_orderkey": 5863 }
+{ "o_orderkey": 5888 }
+{ "o_orderkey": 5889 }
+{ "o_orderkey": 5890 }
+{ "o_orderkey": 5891 }
+{ "o_orderkey": 5892 }
+{ "o_orderkey": 5893 }
+{ "o_orderkey": 5894 }
+{ "o_orderkey": 5895 }
+{ "o_orderkey": 5920 }
+{ "o_orderkey": 5921 }
+{ "o_orderkey": 5922 }
+{ "o_orderkey": 5923 }
+{ "o_orderkey": 5924 }
+{ "o_orderkey": 5925 }
+{ "o_orderkey": 5926 }
+{ "o_orderkey": 5927 }
+{ "o_orderkey": 5952 }
+{ "o_orderkey": 5953 }
+{ "o_orderkey": 5954 }
+{ "o_orderkey": 5955 }
+{ "o_orderkey": 5956 }
+{ "o_orderkey": 5957 }
+{ "o_orderkey": 5958 }
+{ "o_orderkey": 5959 }
+{ "o_orderkey": 5984 }
+{ "o_orderkey": 5985 }
+{ "o_orderkey": 5986 }
+{ "o_orderkey": 5987 }
+{ "o_orderkey": 5988 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.08.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.08.plan
new file mode 100644
index 0000000..cf774fa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.08.plan
@@ -0,0 +1,50 @@
+distribute result [$$36] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$36]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$36] <- [{"o_orderkey": $$41, "l_orderkey": $$42, "l_suppkey": $$45}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- SORT_MERGE_EXCHANGE [$$41(ASC), $$42(ASC), $$45(ASC) ] |PARTITIONED|
+ order (ASC, $$41) (ASC, $$42) (ASC, $$45) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STABLE_SORT [$$41(ASC), $$42(ASC), $$45(ASC)] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$41, $$42, $$45]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ left outer join (and(eq($$41, $$42), eq($$54, $$45))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- HYBRID_HASH_JOIN [$$41, $$54][$$42, $$45] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ assign [$$54] <- [$$41] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$41]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$41, $$o] <- tpch.Orders [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- HASH_PARTITION_EXCHANGE [$$42] |PARTITIONED|
+ project ([$$42, $$45]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$45] <- [$$l.getField(2)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$42, $$l]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$42, $$43, $$l] <- tpch.LineItem [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.09.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.09.adm
new file mode 100644
index 0000000..322cb10
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.09.adm
@@ -0,0 +1,58 @@
+{ "n_nationkey": 1, "s_nationkey": 1, "c_nationkey": 1 }
+{ "n_nationkey": 1, "s_nationkey": 1, "c_nationkey": 1 }
+{ "n_nationkey": 1, "s_nationkey": 1, "c_nationkey": 1 }
+{ "n_nationkey": 1, "s_nationkey": 1, "c_nationkey": 1 }
+{ "n_nationkey": 1, "s_nationkey": 1, "c_nationkey": 1 }
+{ "n_nationkey": 1, "s_nationkey": 1, "c_nationkey": 1 }
+{ "n_nationkey": 1, "s_nationkey": 1, "c_nationkey": 1 }
+{ "n_nationkey": 5, "s_nationkey": 5, "c_nationkey": 5 }
+{ "n_nationkey": 5, "s_nationkey": 5, "c_nationkey": 5 }
+{ "n_nationkey": 5, "s_nationkey": 5, "c_nationkey": 5 }
+{ "n_nationkey": 5, "s_nationkey": 5, "c_nationkey": 5 }
+{ "n_nationkey": 5, "s_nationkey": 5, "c_nationkey": 5 }
+{ "n_nationkey": 5, "s_nationkey": 5, "c_nationkey": 5 }
+{ "n_nationkey": 10, "s_nationkey": 10, "c_nationkey": 10 }
+{ "n_nationkey": 10, "s_nationkey": 10, "c_nationkey": 10 }
+{ "n_nationkey": 10, "s_nationkey": 10, "c_nationkey": 10 }
+{ "n_nationkey": 10, "s_nationkey": 10, "c_nationkey": 10 }
+{ "n_nationkey": 10, "s_nationkey": 10, "c_nationkey": 10 }
+{ "n_nationkey": 10, "s_nationkey": 10, "c_nationkey": 10 }
+{ "n_nationkey": 10, "s_nationkey": 10, "c_nationkey": 10 }
+{ "n_nationkey": 10, "s_nationkey": 10, "c_nationkey": 10 }
+{ "n_nationkey": 11, "s_nationkey": 11, "c_nationkey": 11 }
+{ "n_nationkey": 11, "s_nationkey": 11, "c_nationkey": 11 }
+{ "n_nationkey": 11, "s_nationkey": 11, "c_nationkey": 11 }
+{ "n_nationkey": 11, "s_nationkey": 11, "c_nationkey": 11 }
+{ "n_nationkey": 11, "s_nationkey": 11, "c_nationkey": 11 }
+{ "n_nationkey": 14, "s_nationkey": 14, "c_nationkey": 14 }
+{ "n_nationkey": 14, "s_nationkey": 14, "c_nationkey": 14 }
+{ "n_nationkey": 15, "s_nationkey": 15, "c_nationkey": 15 }
+{ "n_nationkey": 15, "s_nationkey": 15, "c_nationkey": 15 }
+{ "n_nationkey": 15, "s_nationkey": 15, "c_nationkey": 15 }
+{ "n_nationkey": 15, "s_nationkey": 15, "c_nationkey": 15 }
+{ "n_nationkey": 15, "s_nationkey": 15, "c_nationkey": 15 }
+{ "n_nationkey": 15, "s_nationkey": 15, "c_nationkey": 15 }
+{ "n_nationkey": 15, "s_nationkey": 15, "c_nationkey": 15 }
+{ "n_nationkey": 15, "s_nationkey": 15, "c_nationkey": 15 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 23, "s_nationkey": 23, "c_nationkey": 23 }
+{ "n_nationkey": 23, "s_nationkey": 23, "c_nationkey": 23 }
+{ "n_nationkey": 23, "s_nationkey": 23, "c_nationkey": 23 }
+{ "n_nationkey": 23, "s_nationkey": 23, "c_nationkey": 23 }
+{ "n_nationkey": 23, "s_nationkey": 23, "c_nationkey": 23 }
+{ "n_nationkey": 24, "s_nationkey": 24, "c_nationkey": 24 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.10.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.10.plan
new file mode 100644
index 0000000..554d42f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.10.plan
@@ -0,0 +1,64 @@
+distribute result [$$48] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$48]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$48] <- [{"n_nationkey": $$55, "s_nationkey": $$53, "c_nationkey": $$52}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- SORT_MERGE_EXCHANGE [$$55(ASC), $$53(ASC), $$52(ASC) ] |PARTITIONED|
+ order (ASC, $$55) (ASC, $$53) (ASC, $$52) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STABLE_SORT [$$55(ASC), $$53(ASC), $$52(ASC)] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ join (eq($$52, $$55)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- HYBRID_HASH_JOIN [$$55][$$52] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ join (eq($$53, $$55)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- HYBRID_HASH_JOIN [$$55][$$53] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- HASH_PARTITION_EXCHANGE [$$55] |PARTITIONED|
+ project ([$$55]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$55, $$n] <- tpch.Nation [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- HASH_PARTITION_EXCHANGE [$$53] |PARTITIONED|
+ project ([$$53]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$53] <- [$$s.getField(3)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$s]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$56, $$s] <- tpch.Supplier [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- HASH_PARTITION_EXCHANGE [$$52] |PARTITIONED|
+ project ([$$52]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$52] <- [$$c.getField(3)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$c]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$57, $$c] <- tpch.Customer [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.11.adm
new file mode 100644
index 0000000..322cb10
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.11.adm
@@ -0,0 +1,58 @@
+{ "n_nationkey": 1, "s_nationkey": 1, "c_nationkey": 1 }
+{ "n_nationkey": 1, "s_nationkey": 1, "c_nationkey": 1 }
+{ "n_nationkey": 1, "s_nationkey": 1, "c_nationkey": 1 }
+{ "n_nationkey": 1, "s_nationkey": 1, "c_nationkey": 1 }
+{ "n_nationkey": 1, "s_nationkey": 1, "c_nationkey": 1 }
+{ "n_nationkey": 1, "s_nationkey": 1, "c_nationkey": 1 }
+{ "n_nationkey": 1, "s_nationkey": 1, "c_nationkey": 1 }
+{ "n_nationkey": 5, "s_nationkey": 5, "c_nationkey": 5 }
+{ "n_nationkey": 5, "s_nationkey": 5, "c_nationkey": 5 }
+{ "n_nationkey": 5, "s_nationkey": 5, "c_nationkey": 5 }
+{ "n_nationkey": 5, "s_nationkey": 5, "c_nationkey": 5 }
+{ "n_nationkey": 5, "s_nationkey": 5, "c_nationkey": 5 }
+{ "n_nationkey": 5, "s_nationkey": 5, "c_nationkey": 5 }
+{ "n_nationkey": 10, "s_nationkey": 10, "c_nationkey": 10 }
+{ "n_nationkey": 10, "s_nationkey": 10, "c_nationkey": 10 }
+{ "n_nationkey": 10, "s_nationkey": 10, "c_nationkey": 10 }
+{ "n_nationkey": 10, "s_nationkey": 10, "c_nationkey": 10 }
+{ "n_nationkey": 10, "s_nationkey": 10, "c_nationkey": 10 }
+{ "n_nationkey": 10, "s_nationkey": 10, "c_nationkey": 10 }
+{ "n_nationkey": 10, "s_nationkey": 10, "c_nationkey": 10 }
+{ "n_nationkey": 10, "s_nationkey": 10, "c_nationkey": 10 }
+{ "n_nationkey": 11, "s_nationkey": 11, "c_nationkey": 11 }
+{ "n_nationkey": 11, "s_nationkey": 11, "c_nationkey": 11 }
+{ "n_nationkey": 11, "s_nationkey": 11, "c_nationkey": 11 }
+{ "n_nationkey": 11, "s_nationkey": 11, "c_nationkey": 11 }
+{ "n_nationkey": 11, "s_nationkey": 11, "c_nationkey": 11 }
+{ "n_nationkey": 14, "s_nationkey": 14, "c_nationkey": 14 }
+{ "n_nationkey": 14, "s_nationkey": 14, "c_nationkey": 14 }
+{ "n_nationkey": 15, "s_nationkey": 15, "c_nationkey": 15 }
+{ "n_nationkey": 15, "s_nationkey": 15, "c_nationkey": 15 }
+{ "n_nationkey": 15, "s_nationkey": 15, "c_nationkey": 15 }
+{ "n_nationkey": 15, "s_nationkey": 15, "c_nationkey": 15 }
+{ "n_nationkey": 15, "s_nationkey": 15, "c_nationkey": 15 }
+{ "n_nationkey": 15, "s_nationkey": 15, "c_nationkey": 15 }
+{ "n_nationkey": 15, "s_nationkey": 15, "c_nationkey": 15 }
+{ "n_nationkey": 15, "s_nationkey": 15, "c_nationkey": 15 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 23, "s_nationkey": 23, "c_nationkey": 23 }
+{ "n_nationkey": 23, "s_nationkey": 23, "c_nationkey": 23 }
+{ "n_nationkey": 23, "s_nationkey": 23, "c_nationkey": 23 }
+{ "n_nationkey": 23, "s_nationkey": 23, "c_nationkey": 23 }
+{ "n_nationkey": 23, "s_nationkey": 23, "c_nationkey": 23 }
+{ "n_nationkey": 24, "s_nationkey": 24, "c_nationkey": 24 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.12.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.12.plan
new file mode 100644
index 0000000..554d42f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.12.plan
@@ -0,0 +1,64 @@
+distribute result [$$48] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$48]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$48] <- [{"n_nationkey": $$55, "s_nationkey": $$53, "c_nationkey": $$52}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- SORT_MERGE_EXCHANGE [$$55(ASC), $$53(ASC), $$52(ASC) ] |PARTITIONED|
+ order (ASC, $$55) (ASC, $$53) (ASC, $$52) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STABLE_SORT [$$55(ASC), $$53(ASC), $$52(ASC)] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ join (eq($$52, $$55)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- HYBRID_HASH_JOIN [$$55][$$52] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ join (eq($$53, $$55)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- HYBRID_HASH_JOIN [$$55][$$53] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- HASH_PARTITION_EXCHANGE [$$55] |PARTITIONED|
+ project ([$$55]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$55, $$n] <- tpch.Nation [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- HASH_PARTITION_EXCHANGE [$$53] |PARTITIONED|
+ project ([$$53]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$53] <- [$$s.getField(3)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$s]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$56, $$s] <- tpch.Supplier [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- HASH_PARTITION_EXCHANGE [$$52] |PARTITIONED|
+ project ([$$52]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$52] <- [$$c.getField(3)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$c]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$57, $$c] <- tpch.Customer [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.13.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.13.adm
new file mode 100644
index 0000000..322cb10
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.13.adm
@@ -0,0 +1,58 @@
+{ "n_nationkey": 1, "s_nationkey": 1, "c_nationkey": 1 }
+{ "n_nationkey": 1, "s_nationkey": 1, "c_nationkey": 1 }
+{ "n_nationkey": 1, "s_nationkey": 1, "c_nationkey": 1 }
+{ "n_nationkey": 1, "s_nationkey": 1, "c_nationkey": 1 }
+{ "n_nationkey": 1, "s_nationkey": 1, "c_nationkey": 1 }
+{ "n_nationkey": 1, "s_nationkey": 1, "c_nationkey": 1 }
+{ "n_nationkey": 1, "s_nationkey": 1, "c_nationkey": 1 }
+{ "n_nationkey": 5, "s_nationkey": 5, "c_nationkey": 5 }
+{ "n_nationkey": 5, "s_nationkey": 5, "c_nationkey": 5 }
+{ "n_nationkey": 5, "s_nationkey": 5, "c_nationkey": 5 }
+{ "n_nationkey": 5, "s_nationkey": 5, "c_nationkey": 5 }
+{ "n_nationkey": 5, "s_nationkey": 5, "c_nationkey": 5 }
+{ "n_nationkey": 5, "s_nationkey": 5, "c_nationkey": 5 }
+{ "n_nationkey": 10, "s_nationkey": 10, "c_nationkey": 10 }
+{ "n_nationkey": 10, "s_nationkey": 10, "c_nationkey": 10 }
+{ "n_nationkey": 10, "s_nationkey": 10, "c_nationkey": 10 }
+{ "n_nationkey": 10, "s_nationkey": 10, "c_nationkey": 10 }
+{ "n_nationkey": 10, "s_nationkey": 10, "c_nationkey": 10 }
+{ "n_nationkey": 10, "s_nationkey": 10, "c_nationkey": 10 }
+{ "n_nationkey": 10, "s_nationkey": 10, "c_nationkey": 10 }
+{ "n_nationkey": 10, "s_nationkey": 10, "c_nationkey": 10 }
+{ "n_nationkey": 11, "s_nationkey": 11, "c_nationkey": 11 }
+{ "n_nationkey": 11, "s_nationkey": 11, "c_nationkey": 11 }
+{ "n_nationkey": 11, "s_nationkey": 11, "c_nationkey": 11 }
+{ "n_nationkey": 11, "s_nationkey": 11, "c_nationkey": 11 }
+{ "n_nationkey": 11, "s_nationkey": 11, "c_nationkey": 11 }
+{ "n_nationkey": 14, "s_nationkey": 14, "c_nationkey": 14 }
+{ "n_nationkey": 14, "s_nationkey": 14, "c_nationkey": 14 }
+{ "n_nationkey": 15, "s_nationkey": 15, "c_nationkey": 15 }
+{ "n_nationkey": 15, "s_nationkey": 15, "c_nationkey": 15 }
+{ "n_nationkey": 15, "s_nationkey": 15, "c_nationkey": 15 }
+{ "n_nationkey": 15, "s_nationkey": 15, "c_nationkey": 15 }
+{ "n_nationkey": 15, "s_nationkey": 15, "c_nationkey": 15 }
+{ "n_nationkey": 15, "s_nationkey": 15, "c_nationkey": 15 }
+{ "n_nationkey": 15, "s_nationkey": 15, "c_nationkey": 15 }
+{ "n_nationkey": 15, "s_nationkey": 15, "c_nationkey": 15 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 23, "s_nationkey": 23, "c_nationkey": 23 }
+{ "n_nationkey": 23, "s_nationkey": 23, "c_nationkey": 23 }
+{ "n_nationkey": 23, "s_nationkey": 23, "c_nationkey": 23 }
+{ "n_nationkey": 23, "s_nationkey": 23, "c_nationkey": 23 }
+{ "n_nationkey": 23, "s_nationkey": 23, "c_nationkey": 23 }
+{ "n_nationkey": 24, "s_nationkey": 24, "c_nationkey": 24 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.14.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.14.plan
new file mode 100644
index 0000000..2dd4acc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.14.plan
@@ -0,0 +1,66 @@
+distribute result [$$48] [cardinality: 2.1, op-cost: 0.0, total-cost: 1.1E7]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 1.1E7]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$48]) [cardinality: 2.1, op-cost: 0.0, total-cost: 1.1E7]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$48] <- [{"n_nationkey": $$56, "s_nationkey": $$53, "c_nationkey": $$52}] [cardinality: 2.1, op-cost: 0.0, total-cost: 1.1E7]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 1.1E7]
+ -- SORT_MERGE_EXCHANGE [$$56(ASC), $$53(ASC), $$52(ASC) ] |PARTITIONED|
+ order (ASC, $$56) (ASC, $$53) (ASC, $$52) [cardinality: 2.1, op-cost: 0.0, total-cost: 1.1E7]
+ -- STABLE_SORT [$$56(ASC), $$53(ASC), $$52(ASC)] |PARTITIONED|
+ exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 1.1E7]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$56, $$53, $$52]) [cardinality: 2.1, op-cost: 0.0, total-cost: 1.1E7]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 1.1E7]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ join (and(eq($$52, $$56), eq($$53, $$63))) [cardinality: 2.1, op-cost: 2000000.0, total-cost: 1.1E7]
+ -- HYBRID_HASH_JOIN [$$56, $$53][$$52, $$63] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 7000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$53, $$56]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 7000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ unnest-map [$$56, $$n] <- index-search("Nation", 0, "tpch", "Nation", true, true, 1, $$53, 1, $$53, true, true, true) [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 6000000.0]
+ -- BTREE_SEARCH |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ order (ASC, $$53) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STABLE_SORT [$$53(ASC)] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- HASH_PARTITION_EXCHANGE [$$53] |PARTITIONED|
+ project ([$$53]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$53] <- [$$s.getField(3)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$s]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$55, $$s] <- tpch.Supplier [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
+ -- HASH_PARTITION_EXCHANGE [$$52] |PARTITIONED|
+ assign [$$63] <- [$$52] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$52]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$52] <- [$$c.getField(3)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$c]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$57, $$c] <- tpch.Customer [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.15.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.15.adm
new file mode 100644
index 0000000..322cb10
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.15.adm
@@ -0,0 +1,58 @@
+{ "n_nationkey": 1, "s_nationkey": 1, "c_nationkey": 1 }
+{ "n_nationkey": 1, "s_nationkey": 1, "c_nationkey": 1 }
+{ "n_nationkey": 1, "s_nationkey": 1, "c_nationkey": 1 }
+{ "n_nationkey": 1, "s_nationkey": 1, "c_nationkey": 1 }
+{ "n_nationkey": 1, "s_nationkey": 1, "c_nationkey": 1 }
+{ "n_nationkey": 1, "s_nationkey": 1, "c_nationkey": 1 }
+{ "n_nationkey": 1, "s_nationkey": 1, "c_nationkey": 1 }
+{ "n_nationkey": 5, "s_nationkey": 5, "c_nationkey": 5 }
+{ "n_nationkey": 5, "s_nationkey": 5, "c_nationkey": 5 }
+{ "n_nationkey": 5, "s_nationkey": 5, "c_nationkey": 5 }
+{ "n_nationkey": 5, "s_nationkey": 5, "c_nationkey": 5 }
+{ "n_nationkey": 5, "s_nationkey": 5, "c_nationkey": 5 }
+{ "n_nationkey": 5, "s_nationkey": 5, "c_nationkey": 5 }
+{ "n_nationkey": 10, "s_nationkey": 10, "c_nationkey": 10 }
+{ "n_nationkey": 10, "s_nationkey": 10, "c_nationkey": 10 }
+{ "n_nationkey": 10, "s_nationkey": 10, "c_nationkey": 10 }
+{ "n_nationkey": 10, "s_nationkey": 10, "c_nationkey": 10 }
+{ "n_nationkey": 10, "s_nationkey": 10, "c_nationkey": 10 }
+{ "n_nationkey": 10, "s_nationkey": 10, "c_nationkey": 10 }
+{ "n_nationkey": 10, "s_nationkey": 10, "c_nationkey": 10 }
+{ "n_nationkey": 10, "s_nationkey": 10, "c_nationkey": 10 }
+{ "n_nationkey": 11, "s_nationkey": 11, "c_nationkey": 11 }
+{ "n_nationkey": 11, "s_nationkey": 11, "c_nationkey": 11 }
+{ "n_nationkey": 11, "s_nationkey": 11, "c_nationkey": 11 }
+{ "n_nationkey": 11, "s_nationkey": 11, "c_nationkey": 11 }
+{ "n_nationkey": 11, "s_nationkey": 11, "c_nationkey": 11 }
+{ "n_nationkey": 14, "s_nationkey": 14, "c_nationkey": 14 }
+{ "n_nationkey": 14, "s_nationkey": 14, "c_nationkey": 14 }
+{ "n_nationkey": 15, "s_nationkey": 15, "c_nationkey": 15 }
+{ "n_nationkey": 15, "s_nationkey": 15, "c_nationkey": 15 }
+{ "n_nationkey": 15, "s_nationkey": 15, "c_nationkey": 15 }
+{ "n_nationkey": 15, "s_nationkey": 15, "c_nationkey": 15 }
+{ "n_nationkey": 15, "s_nationkey": 15, "c_nationkey": 15 }
+{ "n_nationkey": 15, "s_nationkey": 15, "c_nationkey": 15 }
+{ "n_nationkey": 15, "s_nationkey": 15, "c_nationkey": 15 }
+{ "n_nationkey": 15, "s_nationkey": 15, "c_nationkey": 15 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 17, "s_nationkey": 17, "c_nationkey": 17 }
+{ "n_nationkey": 23, "s_nationkey": 23, "c_nationkey": 23 }
+{ "n_nationkey": 23, "s_nationkey": 23, "c_nationkey": 23 }
+{ "n_nationkey": 23, "s_nationkey": 23, "c_nationkey": 23 }
+{ "n_nationkey": 23, "s_nationkey": 23, "c_nationkey": 23 }
+{ "n_nationkey": 23, "s_nationkey": 23, "c_nationkey": 23 }
+{ "n_nationkey": 24, "s_nationkey": 24, "c_nationkey": 24 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.16.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.16.plan
new file mode 100644
index 0000000..c492146
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.16.plan
@@ -0,0 +1,66 @@
+distribute result [$$48] [cardinality: 2.1, op-cost: 0.0, total-cost: 1.0E7]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 1.0E7]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$48]) [cardinality: 2.1, op-cost: 0.0, total-cost: 1.0E7]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$48] <- [{"n_nationkey": $$56, "s_nationkey": $$53, "c_nationkey": $$52}] [cardinality: 2.1, op-cost: 0.0, total-cost: 1.0E7]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 1.0E7]
+ -- SORT_MERGE_EXCHANGE [$$56(ASC), $$53(ASC), $$52(ASC) ] |PARTITIONED|
+ order (ASC, $$56) (ASC, $$53) (ASC, $$52) [cardinality: 2.1, op-cost: 0.0, total-cost: 1.0E7]
+ -- STABLE_SORT [$$56(ASC), $$53(ASC), $$52(ASC)] |PARTITIONED|
+ exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 1.0E7]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$56, $$53, $$52]) [cardinality: 2.1, op-cost: 0.0, total-cost: 1.0E7]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 1.0E7]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ join (and(eq($$52, $$56), eq($$53, $$63))) [cardinality: 2.1, op-cost: 2000000.0, total-cost: 1.0E7]
+ -- HYBRID_HASH_JOIN [$$56, $$53][$$52, $$63] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 6000000.0]
+ -- HASH_PARTITION_EXCHANGE [$$56, $$53] |PARTITIONED|
+ project ([$$53, $$56]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 5000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 6000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ unnest-map [$$56, $$n] <- index-search("Nation", 0, "tpch", "Nation", true, true, 1, $$53, 1, $$53, true, true, true) [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 5000000.0]
+ -- BTREE_SEARCH |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ order (ASC, $$53) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STABLE_SORT [$$53(ASC)] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- HASH_PARTITION_EXCHANGE [$$53] |PARTITIONED|
+ project ([$$53]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$53] <- [$$s.getField(3)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$s]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$55, $$s] <- tpch.Supplier [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
+ -- HASH_PARTITION_EXCHANGE [$$52, $$63] |PARTITIONED|
+ assign [$$63] <- [$$52] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$52]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$52] <- [$$c.getField(3)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$c]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$57, $$c] <- tpch.Customer [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/join/inner_right_corr/inner_right_corr.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/inner_right_corr/inner_right_corr.3.adm
new file mode 100644
index 0000000..2b2783f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/inner_right_corr/inner_right_corr.3.adm
@@ -0,0 +1,2 @@
+{ "a": [ 3 ] }
+{ "a": [ 1 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/join/nlj_partitioning_property/nlj_partitioning_property.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/nlj_partitioning_property/nlj_partitioning_property.3.adm
new file mode 100644
index 0000000..e9eb380
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/nlj_partitioning_property/nlj_partitioning_property.3.adm
@@ -0,0 +1 @@
+{ "count": 159800 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/join/nlj_partitioning_property/nlj_partitioning_property.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/nlj_partitioning_property/nlj_partitioning_property.4.adm
new file mode 100644
index 0000000..e9eb380
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/nlj_partitioning_property/nlj_partitioning_property.4.adm
@@ -0,0 +1 @@
+{ "count": 159800 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/join/nlj_partitioning_property/nlj_partitioning_property.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/nlj_partitioning_property/nlj_partitioning_property.5.adm
new file mode 100644
index 0000000..f3142ea
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/nlj_partitioning_property/nlj_partitioning_property.5.adm
@@ -0,0 +1 @@
+{ "count": 1600000 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/limit-non-pure-function/limit-non-pure-function.1.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/limit-non-pure-function/limit-non-pure-function.1.plan
new file mode 100644
index 0000000..d5f6401
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/limit-non-pure-function/limit-non-pure-function.1.plan
@@ -0,0 +1,14 @@
+distribute result [$$A]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ project ([$$A])
+ -- STREAM_PROJECT |UNPARTITIONED|
+ limit switch-case(gt($$13, 0), true, $$13, 0)
+ -- STREAM_LIMIT |UNPARTITIONED|
+ assign [$$13] <- [treat-as-integer(random())]
+ -- ASSIGN |UNPARTITIONED|
+ unnest $$A <- scan-collection(array: [ 1, 2, 3 ])
+ -- UNNEST |UNPARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/limit-non-pure-function/limit-non-pure-function.2.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/limit-non-pure-function/limit-non-pure-function.2.plan
new file mode 100644
index 0000000..54efcb6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/limit-non-pure-function/limit-non-pure-function.2.plan
@@ -0,0 +1,18 @@
+distribute result [$$24]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ project ([$$24])
+ -- STREAM_PROJECT |UNPARTITIONED|
+ limit switch-case(gt($$25, 0), true, $$25, 0)
+ -- STREAM_LIMIT |UNPARTITIONED|
+ project ([$$25, $$24])
+ -- STREAM_PROJECT |UNPARTITIONED|
+ assign [$$25, $$24] <- [treat-as-integer($$21), {"A": $$A, "rand": $$21}]
+ -- ASSIGN |UNPARTITIONED|
+ unnest $$A <- scan-collection(array: [ 1, 2, 3 ])
+ -- UNNEST |UNPARTITIONED|
+ assign [$$21] <- [random()]
+ -- ASSIGN |UNPARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/offset_without_limit/offset_without_limit.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/offset_without_limit/offset_without_limit.6.adm
deleted file mode 100644
index 726ee49..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/offset_without_limit/offset_without_limit.6.adm
+++ /dev/null
@@ -1,18 +0,0 @@
-distribute result [$$15]
--- DISTRIBUTE_RESULT |UNPARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- limit offset 98
- -- STREAM_LIMIT |UNPARTITIONED|
- project ([$$15])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$15] <- [{"id": $$17, "dblpid": $$paper.getField(1)}]
- -- ASSIGN |PARTITIONED|
- exchange
- -- SORT_MERGE_EXCHANGE [$$17(ASC) ] |PARTITIONED|
- data-scan []<-[$$17, $$paper] <- test.DBLP1
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/offset_without_limit/offset_without_limit.6.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/offset_without_limit/offset_without_limit.6.plan
new file mode 100644
index 0000000..0712c07
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/offset_without_limit/offset_without_limit.6.plan
@@ -0,0 +1,18 @@
+distribute result [$$15] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ limit offset 98 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_LIMIT |UNPARTITIONED|
+ project ([$$15]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$15] <- [{"id": $$17, "dblpid": $$paper.getField(1)}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SORT_MERGE_EXCHANGE [$$17(ASC) ] |PARTITIONED|
+ data-scan []<-[$$17, $$paper] <- test.DBLP1 [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-external-scan-select/push-limit-to-external-scan-select.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-external-scan-select/push-limit-to-external-scan-select.2.adm
deleted file mode 100644
index b38ed8b..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-external-scan-select/push-limit-to-external-scan-select.2.adm
+++ /dev/null
@@ -1,22 +0,0 @@
-distribute result [$$16]
--- DISTRIBUTE_RESULT |UNPARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- limit 5
- -- STREAM_LIMIT |UNPARTITIONED|
- exchange
- -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
- limit 5
- -- STREAM_LIMIT |PARTITIONED|
- project ([$$16])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$16] <- [$$t.getField(0)]
- -- ASSIGN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$t] <- test.ds1 condition (gt($$t.getField(0), 2)) limit 5
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-external-scan-select/push-limit-to-external-scan-select.2.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-external-scan-select/push-limit-to-external-scan-select.2.plan
new file mode 100644
index 0000000..4bbde8c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-external-scan-select/push-limit-to-external-scan-select.2.plan
@@ -0,0 +1,22 @@
+distribute result [$$16] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ limit 5 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_LIMIT |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ limit 5 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_LIMIT |PARTITIONED|
+ project ([$$16]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$16] <- [$$t.getField(0)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$t] <- test.ds1 condition (gt($$t.getField(0), 2)) limit 5 [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-external-scan/push-limit-to-external-scan.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-external-scan/push-limit-to-external-scan.2.adm
deleted file mode 100644
index 2c98237..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-external-scan/push-limit-to-external-scan.2.adm
+++ /dev/null
@@ -1,22 +0,0 @@
-distribute result [$$13]
--- DISTRIBUTE_RESULT |UNPARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- limit 5
- -- STREAM_LIMIT |UNPARTITIONED|
- exchange
- -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
- project ([$$13])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$13] <- [$$t.getField(0)]
- -- ASSIGN |PARTITIONED|
- limit 5
- -- STREAM_LIMIT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$t] <- test.ds1 limit 5
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-external-scan/push-limit-to-external-scan.2.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-external-scan/push-limit-to-external-scan.2.plan
new file mode 100644
index 0000000..e06e622
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-external-scan/push-limit-to-external-scan.2.plan
@@ -0,0 +1,22 @@
+distribute result [$$13] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ limit 5 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_LIMIT |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ project ([$$13]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$13] <- [$$t.getField(0)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ limit 5 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_LIMIT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$t] <- test.ds1 limit 5 [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.3.adm
deleted file mode 100644
index 3543f5d..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.3.adm
+++ /dev/null
@@ -1,34 +0,0 @@
-distribute result [$$c]
--- DISTRIBUTE_RESULT |UNPARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- limit 5 offset 5
- -- STREAM_LIMIT |UNPARTITIONED|
- project ([$$c])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- SORT_MERGE_EXCHANGE [$$17(ASC), $$18(ASC) ] |PARTITIONED|
- limit 10
- -- STREAM_LIMIT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- unnest-map [$$17, $$18, $$c] <- index-search("LineItem", 0, "test", "LineItem", FALSE, FALSE, 2, $$23, $$24, 2, $$23, $$24, TRUE, TRUE, TRUE) condition (and(lt($$c.getField(2), 150), lt($$c.getField(5), 10000))) limit 10
- -- BTREE_SEARCH |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- order (ASC, $$23) (ASC, $$24)
- -- STABLE_SORT [$$23(ASC), $$24(ASC)] |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$23, $$24])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- unnest-map [$$22, $$23, $$24] <- index-search("idx_LineItem_suppkey", 0, "test", "LineItem", FALSE, FALSE, 0, 1, $$21, TRUE, FALSE, FALSE)
- -- BTREE_SEARCH |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- assign [$$21] <- [150]
- -- ASSIGN |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.3.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.3.plan
new file mode 100644
index 0000000..cf6f26e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.3.plan
@@ -0,0 +1,34 @@
+distribute result [$$c] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ limit 5 offset 5 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_LIMIT |UNPARTITIONED|
+ project ([$$c]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- SORT_MERGE_EXCHANGE [$$17(ASC), $$18(ASC) ] |PARTITIONED|
+ limit 10 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_LIMIT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ unnest-map [$$17, $$18, $$c] <- index-search("LineItem", 0, "test", "LineItem", false, false, 2, $$23, $$24, 2, $$23, $$24, true, true, true) condition (and(lt($$c.getField(2), 150), lt($$c.getField(5), 10000))) limit 10 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- BTREE_SEARCH |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ order (ASC, $$23) (ASC, $$24) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STABLE_SORT [$$23(ASC), $$24(ASC)] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$23, $$24]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ unnest-map [$$22, $$23, $$24] <- index-search("idx_LineItem_suppkey", 0, "test", "LineItem", false, false, 0, 1, $$21, true, false, false) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- BTREE_SEARCH |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ assign [$$21] <- [150] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.5.adm
deleted file mode 100644
index 1f4ef66..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.5.adm
+++ /dev/null
@@ -1,40 +0,0 @@
-distribute result [$$19]
--- DISTRIBUTE_RESULT |UNPARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- limit 5
- -- STREAM_LIMIT |UNPARTITIONED|
- project ([$$19])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- SORT_MERGE_EXCHANGE [$$21(ASC), $$22(ASC) ] |PARTITIONED|
- project ([$$21, $$22, $$19])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$19] <- [{"shipdate": substring($$c.getField(10), 0, 4), "suppkey": gt($$20, 0)}]
- -- ASSIGN |PARTITIONED|
- limit 5
- -- STREAM_LIMIT |PARTITIONED|
- assign [$$20] <- [$$c.getField(2)]
- -- ASSIGN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- unnest-map [$$21, $$22, $$c] <- index-search("LineItem", 0, "test", "LineItem", FALSE, FALSE, 2, $$26, $$27, 2, $$26, $$27, TRUE, TRUE, TRUE) condition (lt($$c.getField(2), 150)) limit 5
- -- BTREE_SEARCH |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- order (ASC, $$26) (ASC, $$27)
- -- STABLE_SORT [$$26(ASC), $$27(ASC)] |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$26, $$27])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- unnest-map [$$25, $$26, $$27] <- index-search("idx_LineItem_suppkey", 0, "test", "LineItem", FALSE, FALSE, 0, 1, $$24, TRUE, FALSE, FALSE)
- -- BTREE_SEARCH |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- assign [$$24] <- [150]
- -- ASSIGN |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.5.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.5.plan
new file mode 100644
index 0000000..818e376
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.5.plan
@@ -0,0 +1,40 @@
+distribute result [$$19] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ limit 5 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_LIMIT |UNPARTITIONED|
+ project ([$$19]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- SORT_MERGE_EXCHANGE [$$21(ASC), $$22(ASC) ] |PARTITIONED|
+ project ([$$21, $$22, $$19]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$19] <- [{"shipdate": substring($$c.getField(10), 0, 4), "suppkey": gt($$20, 0)}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ limit 5 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_LIMIT |PARTITIONED|
+ assign [$$20] <- [$$c.getField(2)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ unnest-map [$$21, $$22, $$c] <- index-search("LineItem", 0, "test", "LineItem", false, false, 2, $$26, $$27, 2, $$26, $$27, true, true, true) condition (lt($$c.getField(2), 150)) limit 5 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- BTREE_SEARCH |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ order (ASC, $$26) (ASC, $$27) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STABLE_SORT [$$26(ASC), $$27(ASC)] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$26, $$27]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ unnest-map [$$25, $$26, $$27] <- index-search("idx_LineItem_suppkey", 0, "test", "LineItem", false, false, 0, 1, $$24, true, false, false) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- BTREE_SEARCH |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ assign [$$24] <- [150] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.3.adm
deleted file mode 100644
index a0a3c84..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.3.adm
+++ /dev/null
@@ -1,34 +0,0 @@
-distribute result [$$c]
--- DISTRIBUTE_RESULT |UNPARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- limit 5 offset 5
- -- STREAM_LIMIT |UNPARTITIONED|
- project ([$$c])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- SORT_MERGE_EXCHANGE [$$14(ASC), $$15(ASC) ] |PARTITIONED|
- limit 10
- -- STREAM_LIMIT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- unnest-map [$$14, $$15, $$c] <- index-search("LineItem", 0, "test", "LineItem", FALSE, FALSE, 2, $$19, $$20, 2, $$19, $$20, TRUE, TRUE, TRUE) condition (lt($$c.getField(2), 150)) limit 10
- -- BTREE_SEARCH |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- order (ASC, $$19) (ASC, $$20)
- -- STABLE_SORT [$$19(ASC), $$20(ASC)] |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$19, $$20])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- unnest-map [$$18, $$19, $$20] <- index-search("idx_LineItem_suppkey", 0, "test", "LineItem", FALSE, FALSE, 0, 1, $$17, TRUE, FALSE, FALSE)
- -- BTREE_SEARCH |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- assign [$$17] <- [150]
- -- ASSIGN |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.3.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.3.plan
new file mode 100644
index 0000000..968ced8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.3.plan
@@ -0,0 +1,34 @@
+distribute result [$$c] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ limit 5 offset 5 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_LIMIT |UNPARTITIONED|
+ project ([$$c]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- SORT_MERGE_EXCHANGE [$$14(ASC), $$15(ASC) ] |PARTITIONED|
+ limit 10 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_LIMIT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ unnest-map [$$14, $$15, $$c] <- index-search("LineItem", 0, "test", "LineItem", false, false, 2, $$19, $$20, 2, $$19, $$20, true, true, true) condition (lt($$c.getField(2), 150)) limit 10 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- BTREE_SEARCH |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ order (ASC, $$19) (ASC, $$20) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STABLE_SORT [$$19(ASC), $$20(ASC)] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$19, $$20]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ unnest-map [$$18, $$19, $$20] <- index-search("idx_LineItem_suppkey", 0, "test", "LineItem", false, false, 0, 1, $$17, true, false, false) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- BTREE_SEARCH |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ assign [$$17] <- [150] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.5.adm
deleted file mode 100644
index 44507f4..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.5.adm
+++ /dev/null
@@ -1,34 +0,0 @@
-distribute result [$$c]
--- DISTRIBUTE_RESULT |UNPARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- limit 5 offset 5
- -- STREAM_LIMIT |UNPARTITIONED|
- project ([$$c])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- SORT_MERGE_EXCHANGE [$$16(ASC), $$17(ASC) ] |PARTITIONED|
- limit 10
- -- STREAM_LIMIT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- unnest-map [$$16, $$17, $$c] <- index-search("LineItem", 0, "test", "LineItem", FALSE, FALSE, 2, $$21, $$22, 2, $$21, $$22, TRUE, TRUE, TRUE) condition (lt($$c.getField(2), 150)) limit 10
- -- BTREE_SEARCH |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- order (ASC, $$21) (ASC, $$22)
- -- STABLE_SORT [$$21(ASC), $$22(ASC)] |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$21, $$22])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- unnest-map [$$20, $$21, $$22] <- index-search("idx_LineItem_suppkey", 0, "test", "LineItem", FALSE, FALSE, 0, 1, $$19, TRUE, FALSE, FALSE)
- -- BTREE_SEARCH |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- assign [$$19] <- [150]
- -- ASSIGN |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.5.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.5.plan
new file mode 100644
index 0000000..cacdbb8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.5.plan
@@ -0,0 +1,34 @@
+distribute result [$$c] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ limit 5 offset 5 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_LIMIT |UNPARTITIONED|
+ project ([$$c]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- SORT_MERGE_EXCHANGE [$$16(ASC), $$17(ASC) ] |PARTITIONED|
+ limit 10 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_LIMIT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ unnest-map [$$16, $$17, $$c] <- index-search("LineItem", 0, "test", "LineItem", false, false, 2, $$21, $$22, 2, $$21, $$22, true, true, true) condition (lt($$c.getField(2), 150)) limit 10 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- BTREE_SEARCH |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ order (ASC, $$21) (ASC, $$22) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STABLE_SORT [$$21(ASC), $$22(ASC)] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$21, $$22]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ unnest-map [$$20, $$21, $$22] <- index-search("idx_LineItem_suppkey", 0, "test", "LineItem", false, false, 0, 1, $$19, true, false, false) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- BTREE_SEARCH |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ assign [$$19] <- [150] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.11.adm
deleted file mode 100644
index 28c74ac..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.11.adm
+++ /dev/null
@@ -1,52 +0,0 @@
-distribute result [$$202]
--- DISTRIBUTE_RESULT |LOCAL|
- exchange
- -- ONE_TO_ONE_EXCHANGE |LOCAL|
- aggregate [$$202] <- [agg-sql-sum($$231)]
- -- AGGREGATE |LOCAL|
- aggregate [$$231] <- [agg-sql-count(1)]
- -- AGGREGATE |LOCAL|
- exchange
- -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- union
- -- UNION_ALL |UNPARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- limit 1000
- -- STREAM_LIMIT |UNPARTITIONED|
- project ([])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- SORT_MERGE_EXCHANGE [$$134(ASC) ] |PARTITIONED|
- limit 1000
- -- STREAM_LIMIT |PARTITIONED|
- project ([$$134])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$134, $$onek1] <- test.onek1 condition (and(ge($$onek1.getField(2), 1), le($$onek1.getField(2), 10))) limit 1000
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- limit 1000
- -- STREAM_LIMIT |UNPARTITIONED|
- project ([])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- SORT_MERGE_EXCHANGE [$$135(ASC) ] |PARTITIONED|
- limit 1000
- -- STREAM_LIMIT |PARTITIONED|
- project ([$$135])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$135, $$onek1] <- test.onek1 condition (and(ge($$onek1.getField(2), -10), le($$onek1.getField(2), -1))) limit 1000
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.11.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.11.plan
new file mode 100644
index 0000000..e8b44d5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.11.plan
@@ -0,0 +1,52 @@
+distribute result [$$202] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+-- DISTRIBUTE_RESULT |LOCAL|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |LOCAL|
+ aggregate [$$202] <- [agg-sql-sum($$231)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- AGGREGATE |LOCAL|
+ aggregate [$$231] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- AGGREGATE |LOCAL|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ union [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- UNION_ALL |UNPARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ limit 1000 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_LIMIT |UNPARTITIONED|
+ project ([]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- SORT_MERGE_EXCHANGE [$$134(ASC) ] |PARTITIONED|
+ limit 1000 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_LIMIT |PARTITIONED|
+ project ([$$134]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$134, $$onek1] <- test.onek1 condition (and(ge($$onek1.getField(2), 1), le($$onek1.getField(2), 10))) limit 1000 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ limit 1000 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_LIMIT |UNPARTITIONED|
+ project ([]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- SORT_MERGE_EXCHANGE [$$135(ASC) ] |PARTITIONED|
+ limit 1000 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_LIMIT |PARTITIONED|
+ project ([$$135]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$135, $$onek1] <- test.onek1 condition (and(ge($$onek1.getField(2), -10), le($$onek1.getField(2), -1))) limit 1000 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.3.adm
deleted file mode 100644
index db1c3d8..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.3.adm
+++ /dev/null
@@ -1,20 +0,0 @@
-distribute result [$$paper]
--- DISTRIBUTE_RESULT |UNPARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- limit 5 offset 5
- -- STREAM_LIMIT |UNPARTITIONED|
- project ([$$paper])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- SORT_MERGE_EXCHANGE [$$14(ASC) ] |PARTITIONED|
- limit 10
- -- STREAM_LIMIT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$14, $$paper] <- test.DBLP1 condition (contains($$paper.getField(1), "kimL89")) limit 10
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.3.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.3.plan
new file mode 100644
index 0000000..25c2869
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.3.plan
@@ -0,0 +1,20 @@
+distribute result [$$paper] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ limit 5 offset 5 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_LIMIT |UNPARTITIONED|
+ project ([$$paper]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SORT_MERGE_EXCHANGE [$$14(ASC) ] |PARTITIONED|
+ limit 10 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_LIMIT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$14, $$paper] <- test.DBLP1 condition (contains($$paper.getField(1), "kimL89")) limit 10 [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.5.adm
deleted file mode 100644
index 957c52b..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.5.adm
+++ /dev/null
@@ -1,64 +0,0 @@
-distribute result [$$35]
--- DISTRIBUTE_RESULT |UNPARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- limit 2
- -- STREAM_LIMIT |UNPARTITIONED|
- exchange
- -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
- project ([$$35])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$35] <- [{"dblpid": $$36}]
- -- ASSIGN |PARTITIONED|
- limit 2
- -- STREAM_LIMIT |PARTITIONED|
- project ([$$36])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- join (eq($$36, $$39))
- -- HYBRID_HASH_JOIN [$$36][$$39] |PARTITIONED|
- exchange
- -- HASH_PARTITION_EXCHANGE [$$36] |PARTITIONED|
- project ([$$36])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$36] <- [$$d.getField(1)]
- -- ASSIGN |PARTITIONED|
- project ([$$d])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$37, $$d] <- test.DBLP1
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- exchange
- -- HASH_PARTITION_EXCHANGE [$$39] |PARTITIONED|
- project ([$$39])
- -- STREAM_PROJECT |UNPARTITIONED|
- assign [$$39] <- [get-item($$28, 0).getField(0).getField(1)]
- -- ASSIGN |UNPARTITIONED|
- aggregate [$$28] <- [listify($$27)]
- -- AGGREGATE |UNPARTITIONED|
- limit 1
- -- STREAM_LIMIT |UNPARTITIONED|
- project ([$$27])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- SORT_MERGE_EXCHANGE [$$38(ASC) ] |PARTITIONED|
- project ([$$38, $$27])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$27] <- [{"d": $$d}]
- -- ASSIGN |PARTITIONED|
- limit 1
- -- STREAM_LIMIT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$38, $$d] <- test.DBLP1 condition (ends-with($$d.getField(1), "Blakeley95")) limit 1
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.5.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.5.plan
new file mode 100644
index 0000000..5beb697
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.5.plan
@@ -0,0 +1,64 @@
+distribute result [$$35] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ limit 2 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- STREAM_LIMIT |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ project ([$$35]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$35] <- [{"dblpid": $$36}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- ASSIGN |PARTITIONED|
+ limit 2 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- STREAM_LIMIT |PARTITIONED|
+ project ([$$36]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ join (eq($$36, $$39)) [cardinality: 1000000.0, op-cost: 2000000.0, total-cost: 6000000.0]
+ -- HYBRID_HASH_JOIN [$$36][$$39] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
+ -- HASH_PARTITION_EXCHANGE [$$36] |PARTITIONED|
+ project ([$$36]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$36] <- [$$d.getField(1)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$d]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$37, $$d] <- test.DBLP1 [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
+ -- HASH_PARTITION_EXCHANGE [$$39] |PARTITIONED|
+ project ([$$39]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |UNPARTITIONED|
+ assign [$$39] <- [get-item($$28, 0).getField(0).getField(1)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |UNPARTITIONED|
+ aggregate [$$28] <- [listify($$27)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- AGGREGATE |UNPARTITIONED|
+ limit 1 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_LIMIT |UNPARTITIONED|
+ project ([$$27]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
+ -- SORT_MERGE_EXCHANGE [$$38(ASC) ] |PARTITIONED|
+ project ([$$38, $$27]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$27] <- [{"d": $$d}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ limit 1 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_LIMIT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$38, $$d] <- test.DBLP1 condition (ends-with($$d.getField(1), "Blakeley95")) limit 1 [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.6.adm
deleted file mode 100644
index 7509819..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.6.adm
+++ /dev/null
@@ -1,28 +0,0 @@
-distribute result [$$18]
--- DISTRIBUTE_RESULT |UNPARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- limit 1
- -- STREAM_LIMIT |UNPARTITIONED|
- project ([$$18])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- SORT_MERGE_EXCHANGE [$$20(ASC) ] |PARTITIONED|
- project ([$$20, $$18])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$18] <- [{"$1": substring($$19, 0, 21)}]
- -- ASSIGN |PARTITIONED|
- limit 1
- -- STREAM_LIMIT |PARTITIONED|
- project ([$$20, $$19])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$19] <- [$$DBLP1.getField(1)]
- -- ASSIGN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$20, $$DBLP1] <- test.DBLP1 condition (gt($$DBLP1.getField(1), "series")) limit 1
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.6.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.6.plan
new file mode 100644
index 0000000..46283c5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.6.plan
@@ -0,0 +1,28 @@
+distribute result [$$18] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ limit 1 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_LIMIT |UNPARTITIONED|
+ project ([$$18]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SORT_MERGE_EXCHANGE [$$20(ASC) ] |PARTITIONED|
+ project ([$$20, $$18]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$18] <- [{"$1": substring($$19, 0, 21)}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ limit 1 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_LIMIT |PARTITIONED|
+ project ([$$20, $$19]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$19] <- [$$DBLP1.getField(1)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$20, $$DBLP1] <- test.DBLP1 condition (gt($$DBLP1.getField(1), "series")) limit 1 [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.8.adm
deleted file mode 100644
index 5b408ad..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.8.adm
+++ /dev/null
@@ -1,28 +0,0 @@
-distribute result [$$21]
--- DISTRIBUTE_RESULT |UNPARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- limit 2
- -- STREAM_LIMIT |UNPARTITIONED|
- project ([$$21])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- SORT_MERGE_EXCHANGE [$$24(ASC) ] |PARTITIONED|
- limit 2
- -- STREAM_LIMIT |PARTITIONED|
- project ([$$24, $$21])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$21] <- [$$25.getField("lang")]
- -- ASSIGN |PARTITIONED|
- project ([$$24, $$25])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$25] <- [$$t.getField("user")]
- -- ASSIGN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$24, $$t] <- test.TweetMessages condition (and(ge($$t.getField("user").getField("friends_count"), 0), le($$t.getField("user").getField("friends_count"), 150))) limit 2
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.8.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.8.plan
new file mode 100644
index 0000000..06387be
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.8.plan
@@ -0,0 +1,28 @@
+distribute result [$$21] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ limit 2 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_LIMIT |UNPARTITIONED|
+ project ([$$21]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SORT_MERGE_EXCHANGE [$$24(ASC) ] |PARTITIONED|
+ limit 2 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_LIMIT |PARTITIONED|
+ project ([$$24, $$21]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$21] <- [$$25.getField("lang")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$24, $$25]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$25] <- [$$t.getField("user")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$24, $$t] <- test.TweetMessages condition (and(ge($$t.getField("user").getField("friends_count"), 0), le($$t.getField("user").getField("friends_count"), 150))) limit 2 [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.3.adm
deleted file mode 100644
index aaf0c53..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.3.adm
+++ /dev/null
@@ -1,20 +0,0 @@
-distribute result [$$paper]
--- DISTRIBUTE_RESULT |UNPARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- limit 5 offset 5
- -- STREAM_LIMIT |UNPARTITIONED|
- project ([$$paper])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- SORT_MERGE_EXCHANGE [$$12(ASC) ] |PARTITIONED|
- limit 10
- -- STREAM_LIMIT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$12, $$paper] <- test.DBLP1 limit 10
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.3.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.3.plan
new file mode 100644
index 0000000..3e40aa9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.3.plan
@@ -0,0 +1,20 @@
+distribute result [$$paper] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ limit 5 offset 5 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_LIMIT |UNPARTITIONED|
+ project ([$$paper]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SORT_MERGE_EXCHANGE [$$12(ASC) ] |PARTITIONED|
+ limit 10 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_LIMIT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$12, $$paper] <- test.DBLP1 limit 10 [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.5.adm
deleted file mode 100644
index 2176e36..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.5.adm
+++ /dev/null
@@ -1,20 +0,0 @@
-distribute result [$$paper]
--- DISTRIBUTE_RESULT |UNPARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- limit 5 offset 5
- -- STREAM_LIMIT |UNPARTITIONED|
- project ([$$paper])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- SORT_MERGE_EXCHANGE [$$14(ASC) ] |PARTITIONED|
- limit 10
- -- STREAM_LIMIT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$14, $$paper] <- test.DBLP1 limit 10
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.5.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.5.plan
new file mode 100644
index 0000000..fa06729
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.5.plan
@@ -0,0 +1,20 @@
+distribute result [$$paper] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ limit 5 offset 5 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_LIMIT |UNPARTITIONED|
+ project ([$$paper]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SORT_MERGE_EXCHANGE [$$14(ASC) ] |PARTITIONED|
+ limit 10 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_LIMIT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$14, $$paper] <- test.DBLP1 limit 10 [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.7.adm
deleted file mode 100644
index f8a800a..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.7.adm
+++ /dev/null
@@ -1,52 +0,0 @@
-distribute result [$$180]
--- DISTRIBUTE_RESULT |LOCAL|
- exchange
- -- ONE_TO_ONE_EXCHANGE |LOCAL|
- aggregate [$$180] <- [agg-sql-sum($$205)]
- -- AGGREGATE |LOCAL|
- aggregate [$$205] <- [agg-sql-count(1)]
- -- AGGREGATE |LOCAL|
- exchange
- -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- union
- -- UNION_ALL |UNPARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- limit 100
- -- STREAM_LIMIT |UNPARTITIONED|
- project ([])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- SORT_MERGE_EXCHANGE [$$120(ASC) ] |PARTITIONED|
- limit 100
- -- STREAM_LIMIT |PARTITIONED|
- project ([$$120])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$120, $$onek1] <- test.onek1 limit 100
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- limit 10
- -- STREAM_LIMIT |UNPARTITIONED|
- project ([])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- SORT_MERGE_EXCHANGE [$$121(ASC) ] |PARTITIONED|
- limit 10
- -- STREAM_LIMIT |PARTITIONED|
- project ([$$121])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$121, $$onek1] <- test.onek1 limit 10
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.7.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.7.plan
new file mode 100644
index 0000000..81a8266
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.7.plan
@@ -0,0 +1,52 @@
+distribute result [$$180] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+-- DISTRIBUTE_RESULT |LOCAL|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |LOCAL|
+ aggregate [$$180] <- [agg-sql-sum($$205)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- AGGREGATE |LOCAL|
+ aggregate [$$205] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- AGGREGATE |LOCAL|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ union [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- UNION_ALL |UNPARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ limit 100 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_LIMIT |UNPARTITIONED|
+ project ([]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- SORT_MERGE_EXCHANGE [$$120(ASC) ] |PARTITIONED|
+ limit 100 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_LIMIT |PARTITIONED|
+ project ([$$120]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$120, $$onek1] <- test.onek1 limit 100 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ limit 10 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_LIMIT |UNPARTITIONED|
+ project ([]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- SORT_MERGE_EXCHANGE [$$121(ASC) ] |PARTITIONED|
+ limit 10 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_LIMIT |PARTITIONED|
+ project ([$$121]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$121, $$onek1] <- test.onek1 limit 10 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.8.adm
deleted file mode 100644
index bb5ac24..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.8.adm
+++ /dev/null
@@ -1,37 +0,0 @@
-distribute result [$$75]
--- DISTRIBUTE_RESULT |UNPARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- limit 5 offset 5
- -- STREAM_LIMIT |UNPARTITIONED|
- project ([$$75])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$75] <- [get-item($$73, 0)]
- -- ASSIGN |PARTITIONED|
- project ([$$73])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- SORT_MERGE_EXCHANGE [$$77(ASC) ] |PARTITIONED|
- project ([$$73, $$77])
- -- STREAM_PROJECT |PARTITIONED|
- subplan {
- aggregate [$$73] <- [listify($$72)]
- -- AGGREGATE |LOCAL|
- assign [$$72] <- [object-remove(object-remove(object-remove($$t0, "title"), "authors"), "misc")]
- -- ASSIGN |LOCAL|
- unnest $$t0 <- scan-collection(to-array($$paper))
- -- UNNEST |LOCAL|
- nested tuple source
- -- NESTED_TUPLE_SOURCE |LOCAL|
- }
- -- SUBPLAN |PARTITIONED|
- limit 10
- -- STREAM_LIMIT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$77, $$paper] <- test.DBLP1 limit 10
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.8.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.8.plan
new file mode 100644
index 0000000..0231545
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.8.plan
@@ -0,0 +1,37 @@
+distribute result [$$75] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ limit 5 offset 5 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_LIMIT |UNPARTITIONED|
+ project ([$$75]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$75] <- [get-item($$73, 0)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$73]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SORT_MERGE_EXCHANGE [$$77(ASC) ] |PARTITIONED|
+ project ([$$73, $$77]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ subplan {
+ aggregate [$$73] <- [listify($$72)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- AGGREGATE |LOCAL|
+ assign [$$72] <- [object-remove(object-remove(object-remove($$t0, "title"), "authors"), "misc")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |LOCAL|
+ unnest $$t0 <- scan-collection(to-array($$paper)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- UNNEST |LOCAL|
+ nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ } [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SUBPLAN |PARTITIONED|
+ limit 10 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_LIMIT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$77, $$paper] <- test.DBLP1 limit 10 [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/list/enforcing_item_type/enforcing_item_type.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/list/enforcing_item_type/enforcing_item_type.1.adm
deleted file mode 100644
index f0da628..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/list/enforcing_item_type/enforcing_item_type.1.adm
+++ /dev/null
@@ -1,12 +0,0 @@
-distribute result [$$20]
--- DISTRIBUTE_RESULT |UNPARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- project ([$$20])
- -- STREAM_PROJECT |UNPARTITIONED|
- assign [$$20] <- [{"id": get-item($$t, 0), "v": get-item($$t, 1)}]
- -- ASSIGN |UNPARTITIONED|
- unnest $$t <- scan-collection(ordered-list-constructor(ordered-list-constructor(29, cast({ f1: "a", f2: 3 }))))
- -- UNNEST |UNPARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/list/enforcing_item_type/enforcing_item_type.1.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/list/enforcing_item_type/enforcing_item_type.1.plan
new file mode 100644
index 0000000..bc910b3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/list/enforcing_item_type/enforcing_item_type.1.plan
@@ -0,0 +1,12 @@
+distribute result [$$20] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ project ([$$20]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |UNPARTITIONED|
+ assign [$$20] <- [{"id": get-item($$t, 0), "v": get-item($$t, 1)}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |UNPARTITIONED|
+ unnest $$t <- scan-collection(ordered-list-constructor(ordered-list-constructor(29, cast({ "f1": "a", "f2": 3 })))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- UNNEST |UNPARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/active_requests/active_requests.2.regex b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/active_requests/active_requests.2.regex
index e31fe3b..170838e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/active_requests/active_requests.2.regex
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/active_requests/active_requests.2.regex
@@ -1 +1 @@
-/\{ "cancellable": true, "jobId": "JID:.*", "state": "running", "uuid": ".*" \}/
\ No newline at end of file
+/\{ "cancellable": true, "jobId": "JID:.*", "state": "running", "uuid": ".*", "jobStatus": "RUNNING" \}/
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/active_requests/active_requests.4.regex b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/active_requests/active_requests.4.regex
index e2d166b..695472c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/active_requests/active_requests.4.regex
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/active_requests/active_requests.4.regex
@@ -1 +1 @@
-/memory\D+229376/
\ No newline at end of file
+/memory\D+0/
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/completed_requests/completed_requests.3.regex b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/completed_requests/completed_requests.3.regex
index e2d166b..695472c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/completed_requests/completed_requests.3.regex
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/completed_requests/completed_requests.3.regex
@@ -1 +1 @@
-/memory\D+229376/
\ No newline at end of file
+/memory\D+0/
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/completed_requests/completed_requests.4.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/completed_requests/completed_requests.4.regexjson
new file mode 100644
index 0000000..15affd1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/completed_requests/completed_requests.4.regexjson
@@ -0,0 +1,19 @@
+{
+ "cancellable": true,
+ "clientContextID": "completed_requests_query",
+ "elapsedTime": "R{.*}",
+ "jobCreateTime": "R{.*}",
+ "jobEndTime": "R{.*}",
+ "jobId": "R{.*}",
+ "jobQueueTime": "R{.*}",
+ "jobRequiredCPUs": "R{.*}",
+ "jobRequiredMemory": "R{.*}",
+ "jobStartTime": "R{.*}",
+ "jobStatus": "TERMINATED",
+ "node": "R{.*}",
+ "remoteAddr": "R{.*}",
+ "requestTime": "R{.*}",
+ "state": "completed",
+ "userAgent": "R{.*}",
+ "uuid": "R{.*}"
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.1.adm
deleted file mode 100644
index 379603f..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.1.adm
+++ /dev/null
@@ -1,8 +0,0 @@
-distribute result [$$5]
--- DISTRIBUTE_RESULT |UNPARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- assign [$$5] <- [{ f1: 5, f2: 6, f3: 7 }]
- -- ASSIGN |UNPARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.1.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.1.plan
new file mode 100644
index 0000000..609bfb1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.1.plan
@@ -0,0 +1,8 @@
+distribute result [$$5] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ assign [$$5] <- [{ "f1": 5, "f2": 6, "f3": 7 }] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |UNPARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.3.adm
deleted file mode 100644
index aafd2d0..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.3.adm
+++ /dev/null
@@ -1,18 +0,0 @@
-distribute result [$$14]
--- DISTRIBUTE_RESULT |UNPARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- assign [$$14] <- [TRUE]
- -- ASSIGN |UNPARTITIONED|
- project ([])
- -- STREAM_PROJECT |UNPARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- order (ASC, $$x)
- -- STABLE_SORT [$$x(ASC)] |UNPARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- unnest $$x <- range(1, 4)
- -- UNNEST |UNPARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.3.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.3.plan
new file mode 100644
index 0000000..b8a2fc4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.3.plan
@@ -0,0 +1,18 @@
+distribute result [$$14] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ assign [$$14] <- [true] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |UNPARTITIONED|
+ project ([]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |UNPARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ order (ASC, $$x) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STABLE_SORT [$$x(ASC)] |UNPARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ unnest $$x <- range(1, 4) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- UNNEST |UNPARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.5.adm
deleted file mode 100644
index 7209f96..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.5.adm
+++ /dev/null
@@ -1,18 +0,0 @@
-distribute result [$$14]
--- DISTRIBUTE_RESULT |UNPARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- project ([$$14])
- -- STREAM_PROJECT |UNPARTITIONED|
- assign [$$14] <- [le($$x, 2)]
- -- ASSIGN |UNPARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- order (ASC, $$x)
- -- STABLE_SORT [$$x(ASC)] |UNPARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- unnest $$x <- range(1, 4)
- -- UNNEST |UNPARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.5.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.5.plan
new file mode 100644
index 0000000..d4c5285
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.5.plan
@@ -0,0 +1,18 @@
+distribute result [$$14] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ project ([$$14]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |UNPARTITIONED|
+ assign [$$14] <- [le($$x, 2)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |UNPARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ order (ASC, $$x) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STABLE_SORT [$$x(ASC)] |UNPARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ unnest $$x <- range(1, 4) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- UNNEST |UNPARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.6.adm
deleted file mode 100644
index 351c8e5..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.6.adm
+++ /dev/null
@@ -1,18 +0,0 @@
-distribute result [$$14]
--- DISTRIBUTE_RESULT |UNPARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- project ([$$14])
- -- STREAM_PROJECT |UNPARTITIONED|
- assign [$$14] <- [or(null, le($$x, 2))]
- -- ASSIGN |UNPARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- order (ASC, $$x)
- -- STABLE_SORT [$$x(ASC)] |UNPARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- unnest $$x <- range(1, 4)
- -- UNNEST |UNPARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.6.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.6.plan
new file mode 100644
index 0000000..a97f959
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.6.plan
@@ -0,0 +1,18 @@
+distribute result [$$14] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ project ([$$14]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |UNPARTITIONED|
+ assign [$$14] <- [or(null, le($$x, 2))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |UNPARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ order (ASC, $$x) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STABLE_SORT [$$x(ASC)] |UNPARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ unnest $$x <- range(1, 4) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- UNNEST |UNPARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.7.adm
deleted file mode 100644
index d2f3187..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.7.adm
+++ /dev/null
@@ -1,18 +0,0 @@
-distribute result [$$16]
--- DISTRIBUTE_RESULT |UNPARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- project ([$$16])
- -- STREAM_PROJECT |UNPARTITIONED|
- assign [$$16] <- [or(TRUE, lt(get-year(current-date()), $$x))]
- -- ASSIGN |UNPARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- order (ASC, $$x)
- -- STABLE_SORT [$$x(ASC)] |UNPARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- unnest $$x <- range(1, 4)
- -- UNNEST |UNPARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.7.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.7.plan
new file mode 100644
index 0000000..3d3d289
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.7.plan
@@ -0,0 +1,18 @@
+distribute result [$$16] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ assign [$$16] <- [true] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |UNPARTITIONED|
+ project ([]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |UNPARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ order (ASC, $$x) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STABLE_SORT [$$x(ASC)] |UNPARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ unnest $$x <- range(1, 4) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- UNNEST |UNPARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/dump_index/dump_index.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/dump_index/dump_index.10.adm
new file mode 100644
index 0000000..d13e452
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/dump_index/dump_index.10.adm
@@ -0,0 +1 @@
+[ { "values": [ 1, { "id": 1, "age": "t\\'s", "a1": { "b": [ { "x": [ 1, 2 ] }, { "x": [ 1, 2 ] } ] }, "a2": [ { "x": [ 1, 2 ] }, { "x": [ 1, 2 ] } ] } ] } ]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/metadata_only_01/metadata_only_01.1.regex b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/metadata_only_01/metadata_only_01.1.regex
index 94ff0a4..959bc0d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/metadata_only_01/metadata_only_01.1.regex
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/metadata_only_01/metadata_only_01.1.regex
@@ -1 +1 @@
-/memory\D+917504/
\ No newline at end of file
+/memory\D+0/
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/metadata_only_01/metadata_only_01.2.regex b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/metadata_only_01/metadata_only_01.2.regex
index 6af6770..959bc0d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/metadata_only_01/metadata_only_01.2.regex
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/metadata_only_01/metadata_only_01.2.regex
@@ -1 +1 @@
-/memory\D+1146880/
\ No newline at end of file
+/memory\D+0/
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/metadata_only_02/metadata_only_02.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/metadata_only_02/metadata_only_02.3.adm
new file mode 100644
index 0000000..743bb8a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/metadata_only_02/metadata_only_02.3.adm
@@ -0,0 +1,2 @@
+{ "DataverseName": "test1", "SynonymName": "syn1" }
+{ "DataverseName": "test2", "SynonymName": "syn2" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-2550/query-ASTERIXDB-2886.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-2550/query-ASTERIXDB-2886.3.adm
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-2550/query-ASTERIXDB-2886.3.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-3316/query-ASTERIXDB-3316.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-3316/query-ASTERIXDB-3316.3.adm
new file mode 100644
index 0000000..190adc2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-3316/query-ASTERIXDB-3316.3.adm
@@ -0,0 +1 @@
+{ "sub_query1": [ { "u": "col1" } ], "sub_query2": [ { "u": "col1" } ], "x_id": 5 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-3334/query-ASTERIXDB-3334.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-3334/query-ASTERIXDB-3334.3.adm
new file mode 100644
index 0000000..e99dc40
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-3334/query-ASTERIXDB-3334.3.adm
@@ -0,0 +1 @@
+{ "Employee Name": "Purkey, Jan", "avg:Employee Salary:ok": 104677.0 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-3403/query-ASTERIXDB-3403.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-3403/query-ASTERIXDB-3403.3.adm
new file mode 100644
index 0000000..433b8ee
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-3403/query-ASTERIXDB-3403.3.adm
@@ -0,0 +1 @@
+{ "matches": 3 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-3418/query-ASTERIXDB-3418.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-3418/query-ASTERIXDB-3418.1.adm
new file mode 100644
index 0000000..f32a580
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-3418/query-ASTERIXDB-3418.1.adm
@@ -0,0 +1 @@
+true
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-3418/query-ASTERIXDB-3418.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-3418/query-ASTERIXDB-3418.2.adm
new file mode 100644
index 0000000..6a6d86c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-3418/query-ASTERIXDB-3418.2.adm
@@ -0,0 +1 @@
+{ "a": -0.0, "$1": 2 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-3418/query-ASTERIXDB-3418.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-3418/query-ASTERIXDB-3418.3.adm
new file mode 100644
index 0000000..767521e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-3418/query-ASTERIXDB-3418.3.adm
@@ -0,0 +1 @@
+{ "a": -0.0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-3418/query-ASTERIXDB-3418.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-3418/query-ASTERIXDB-3418.4.adm
new file mode 100644
index 0000000..02e4a84
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-3418/query-ASTERIXDB-3418.4.adm
@@ -0,0 +1 @@
+false
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-3490/query-ASTERIXDB-3490.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-3490/query-ASTERIXDB-3490.3.adm
new file mode 100644
index 0000000..22c03b0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-3490/query-ASTERIXDB-3490.3.adm
@@ -0,0 +1 @@
+{ "id": 1, "t1": 10, "t2": 10, "t": 10, "p1": 3, "p2": 3, "p": 3 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-3490/query-ASTERIXDB-3490.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-3490/query-ASTERIXDB-3490.4.adm
new file mode 100644
index 0000000..a617f17
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-3490/query-ASTERIXDB-3490.4.adm
@@ -0,0 +1 @@
+{ "b_max": 20, "ROW_NUMBER": 1, "x_id": 10 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-3512/query-ASTERIXDB-3512.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-3512/query-ASTERIXDB-3512.3.adm
new file mode 100644
index 0000000..9ea757d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-3512/query-ASTERIXDB-3512.3.adm
@@ -0,0 +1,2 @@
+{ "sym": 10 }
+{ "sym": 10 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/remove_listify/remove_listify.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/remove_listify/remove_listify.3.adm
new file mode 100644
index 0000000..a4142d4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/remove_listify/remove_listify.3.adm
@@ -0,0 +1,15 @@
+{ "counts": 1, "f1": 101 }
+{ "counts": 1, "f1": 102 }
+{ "counts": 2, "f1": 102 }
+{ "counts": 2, "f1": 102 }
+{ "counts": 1, "f1": 103 }
+{ "counts": 1, "f1": 103 }
+{ "counts": 1, "f1": 103 }
+{ "counts": 2, "f1": 103 }
+{ "counts": 3, "f1": 103 }
+{ "counts": 1, "f1": 104 }
+{ "counts": 1, "f1": 104 }
+{ "counts": 2, "f1": 104 }
+{ "counts": 0, "f1": 105 }
+{ "counts": 1, "f1": 105 }
+{ "counts": 2, "f1": 105 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/serialized_size_fun/serialized_size_fun.01.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/serialized_size_fun/serialized_size_fun.01.adm
new file mode 100644
index 0000000..511674b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/serialized_size_fun/serialized_size_fun.01.adm
@@ -0,0 +1,5 @@
+{ "serialized_size": 9, "val": 1 }
+{ "serialized_size": 2, "val": true }
+{ "serialized_size": 6, "val": "test" }
+{ "serialized_size": 36, "val": [ 1, 2 ] }
+{ "serialized_size": 88, "val": { "f1": 1, "f2": [ { "n": "str" } ] } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/load-record-fields/load-record-fields.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/load-record-fields/load-record-fields.3.adm
new file mode 100644
index 0000000..ac2dc97
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/load-record-fields/load-record-fields.3.adm
@@ -0,0 +1,2 @@
+"Alice"
+"Bob"
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/load-record-fields/load-record-fields.4.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/load-record-fields/load-record-fields.4.plan
new file mode 100644
index 0000000..0c25386
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/load-record-fields/load-record-fields.4.plan
@@ -0,0 +1,22 @@
+distribute result [$$28] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$28]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$28] <- [$$md.getField("name")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$md]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SORT_MERGE_EXCHANGE [$$30(ASC) ] |PARTITIONED|
+ select (neq(uuid(), uuid())) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_SELECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$30, $$md] <- test.MyDataset [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/load-record-fields/load-record-fields.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/load-record-fields/load-record-fields.5.adm
new file mode 100644
index 0000000..ac2dc97
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/load-record-fields/load-record-fields.5.adm
@@ -0,0 +1,2 @@
+"Alice"
+"Bob"
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/load-record-fields/load-record-fields.6.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/load-record-fields/load-record-fields.6.plan
new file mode 100644
index 0000000..dc9b1d7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/load-record-fields/load-record-fields.6.plan
@@ -0,0 +1,22 @@
+distribute result [$$28] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$28]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$28] <- [$$md.getField("name")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$md]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SORT_MERGE_EXCHANGE [$$30(ASC) ] |PARTITIONED|
+ select (neq(current-date(), date: { 1980-09-10 })) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_SELECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$30, $$md] <- test.MyDataset [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue453-2/query-issue453-2.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue453-2/query-issue453-2.1.adm
index 67218de..4b8eef0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue453-2/query-issue453-2.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue453-2/query-issue453-2.1.adm
@@ -1,16 +1,16 @@
-{ "id": 1, "int_m": 1, "int_o": 1, "string_m": "a", "string_o": "a" }
-{ "id": 2, "int_m": 1, "int_o": 1, "string_m": "a", "string_o": null }
-{ "id": 3, "int_m": 1, "int_o": 1, "string_m": "b", "string_o": "a" }
-{ "id": 4, "int_m": 1, "int_o": 1, "string_m": "b", "string_o": null }
-{ "id": 5, "int_m": 1, "int_o": null, "string_m": "a", "string_o": "a" }
-{ "id": 6, "int_m": 1, "int_o": null, "string_m": "a", "string_o": null }
-{ "id": 7, "int_m": 1, "int_o": null, "string_m": "b", "string_o": "a" }
-{ "id": 8, "int_m": 1, "int_o": null, "string_m": "b", "string_o": null }
-{ "id": 9, "int_m": 2, "int_o": 1, "string_m": "a", "string_o": "a" }
-{ "id": 10, "int_m": 2, "int_o": 1, "string_m": "a", "string_o": null }
-{ "id": 11, "int_m": 2, "int_o": 1, "string_m": "b", "string_o": "a" }
-{ "id": 12, "int_m": 2, "int_o": 1, "string_m": "b", "string_o": null }
-{ "id": 13, "int_m": 2, "int_o": null, "string_m": "a", "string_o": "a" }
-{ "id": 14, "int_m": 2, "int_o": null, "string_m": "a", "string_o": null }
-{ "id": 15, "int_m": 2, "int_o": null, "string_m": "b", "string_o": "a" }
-{ "id": 16, "int_m": 2, "int_o": null, "string_m": "b", "string_o": null }
+{ "int_m": 1, "int_o": null, "string_m": "a", "string_o": null }
+{ "int_m": 1, "int_o": null, "string_m": "a", "string_o": "a" }
+{ "int_m": 1, "int_o": null, "string_m": "b", "string_o": null }
+{ "int_m": 1, "int_o": null, "string_m": "b", "string_o": "a" }
+{ "int_m": 1, "int_o": 1, "string_m": "a", "string_o": null }
+{ "int_m": 1, "int_o": 1, "string_m": "a", "string_o": "a" }
+{ "int_m": 1, "int_o": 1, "string_m": "b", "string_o": null }
+{ "int_m": 1, "int_o": 1, "string_m": "b", "string_o": "a" }
+{ "int_m": 2, "int_o": null, "string_m": "a", "string_o": null }
+{ "int_m": 2, "int_o": null, "string_m": "a", "string_o": "a" }
+{ "int_m": 2, "int_o": null, "string_m": "b", "string_o": null }
+{ "int_m": 2, "int_o": null, "string_m": "b", "string_o": "a" }
+{ "int_m": 2, "int_o": 1, "string_m": "a", "string_o": null }
+{ "int_m": 2, "int_o": 1, "string_m": "a", "string_o": "a" }
+{ "int_m": 2, "int_o": 1, "string_m": "b", "string_o": null }
+{ "int_m": 2, "int_o": 1, "string_m": "b", "string_o": "a" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.003.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.003.adm
new file mode 100644
index 0000000..1b32b5f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.003.adm
@@ -0,0 +1,14 @@
+{ "SK0": 34, "PK0": 1 }
+{ "SK0": 20, "PK0": 2 }
+{ "SK0": null, "PK0": 3 }
+{ "PK0": 4 }
+{ "SK0": 34, "PK0": 5 }
+{ "SK0": 34, "PK0": 6 }
+{ "SK0": 20, "PK0": 7 }
+{ "SK0": 34, "PK0": 8 }
+{ "SK0": 34, "PK0": 9 }
+{ "SK0": 34, "PK0": 10 }
+{ "PK0": 11 }
+{ "SK0": null, "PK0": 12 }
+{ "SK0": null, "PK0": 13 }
+{ "PK0": 14 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.004.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.004.adm
new file mode 100644
index 0000000..7d4626e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.004.adm
@@ -0,0 +1,14 @@
+{ "SK0": "cs", "PK0": 1 }
+{ "SK0": "ms", "PK0": 2 }
+{ "SK0": "cs", "PK0": 3 }
+{ "SK0": "ms", "PK0": 4 }
+{ "SK0": null, "PK0": 5 }
+{ "PK0": 6 }
+{ "SK0": "cs", "PK0": 7 }
+{ "SK0": "ms", "PK0": 8 }
+{ "SK0": "cs", "PK0": 9 }
+{ "SK0": "ms", "PK0": 10 }
+{ "PK0": 11 }
+{ "SK0": null, "PK0": 12 }
+{ "PK0": 13 }
+{ "SK0": null, "PK0": 14 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.005.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.005.adm
new file mode 100644
index 0000000..e5d1135
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.005.adm
@@ -0,0 +1,14 @@
+{ "SK0": 34, "SK1": "cs", "PK0": 1 }
+{ "SK0": 20, "SK1": "ms", "PK0": 2 }
+{ "SK0": null, "SK1": "cs", "PK0": 3 }
+{ "SK1": "ms", "PK0": 4 }
+{ "SK0": 34, "SK1": null, "PK0": 5 }
+{ "SK0": 34, "PK0": 6 }
+{ "SK0": 20, "SK1": "cs", "PK0": 7 }
+{ "SK0": 34, "SK1": "ms", "PK0": 8 }
+{ "SK0": 34, "SK1": "cs", "PK0": 9 }
+{ "SK0": 34, "SK1": "ms", "PK0": 10 }
+{ "PK0": 11 }
+{ "SK0": null, "SK1": null, "PK0": 12 }
+{ "SK0": null, "PK0": 13 }
+{ "SK1": null, "PK0": 14 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.006.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.006.adm
new file mode 100644
index 0000000..d0d8cc8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.006.adm
@@ -0,0 +1,14 @@
+{ "SK0": "cs", "SK1": 34, "PK0": 1 }
+{ "SK0": "ms", "SK1": 20, "PK0": 2 }
+{ "SK0": "cs", "SK1": null, "PK0": 3 }
+{ "SK0": "ms", "PK0": 4 }
+{ "SK0": null, "SK1": 34, "PK0": 5 }
+{ "SK1": 34, "PK0": 6 }
+{ "SK0": "cs", "SK1": 20, "PK0": 7 }
+{ "SK0": "ms", "SK1": 34, "PK0": 8 }
+{ "SK0": "cs", "SK1": 34, "PK0": 9 }
+{ "SK0": "ms", "SK1": 34, "PK0": 10 }
+{ "PK0": 11 }
+{ "SK0": null, "SK1": null, "PK0": 12 }
+{ "SK1": null, "PK0": 13 }
+{ "SK0": null, "PK0": 14 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.007.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.007.adm
new file mode 100644
index 0000000..1b32b5f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.007.adm
@@ -0,0 +1,14 @@
+{ "SK0": 34, "PK0": 1 }
+{ "SK0": 20, "PK0": 2 }
+{ "SK0": null, "PK0": 3 }
+{ "PK0": 4 }
+{ "SK0": 34, "PK0": 5 }
+{ "SK0": 34, "PK0": 6 }
+{ "SK0": 20, "PK0": 7 }
+{ "SK0": 34, "PK0": 8 }
+{ "SK0": 34, "PK0": 9 }
+{ "SK0": 34, "PK0": 10 }
+{ "PK0": 11 }
+{ "SK0": null, "PK0": 12 }
+{ "SK0": null, "PK0": 13 }
+{ "PK0": 14 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.008.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.008.adm
new file mode 100644
index 0000000..7d4626e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.008.adm
@@ -0,0 +1,14 @@
+{ "SK0": "cs", "PK0": 1 }
+{ "SK0": "ms", "PK0": 2 }
+{ "SK0": "cs", "PK0": 3 }
+{ "SK0": "ms", "PK0": 4 }
+{ "SK0": null, "PK0": 5 }
+{ "PK0": 6 }
+{ "SK0": "cs", "PK0": 7 }
+{ "SK0": "ms", "PK0": 8 }
+{ "SK0": "cs", "PK0": 9 }
+{ "SK0": "ms", "PK0": 10 }
+{ "PK0": 11 }
+{ "SK0": null, "PK0": 12 }
+{ "PK0": 13 }
+{ "SK0": null, "PK0": 14 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.009.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.009.adm
new file mode 100644
index 0000000..e5d1135
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.009.adm
@@ -0,0 +1,14 @@
+{ "SK0": 34, "SK1": "cs", "PK0": 1 }
+{ "SK0": 20, "SK1": "ms", "PK0": 2 }
+{ "SK0": null, "SK1": "cs", "PK0": 3 }
+{ "SK1": "ms", "PK0": 4 }
+{ "SK0": 34, "SK1": null, "PK0": 5 }
+{ "SK0": 34, "PK0": 6 }
+{ "SK0": 20, "SK1": "cs", "PK0": 7 }
+{ "SK0": 34, "SK1": "ms", "PK0": 8 }
+{ "SK0": 34, "SK1": "cs", "PK0": 9 }
+{ "SK0": 34, "SK1": "ms", "PK0": 10 }
+{ "PK0": 11 }
+{ "SK0": null, "SK1": null, "PK0": 12 }
+{ "SK0": null, "PK0": 13 }
+{ "SK1": null, "PK0": 14 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.010.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.010.adm
new file mode 100644
index 0000000..d0d8cc8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.010.adm
@@ -0,0 +1,14 @@
+{ "SK0": "cs", "SK1": 34, "PK0": 1 }
+{ "SK0": "ms", "SK1": 20, "PK0": 2 }
+{ "SK0": "cs", "SK1": null, "PK0": 3 }
+{ "SK0": "ms", "PK0": 4 }
+{ "SK0": null, "SK1": 34, "PK0": 5 }
+{ "SK1": 34, "PK0": 6 }
+{ "SK0": "cs", "SK1": 20, "PK0": 7 }
+{ "SK0": "ms", "SK1": 34, "PK0": 8 }
+{ "SK0": "cs", "SK1": 34, "PK0": 9 }
+{ "SK0": "ms", "SK1": 34, "PK0": 10 }
+{ "PK0": 11 }
+{ "SK0": null, "SK1": null, "PK0": 12 }
+{ "SK1": null, "PK0": 13 }
+{ "SK0": null, "PK0": 14 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.011.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.011.adm
new file mode 100644
index 0000000..2a57ebd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.011.adm
@@ -0,0 +1,8 @@
+{ "SK0": 34, "PK0": 1 }
+{ "SK0": 20, "PK0": 2 }
+{ "SK0": 34, "PK0": 5 }
+{ "SK0": 34, "PK0": 6 }
+{ "SK0": 20, "PK0": 7 }
+{ "SK0": 34, "PK0": 8 }
+{ "SK0": 34, "PK0": 9 }
+{ "SK0": 34, "PK0": 10 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.012.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.012.adm
new file mode 100644
index 0000000..0021c09
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.012.adm
@@ -0,0 +1,8 @@
+{ "SK0": "cs", "PK0": 1 }
+{ "SK0": "ms", "PK0": 2 }
+{ "SK0": null, "PK0": 5 }
+{ "PK0": 6 }
+{ "SK0": "cs", "PK0": 7 }
+{ "SK0": "ms", "PK0": 8 }
+{ "SK0": "cs", "PK0": 9 }
+{ "SK0": "ms", "PK0": 10 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.013.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.013.adm
new file mode 100644
index 0000000..1858896
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.013.adm
@@ -0,0 +1,8 @@
+{ "SK0": 34, "SK1": "cs", "PK0": 1 }
+{ "SK0": 20, "SK1": "ms", "PK0": 2 }
+{ "SK0": 34, "SK1": null, "PK0": 5 }
+{ "SK0": 34, "PK0": 6 }
+{ "SK0": 20, "SK1": "cs", "PK0": 7 }
+{ "SK0": 34, "SK1": "ms", "PK0": 8 }
+{ "SK0": 34, "SK1": "cs", "PK0": 9 }
+{ "SK0": 34, "SK1": "ms", "PK0": 10 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.014.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.014.adm
new file mode 100644
index 0000000..3ce6c09
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.014.adm
@@ -0,0 +1,8 @@
+{ "SK0": "cs", "SK1": 34, "PK0": 1 }
+{ "SK0": "ms", "SK1": 20, "PK0": 2 }
+{ "SK0": null, "SK1": 34, "PK0": 5 }
+{ "SK1": 34, "PK0": 6 }
+{ "SK0": "cs", "SK1": 20, "PK0": 7 }
+{ "SK0": "ms", "SK1": 34, "PK0": 8 }
+{ "SK0": "cs", "SK1": 34, "PK0": 9 }
+{ "SK0": "ms", "SK1": 34, "PK0": 10 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.015.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.015.adm
new file mode 100644
index 0000000..dfec480
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.015.adm
@@ -0,0 +1 @@
+{ "cnt": 14 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.016.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.016.adm
new file mode 100644
index 0000000..dfec480
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.016.adm
@@ -0,0 +1 @@
+{ "cnt": 14 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.017.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.017.adm
new file mode 100644
index 0000000..831b0cb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.017.adm
@@ -0,0 +1,4 @@
+{ "cnt": 3 }
+{ "cnt": 3, "age": null }
+{ "cnt": 2, "age": 20 }
+{ "cnt": 6, "age": 34 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.018.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.018.adm
new file mode 100644
index 0000000..6a94e7b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.018.adm
@@ -0,0 +1,4 @@
+{ "cnt": 3 }
+{ "age": null, "cnt": 3 }
+{ "age": 20, "cnt": 2 }
+{ "age": 34, "cnt": 6 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.019.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.019.adm
new file mode 100644
index 0000000..6a94e7b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.019.adm
@@ -0,0 +1,4 @@
+{ "cnt": 3 }
+{ "age": null, "cnt": 3 }
+{ "age": 20, "cnt": 2 }
+{ "age": 34, "cnt": 6 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.020.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.020.adm
new file mode 100644
index 0000000..6a94e7b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.020.adm
@@ -0,0 +1,4 @@
+{ "cnt": 3 }
+{ "age": null, "cnt": 3 }
+{ "age": 20, "cnt": 2 }
+{ "age": 34, "cnt": 6 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.021.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.021.adm
new file mode 100644
index 0000000..3c82bb9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.021.adm
@@ -0,0 +1,12 @@
+{ "cnt": 1 }
+{ "cnt": 1, "dept": null }
+{ "cnt": 1, "dept": "ms" }
+{ "cnt": 1, "age": null }
+{ "cnt": 1, "age": null, "dept": null }
+{ "cnt": 1, "age": null, "dept": "cs" }
+{ "cnt": 1, "age": 20, "dept": "cs" }
+{ "cnt": 1, "age": 20, "dept": "ms" }
+{ "cnt": 1, "age": 34 }
+{ "cnt": 1, "age": 34, "dept": null }
+{ "cnt": 2, "age": 34, "dept": "cs" }
+{ "cnt": 2, "age": 34, "dept": "ms" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.022.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.022.adm
new file mode 100644
index 0000000..8dd327f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.022.adm
@@ -0,0 +1,12 @@
+{ "cnt": 1 }
+{ "dept": null, "cnt": 1 }
+{ "dept": "ms", "cnt": 1 }
+{ "age": null, "cnt": 1 }
+{ "age": null, "dept": null, "cnt": 1 }
+{ "age": null, "dept": "cs", "cnt": 1 }
+{ "age": 20, "dept": "cs", "cnt": 1 }
+{ "age": 20, "dept": "ms", "cnt": 1 }
+{ "age": 34, "cnt": 1 }
+{ "age": 34, "dept": null, "cnt": 1 }
+{ "age": 34, "dept": "cs", "cnt": 2 }
+{ "age": 34, "dept": "ms", "cnt": 2 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.023.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.023.adm
new file mode 100644
index 0000000..7cf5585
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.023.adm
@@ -0,0 +1,12 @@
+{ "cnt": 1 }
+{ "age": null, "cnt": 1 }
+{ "age": 34, "cnt": 1 }
+{ "dept": null, "cnt": 1 }
+{ "age": null, "dept": null, "cnt": 1 }
+{ "age": 34, "dept": null, "cnt": 1 }
+{ "age": null, "dept": "cs", "cnt": 1 }
+{ "age": 20, "dept": "cs", "cnt": 1 }
+{ "age": 34, "dept": "cs", "cnt": 2 }
+{ "dept": "ms", "cnt": 1 }
+{ "age": 20, "dept": "ms", "cnt": 1 }
+{ "age": 34, "dept": "ms", "cnt": 2 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.024.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.024.plan
new file mode 100644
index 0000000..ef9d27d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.024.plan
@@ -0,0 +1,20 @@
+distribute result [$$v] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$v]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$v] <- [{"SK0": $$13, "PK0": $$14}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SORT_MERGE_EXCHANGE [$$14(ASC) ] |PARTITIONED|
+ order (ASC, $$14) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STABLE_SORT [$$14(ASC)] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$13, $$14] <- test.ds1.ds1_age.query-index [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.025.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.025.plan
new file mode 100644
index 0000000..647913d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.025.plan
@@ -0,0 +1,20 @@
+distribute result [$$v] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$v]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$v] <- [{"SK0": $$13, "SK1": $$14, "PK0": $$15}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SORT_MERGE_EXCHANGE [$$15(ASC) ] |PARTITIONED|
+ order (ASC, $$15) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STABLE_SORT [$$15(ASC)] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$13, $$14, $$15] <- test.ds1.ds1_age_dept.query-index [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.026.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.026.plan
new file mode 100644
index 0000000..9c3d41a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.026.plan
@@ -0,0 +1,38 @@
+distribute result [$$53] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$53]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$53] <- [{"age": $$SK0, "dept": $$SK1, "cnt": $$59}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SORT_MERGE_EXCHANGE [$$SK1(ASC), $$SK0(ASC) ] |PARTITIONED|
+ group by ([$$SK1 := $$61; $$SK0 := $$62]) decor ([]) {
+ aggregate [$$59] <- [agg-sql-sum($$60)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- AGGREGATE |LOCAL|
+ nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ } [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SORT_GROUP_BY[$$61, $$62] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- HASH_PARTITION_EXCHANGE [$$61, $$62] |PARTITIONED|
+ group by ([$$61 := $$57; $$62 := $$56]) decor ([]) {
+ aggregate [$$60] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- AGGREGATE |LOCAL|
+ nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ } [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SORT_GROUP_BY[$$57, $$56] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$57, $$56]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$56, $$57, $$58] <- test.ds1.ds1_age_dept.query-index [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.027.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.027.plan
new file mode 100644
index 0000000..6203b6a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.027.plan
@@ -0,0 +1,22 @@
+distribute result [$$31] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ project ([$$31]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |UNPARTITIONED|
+ assign [$$31] <- [{"cnt": $$34}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |UNPARTITIONED|
+ aggregate [$$34] <- [agg-sql-sum($$35)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- AGGREGATE |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ aggregate [$$35] <- [agg-sql-count(1)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- AGGREGATE |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$32, $$33] <- test.ds1.ds1_dept.query-index [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.028.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.028.plan
new file mode 100644
index 0000000..f46ad89
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.028.plan
@@ -0,0 +1,38 @@
+distribute result [$$42] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$42]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$42] <- [{"age": $$SK0, "cnt": $$46}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SORT_MERGE_EXCHANGE [$$SK0(ASC) ] |PARTITIONED|
+ group by ([$$SK0 := $$48]) decor ([]) {
+ aggregate [$$46] <- [agg-sql-sum($$47)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- AGGREGATE |LOCAL|
+ nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ } [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SORT_GROUP_BY[$$48] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- HASH_PARTITION_EXCHANGE [$$48] |PARTITIONED|
+ group by ([$$48 := $$44]) decor ([]) {
+ aggregate [$$47] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- AGGREGATE |LOCAL|
+ nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ } [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- PRE_CLUSTERED_GROUP_BY[$$44] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$44]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$44, $$45] <- test.ds1.ds1_age.query-index [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.029.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.029.plan
new file mode 100644
index 0000000..3079537
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.029.plan
@@ -0,0 +1,38 @@
+distribute result [$$42] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$42]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$42] <- [{"age": $$SK0, "cnt": $$47}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SORT_MERGE_EXCHANGE [$$SK0(ASC) ] |PARTITIONED|
+ group by ([$$SK0 := $$49]) decor ([]) {
+ aggregate [$$47] <- [agg-sql-sum($$48)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- AGGREGATE |LOCAL|
+ nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ } [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SORT_GROUP_BY[$$49] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- HASH_PARTITION_EXCHANGE [$$49] |PARTITIONED|
+ group by ([$$49 := $$44]) decor ([]) {
+ aggregate [$$48] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- AGGREGATE |LOCAL|
+ nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ } [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- PRE_CLUSTERED_GROUP_BY[$$44] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$44]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$44, $$45, $$46] <- test.ds1.ds1_age_dept.query-index [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.030.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.030.plan
new file mode 100644
index 0000000..d827242
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.030.plan
@@ -0,0 +1,38 @@
+distribute result [$$42] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$42]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$42] <- [{"age": $$SK1, "cnt": $$47}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SORT_MERGE_EXCHANGE [$$SK1(ASC) ] |PARTITIONED|
+ group by ([$$SK1 := $$49]) decor ([]) {
+ aggregate [$$47] <- [agg-sql-sum($$48)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- AGGREGATE |LOCAL|
+ nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ } [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SORT_GROUP_BY[$$49] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- HASH_PARTITION_EXCHANGE [$$49] |PARTITIONED|
+ group by ([$$49 := $$45]) decor ([]) {
+ aggregate [$$48] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- AGGREGATE |LOCAL|
+ nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ } [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SORT_GROUP_BY[$$45] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$45]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$44, $$45, $$46] <- test.ds1.ds1_dept_age.query-index [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.031.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.031.plan
new file mode 100644
index 0000000..41c7bd4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.031.plan
@@ -0,0 +1,38 @@
+distribute result [$$53] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$53]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$53] <- [{"age": $$SK0, "dept": $$SK1, "cnt": $$59}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SORT_MERGE_EXCHANGE [$$SK0(ASC), $$SK1(ASC) ] |PARTITIONED|
+ group by ([$$SK0 := $$61; $$SK1 := $$62]) decor ([]) {
+ aggregate [$$59] <- [agg-sql-sum($$60)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- AGGREGATE |LOCAL|
+ nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ } [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SORT_GROUP_BY[$$61, $$62] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- HASH_PARTITION_EXCHANGE [$$61, $$62] |PARTITIONED|
+ group by ([$$61 := $$56; $$62 := $$57]) decor ([]) {
+ aggregate [$$60] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- AGGREGATE |LOCAL|
+ nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ } [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- PRE_CLUSTERED_GROUP_BY[$$56, $$57] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$56, $$57]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$56, $$57, $$58] <- test.ds1.ds1_age_dept.query-index [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.13.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.13.adm
deleted file mode 100644
index e14f391..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.13.adm
+++ /dev/null
@@ -1,32 +0,0 @@
-distribute result [$$l]
--- DISTRIBUTE_RESULT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$l])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- SORT_MERGE_EXCHANGE [$$16(ASC), $$17(ASC) ] |PARTITIONED|
- select (eq($$l.getField(10), "1994-01-20"))
- -- STREAM_SELECT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- unnest-map [$$16, $$17, $$l] <- index-search("LineItem", 0, "tpch", "LineItem", FALSE, FALSE, 2, $$24, $$25, 2, $$24, $$25, TRUE, TRUE, TRUE)
- -- BTREE_SEARCH |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- order (ASC, $$24) (ASC, $$25)
- -- STABLE_SORT [$$24(ASC), $$25(ASC)] |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$24, $$25])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- unnest-map [$$23, $$24, $$25] <- index-search("lineitem_shipdateIx", 0, "tpch", "LineItem", FALSE, FALSE, 1, $$21, 1, $$22, TRUE, TRUE, TRUE)
- -- BTREE_SEARCH |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- assign [$$21, $$22] <- ["1994-01-20", "1994-01-20"]
- -- ASSIGN |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.13.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.13.plan
new file mode 100644
index 0000000..6faed2f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.13.plan
@@ -0,0 +1,32 @@
+distribute result [$$l]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$l])
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange
+ -- SORT_MERGE_EXCHANGE [$$16(ASC), $$17(ASC) ] |PARTITIONED|
+ select (eq($$l.getField(10), "1994-01-20"))
+ -- STREAM_SELECT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ unnest-map [$$16, $$17, $$l] <- index-search("LineItem", 0, "tpch", "LineItem", false, false, 2, $$24, $$25, 2, $$24, $$25, true, true, true)
+ -- BTREE_SEARCH |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ order (ASC, $$24) (ASC, $$25)
+ -- STABLE_SORT [$$24(ASC), $$25(ASC)] |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$24, $$25])
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ unnest-map [$$23, $$24, $$25] <- index-search("lineitem_shipdateIx", 0, "tpch", "LineItem", false, false, 1, $$21, 1, $$22, true, true, true)
+ -- BTREE_SEARCH |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ assign [$$21, $$22] <- ["1994-01-20", "1994-01-20"]
+ -- ASSIGN |PARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.8.adm
deleted file mode 100644
index e14f391..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.8.adm
+++ /dev/null
@@ -1,32 +0,0 @@
-distribute result [$$l]
--- DISTRIBUTE_RESULT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$l])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- SORT_MERGE_EXCHANGE [$$16(ASC), $$17(ASC) ] |PARTITIONED|
- select (eq($$l.getField(10), "1994-01-20"))
- -- STREAM_SELECT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- unnest-map [$$16, $$17, $$l] <- index-search("LineItem", 0, "tpch", "LineItem", FALSE, FALSE, 2, $$24, $$25, 2, $$24, $$25, TRUE, TRUE, TRUE)
- -- BTREE_SEARCH |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- order (ASC, $$24) (ASC, $$25)
- -- STABLE_SORT [$$24(ASC), $$25(ASC)] |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$24, $$25])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- unnest-map [$$23, $$24, $$25] <- index-search("lineitem_shipdateIx", 0, "tpch", "LineItem", FALSE, FALSE, 1, $$21, 1, $$22, TRUE, TRUE, TRUE)
- -- BTREE_SEARCH |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- assign [$$21, $$22] <- ["1994-01-20", "1994-01-20"]
- -- ASSIGN |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.8.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.8.plan
new file mode 100644
index 0000000..6faed2f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.8.plan
@@ -0,0 +1,32 @@
+distribute result [$$l]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$l])
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange
+ -- SORT_MERGE_EXCHANGE [$$16(ASC), $$17(ASC) ] |PARTITIONED|
+ select (eq($$l.getField(10), "1994-01-20"))
+ -- STREAM_SELECT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ unnest-map [$$16, $$17, $$l] <- index-search("LineItem", 0, "tpch", "LineItem", false, false, 2, $$24, $$25, 2, $$24, $$25, true, true, true)
+ -- BTREE_SEARCH |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ order (ASC, $$24) (ASC, $$25)
+ -- STABLE_SORT [$$24(ASC), $$25(ASC)] |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$24, $$25])
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ unnest-map [$$23, $$24, $$25] <- index-search("lineitem_shipdateIx", 0, "tpch", "LineItem", false, false, 1, $$21, 1, $$22, true, true, true)
+ -- BTREE_SEARCH |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ assign [$$21, $$22] <- ["1994-01-20", "1994-01-20"]
+ -- ASSIGN |PARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/query-ASTERIXDB-3116/query-ASTERIXDB-3116.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/query-ASTERIXDB-3116/query-ASTERIXDB-3116.1.adm
new file mode 100644
index 0000000..597975b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/query-ASTERIXDB-3116/query-ASTERIXDB-3116.1.adm
@@ -0,0 +1 @@
+35
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/query-ASTERIXDB-3116/query-ASTERIXDB-3116.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/query-ASTERIXDB-3116/query-ASTERIXDB-3116.2.adm
new file mode 100644
index 0000000..86ee83a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/query-ASTERIXDB-3116/query-ASTERIXDB-3116.2.adm
@@ -0,0 +1 @@
+40
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/string/invalid-unicode/result.000.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/invalid-unicode/result.000.adm
new file mode 100644
index 0000000..a9fc274
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/invalid-unicode/result.000.adm
@@ -0,0 +1 @@
+[ null, null, null, null, null, null, null, null, null, null, null ]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/select_element/select_element.02.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/select_element/select_element.02.adm
new file mode 100644
index 0000000..f6fb5bd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/select_element/select_element.02.adm
@@ -0,0 +1,3 @@
+{ "a": 1, "test": 2 }
+{ "a": 2, "test": 3 }
+{ "a": 3, "test": 4 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/select_element/select_element.03.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/select_element/select_element.03.adm
new file mode 100644
index 0000000..ea413fe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/select_element/select_element.03.adm
@@ -0,0 +1,3 @@
+{ "a": 1, "test": [ 1 ] }
+{ "a": 2, "test": [ 2 ] }
+{ "a": 3, "test": [ 3 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/select_element/select_element.04.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/select_element/select_element.04.adm
new file mode 100644
index 0000000..2cf7df3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/select_element/select_element.04.adm
@@ -0,0 +1,3 @@
+{ "a": 1, "test": [ [ 1 ] ] }
+{ "a": 2, "test": [ [ 2 ] ] }
+{ "a": 3, "test": [ [ 3 ] ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/select_element/select_element.05.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/select_element/select_element.05.adm
new file mode 100644
index 0000000..59b19361
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/select_element/select_element.05.adm
@@ -0,0 +1,3 @@
+{ "a": 1, "test": [ { "data": 1 } ] }
+{ "a": 2, "test": [ { "data": 2 } ] }
+{ "a": 3, "test": [ { "data": 3 } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/select_element/select_element.06.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/select_element/select_element.06.adm
new file mode 100644
index 0000000..3eed37c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/select_element/select_element.06.adm
@@ -0,0 +1,3 @@
+{ "a": 1, "test": [ { "x": [ { "data": 1 } ] } ] }
+{ "a": 2, "test": [ { "x": [ { "data": 2 } ] } ] }
+{ "a": 3, "test": [ { "x": [ { "data": 3 } ] } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/select_element/select_element.07.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/select_element/select_element.07.adm
new file mode 100644
index 0000000..944e5a2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/select_element/select_element.07.adm
@@ -0,0 +1,3 @@
+{ "a": 1, "$1": 2 }
+{ "a": 2, "$1": 3 }
+{ "a": 3, "$1": 4 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/select_element/select_element.08.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/select_element/select_element.08.adm
new file mode 100644
index 0000000..c8f629b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/select_element/select_element.08.adm
@@ -0,0 +1,3 @@
+{ "a": 1, "$1": [ 1 ] }
+{ "a": 2, "$1": [ 2 ] }
+{ "a": 3, "$1": [ 3 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/select_element/select_element.09.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/select_element/select_element.09.adm
new file mode 100644
index 0000000..325d24f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/select_element/select_element.09.adm
@@ -0,0 +1,3 @@
+{ "a": 1, "$1": [ [ 1 ] ] }
+{ "a": 2, "$1": [ [ 2 ] ] }
+{ "a": 3, "$1": [ [ 3 ] ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/select_element/select_element.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/select_element/select_element.10.adm
new file mode 100644
index 0000000..a5ef5cd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/select_element/select_element.10.adm
@@ -0,0 +1,3 @@
+{ "a": 1, "$1": [ { "data": 1 } ] }
+{ "a": 2, "$1": [ { "data": 2 } ] }
+{ "a": 3, "$1": [ { "data": 3 } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/select_element/select_element.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/select_element/select_element.11.adm
new file mode 100644
index 0000000..1936e65
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/select_element/select_element.11.adm
@@ -0,0 +1,3 @@
+{ "a": 1, "$1": [ { "x": [ { "data": 1 } ] } ] }
+{ "a": 2, "$1": [ { "x": [ { "data": 2 } ] } ] }
+{ "a": 3, "$1": [ { "x": [ { "data": 3 } ] } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/select_element/select_element.12.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/select_element/select_element.12.adm
new file mode 100644
index 0000000..ea413fe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/select_element/select_element.12.adm
@@ -0,0 +1,3 @@
+{ "a": 1, "test": [ 1 ] }
+{ "a": 2, "test": [ 2 ] }
+{ "a": 3, "test": [ 3 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/select_element/select_element.13.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/select_element/select_element.13.adm
new file mode 100644
index 0000000..c8f629b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/select_element/select_element.13.adm
@@ -0,0 +1,3 @@
+{ "a": 1, "$1": [ 1 ] }
+{ "a": 2, "$1": [ 2 ] }
+{ "a": 3, "$1": [ 3 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/select_element/select_element.14.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/select_element/select_element.14.adm
new file mode 100644
index 0000000..2cf7df3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/select_element/select_element.14.adm
@@ -0,0 +1,3 @@
+{ "a": 1, "test": [ [ 1 ] ] }
+{ "a": 2, "test": [ [ 2 ] ] }
+{ "a": 3, "test": [ [ 3 ] ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/select_element/select_element.15.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/select_element/select_element.15.adm
new file mode 100644
index 0000000..325d24f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/select_element/select_element.15.adm
@@ -0,0 +1,3 @@
+{ "a": 1, "$1": [ [ 1 ] ] }
+{ "a": 2, "$1": [ [ 2 ] ] }
+{ "a": 3, "$1": [ [ 3 ] ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/select_element/select_element.16.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/select_element/select_element.16.adm
new file mode 100644
index 0000000..0f584e9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/select_element/select_element.16.adm
@@ -0,0 +1,3 @@
+{ "a": 1, "test": [ 7 ] }
+{ "a": 2, "test": [ 8 ] }
+{ "a": 3, "test": [ 9 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/select_element/select_element.17.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/select_element/select_element.17.adm
new file mode 100644
index 0000000..74fd97f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/select_element/select_element.17.adm
@@ -0,0 +1,3 @@
+{ "a": 1, "$1": 1.0 }
+{ "a": 2, "$1": 2.0 }
+{ "a": 3, "$1": 3.0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/tpcds/q18/q18.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpcds/q18/q18.2.adm
new file mode 100644
index 0000000..6280446
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpcds/q18/q18.2.adm
@@ -0,0 +1 @@
+{ "cnt": 3 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/tpcds/q18/q18.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpcds/q18/q18.3.adm
new file mode 100644
index 0000000..6280446
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpcds/q18/q18.3.adm
@@ -0,0 +1 @@
+{ "cnt": 3 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/union/union_opt_1/union_opt_1.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/union/union_opt_1/union_opt_1.11.adm
deleted file mode 100644
index a809a8e..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/union/union_opt_1/union_opt_1.11.adm
+++ /dev/null
@@ -1,106 +0,0 @@
-distribute result [$$t]
--- DISTRIBUTE_RESULT |UNPARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- limit 4
- -- STREAM_LIMIT |UNPARTITIONED|
- exchange
- -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
- union ($$151, $$178, $$t)
- -- UNION_ALL |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- union ($$213, $$227, $$151)
- -- UNION_ALL |PARTITIONED|
- exchange
- -- RANDOM_PARTITION_EXCHANGE |PARTITIONED|
- project ([$$213])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$213] <- [{"two": $$183}]
- -- ASSIGN |PARTITIONED|
- limit 4
- -- STREAM_LIMIT |PARTITIONED|
- project ([$$183])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$183] <- [$$onek1.getField(2)]
- -- ASSIGN |PARTITIONED|
- project ([$$onek1])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$187, $$onek1] <- test.onek1 condition (and(ge($$onek1.getField(2), 1), le($$onek1.getField(2), 10))) limit 4
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- exchange
- -- RANDOM_PARTITION_EXCHANGE |PARTITIONED|
- project ([$$227])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$227] <- [{"two": $$184}]
- -- ASSIGN |PARTITIONED|
- limit 4
- -- STREAM_LIMIT |PARTITIONED|
- project ([$$184])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$184] <- [$$onek2.getField(2)]
- -- ASSIGN |PARTITIONED|
- project ([$$onek2])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$188, $$onek2] <- test.onek2 condition (and(ge($$onek2.getField(2), 1), le($$onek2.getField(2), 100))) limit 4
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- union ($$345, $$354, $$178)
- -- UNION_ALL |PARTITIONED|
- exchange
- -- RANDOM_PARTITION_EXCHANGE |PARTITIONED|
- project ([$$345])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$345] <- [{"two": $$185}]
- -- ASSIGN |PARTITIONED|
- limit 4
- -- STREAM_LIMIT |PARTITIONED|
- project ([$$185])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$185] <- [$$onek1.getField(2)]
- -- ASSIGN |PARTITIONED|
- project ([$$onek1])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$189, $$onek1] <- test.onek1 condition (and(ge($$onek1.getField(2), 1), le($$onek1.getField(2), 1000))) limit 4
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- exchange
- -- RANDOM_PARTITION_EXCHANGE |PARTITIONED|
- project ([$$354])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$354] <- [{"two": $$186}]
- -- ASSIGN |PARTITIONED|
- limit 4
- -- STREAM_LIMIT |PARTITIONED|
- project ([$$186])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$186] <- [$$onek2.getField(2)]
- -- ASSIGN |PARTITIONED|
- project ([$$onek2])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$190, $$onek2] <- test.onek2 condition (and(ge($$onek2.getField(2), 1), le($$onek2.getField(2), 10000))) limit 4
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/union/union_opt_1/union_opt_1.11.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/union/union_opt_1/union_opt_1.11.plan
new file mode 100644
index 0000000..7fe2631
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/union/union_opt_1/union_opt_1.11.plan
@@ -0,0 +1,106 @@
+distribute result [$$t] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ limit 4 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_LIMIT |UNPARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ union ($$151, $$178, $$t) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- UNION_ALL |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ union ($$213, $$227, $$151) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- UNION_ALL |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- RANDOM_PARTITION_EXCHANGE |PARTITIONED|
+ project ([$$213]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$213] <- [{"two": $$183}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ limit 4 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_LIMIT |PARTITIONED|
+ project ([$$183]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$183] <- [$$onek1.getField(2)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$onek1]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$187, $$onek1] <- test.onek1 condition (and(ge($$onek1.getField(2), 1), le($$onek1.getField(2), 10))) limit 4 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- RANDOM_PARTITION_EXCHANGE |PARTITIONED|
+ project ([$$227]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$227] <- [{"two": $$184}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ limit 4 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_LIMIT |PARTITIONED|
+ project ([$$184]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$184] <- [$$onek2.getField(2)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$onek2]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$188, $$onek2] <- test.onek2 condition (and(ge($$onek2.getField(2), 1), le($$onek2.getField(2), 100))) limit 4 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ union ($$345, $$354, $$178) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- UNION_ALL |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- RANDOM_PARTITION_EXCHANGE |PARTITIONED|
+ project ([$$345]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$345] <- [{"two": $$185}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ limit 4 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_LIMIT |PARTITIONED|
+ project ([$$185]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$185] <- [$$onek1.getField(2)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$onek1]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$189, $$onek1] <- test.onek1 condition (and(ge($$onek1.getField(2), 1), le($$onek1.getField(2), 1000))) limit 4 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- RANDOM_PARTITION_EXCHANGE |PARTITIONED|
+ project ([$$354]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$354] <- [{"two": $$186}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ limit 4 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_LIMIT |PARTITIONED|
+ project ([$$186]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$186] <- [$$onek2.getField(2)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$onek2]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$190, $$onek2] <- test.onek2 condition (and(ge($$onek2.getField(2), 1), le($$onek2.getField(2), 10000))) limit 4 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/union/union_opt_1/union_opt_1.9.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/union/union_opt_1/union_opt_1.9.adm
deleted file mode 100644
index 4a46e2d..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/union/union_opt_1/union_opt_1.9.adm
+++ /dev/null
@@ -1,54 +0,0 @@
-distribute result [$$t]
--- DISTRIBUTE_RESULT |UNPARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- limit 4
- -- STREAM_LIMIT |UNPARTITIONED|
- exchange
- -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
- union ($$48, $$54, $$t)
- -- UNION_ALL |PARTITIONED|
- exchange
- -- RANDOM_PARTITION_EXCHANGE |PARTITIONED|
- limit 4
- -- STREAM_LIMIT |PARTITIONED|
- project ([$$48])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$48] <- [{"two": $$103}]
- -- ASSIGN |PARTITIONED|
- project ([$$103])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$103] <- [$$onek1.getField(2)]
- -- ASSIGN |PARTITIONED|
- project ([$$onek1])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$61, $$onek1] <- test.onek1 condition (gt($$onek1.getField(2), 0)) limit 4
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- exchange
- -- RANDOM_PARTITION_EXCHANGE |PARTITIONED|
- limit 4
- -- STREAM_LIMIT |PARTITIONED|
- project ([$$54])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$54] <- [{"two": $$105}]
- -- ASSIGN |PARTITIONED|
- project ([$$105])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$105] <- [$$onek2.getField(2)]
- -- ASSIGN |PARTITIONED|
- project ([$$onek2])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$62, $$onek2] <- test.onek2 condition (gt($$onek2.getField(2), 0)) limit 4
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/union/union_opt_1/union_opt_1.9.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/union/union_opt_1/union_opt_1.9.plan
new file mode 100644
index 0000000..d4c98e1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/union/union_opt_1/union_opt_1.9.plan
@@ -0,0 +1,54 @@
+distribute result [$$t] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ limit 4 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_LIMIT |UNPARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ union ($$48, $$54, $$t) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- UNION_ALL |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- RANDOM_PARTITION_EXCHANGE |PARTITIONED|
+ limit 4 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_LIMIT |PARTITIONED|
+ project ([$$48]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$48] <- [{"two": $$103}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$103]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$103] <- [$$onek1.getField(2)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$onek1]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$61, $$onek1] <- test.onek1 condition (gt($$onek1.getField(2), 0)) limit 4 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- RANDOM_PARTITION_EXCHANGE |PARTITIONED|
+ limit 4 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_LIMIT |PARTITIONED|
+ project ([$$54]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$54] <- [{"two": $$105}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$105]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$105] <- [$$onek2.getField(2)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$onek2]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$62, $$onek2] <- test.onek2 condition (gt($$onek2.getField(2), 0)) limit 4 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/inline-in-expr/inline-in-expr.03.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/inline-in-expr/inline-in-expr.03.adm
new file mode 100644
index 0000000..145a6e3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/inline-in-expr/inline-in-expr.03.adm
@@ -0,0 +1 @@
+{ "x1": -3, "x2": 40, "x3": 200 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.03.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.03.adm
new file mode 100644
index 0000000..b5aed50
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.03.adm
@@ -0,0 +1,3 @@
+{ "id": 5, "review": "good" }
+{ "id": 7, "review": "good" }
+{ "id": 11, "review": "good" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.04.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.04.plan
new file mode 100644
index 0000000..ad92231
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.04.plan
@@ -0,0 +1,34 @@
+distribute result [$$91] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ project ([$$91]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- STREAM_PROJECT |UNPARTITIONED|
+ assign [$$91] <- [{"id": $$106, "review": $$111}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- ASSIGN |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ order (ASC, $$106) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- STABLE_SORT [$$106(ASC)] |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ limit 3 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- STREAM_LIMIT |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ project ([$$111, $$106]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$106] <- [int64-default-null($$d.getField("id"))] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- ASSIGN |PARTITIONED|
+ limit 3 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- STREAM_LIMIT |PARTITIONED|
+ assign [$$111] <- [string-default-null($$d.getField("review"))] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$d] <- test.ExternalDataset condition (and(not(is-unknown(int64-default-null($$d.getField("year")))), not(is-unknown(int64-default-null($$d.getField("quarter")))), eq(string-default-null($$d.getField("review")), "good"))) limit 3 [cardinality: 1000000.0, op-cost: 2.1, total-cost: 2.1]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.05.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.05.adm
new file mode 100644
index 0000000..28f5b17
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.05.adm
@@ -0,0 +1,3 @@
+{ "id": 1, "review": "good" }
+{ "id": 2, "review": "good" }
+{ "id": 3, "review": "good" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.06.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.06.plan
new file mode 100644
index 0000000..7db40a7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.06.plan
@@ -0,0 +1,22 @@
+distribute result [$$67] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ limit 3 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- STREAM_LIMIT |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ project ([$$67]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$67] <- [{"id": int64-default-null($$d.getField("id")), "review": string-default-null($$d.getField("review"))}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- ASSIGN |PARTITIONED|
+ limit 3 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- STREAM_LIMIT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$d] <- test.ExternalDataset limit 3 [cardinality: 1000000.0, op-cost: 2.1, total-cost: 2.1]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.07.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.07.adm
new file mode 100644
index 0000000..b5aed50
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.07.adm
@@ -0,0 +1,3 @@
+{ "id": 5, "review": "good" }
+{ "id": 7, "review": "good" }
+{ "id": 11, "review": "good" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.08.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.08.plan
new file mode 100644
index 0000000..fe48b86
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.08.plan
@@ -0,0 +1,26 @@
+distribute result [$$85] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ project ([$$85]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- STREAM_PROJECT |UNPARTITIONED|
+ assign [$$85] <- [{"id": $$88, "review": $$92}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- ASSIGN |UNPARTITIONED|
+ limit 3 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- STREAM_LIMIT |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- SORT_MERGE_EXCHANGE [$$88(ASC) ] |PARTITIONED|
+ limit 3 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- STREAM_LIMIT |PARTITIONED|
+ project ([$$88, $$92]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$92] <- [$$d.getField(1)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$88, $$d] <- test.DatasetWithKnownField condition (and(not(is-unknown(int64-default-null($$d.getField("year")))), not(is-unknown(int64-default-null($$d.getField("quarter")))), eq($$d.getField(1), "good"))) limit 3 [cardinality: 1000000.0, op-cost: 2.1, total-cost: 2.1]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.09.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.09.adm
new file mode 100644
index 0000000..28f5b17
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.09.adm
@@ -0,0 +1,3 @@
+{ "id": 1, "review": "good" }
+{ "id": 2, "review": "good" }
+{ "id": 3, "review": "good" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.10.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.10.plan
new file mode 100644
index 0000000..f496dbf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.10.plan
@@ -0,0 +1,26 @@
+distribute result [$$61] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ limit 3 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- STREAM_LIMIT |UNPARTITIONED|
+ project ([$$61]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$61] <- [{"id": $$63, "review": $$67}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- SORT_MERGE_EXCHANGE [$$63(ASC) ] |PARTITIONED|
+ project ([$$63, $$67]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$67] <- [$$d.getField(1)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- ASSIGN |PARTITIONED|
+ limit 3 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- STREAM_LIMIT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$63, $$d] <- test.DatasetWithKnownField limit 3 [cardinality: 1000000.0, op-cost: 2.1, total-cost: 2.1]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.11.adm
new file mode 100644
index 0000000..b5aed50
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.11.adm
@@ -0,0 +1,3 @@
+{ "id": 5, "review": "good" }
+{ "id": 7, "review": "good" }
+{ "id": 11, "review": "good" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.12.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.12.plan
new file mode 100644
index 0000000..7d39583
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.12.plan
@@ -0,0 +1,26 @@
+distribute result [$$91] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ project ([$$91]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- STREAM_PROJECT |UNPARTITIONED|
+ assign [$$91] <- [{"id": $$94, "review": $$98}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- ASSIGN |UNPARTITIONED|
+ limit 3 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- STREAM_LIMIT |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- SORT_MERGE_EXCHANGE [$$94(ASC) ] |PARTITIONED|
+ limit 3 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- STREAM_LIMIT |PARTITIONED|
+ project ([$$94, $$98]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$98] <- [$$d.getField(1)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$94, $$d] <- test.DatasetWithKnownField condition (and(not(is-unknown(int64-default-null($$d.getField("year")))), not(is-unknown(int64-default-null($$d.getField("quarter")))), eq($$d.getField(1), "good"))) limit 3 [cardinality: 1000000.0, op-cost: 2.1, total-cost: 2.1]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.13.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.13.adm
new file mode 100644
index 0000000..28f5b17
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.13.adm
@@ -0,0 +1,3 @@
+{ "id": 1, "review": "good" }
+{ "id": 2, "review": "good" }
+{ "id": 3, "review": "good" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.14.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.14.plan
new file mode 100644
index 0000000..5aa69a4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.14.plan
@@ -0,0 +1,26 @@
+distribute result [$$67] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ limit 3 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- STREAM_LIMIT |UNPARTITIONED|
+ project ([$$67]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$67] <- [{"id": $$69, "review": $$73}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- SORT_MERGE_EXCHANGE [$$69(ASC) ] |PARTITIONED|
+ project ([$$69, $$73]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$73] <- [$$d.getField(1)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- ASSIGN |PARTITIONED|
+ limit 3 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- STREAM_LIMIT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$69, $$d] <- test.DatasetWithKnownField limit 3 [cardinality: 1000000.0, op-cost: 2.1, total-cost: 2.1]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/broadcast-join-hint-warning/broadcast-join-hint-warning.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/broadcast-join-hint-warning/broadcast-join-hint-warning.2.adm
new file mode 100644
index 0000000..267992b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/broadcast-join-hint-warning/broadcast-join-hint-warning.2.adm
@@ -0,0 +1 @@
+{ "$1": 0 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/broadcast-join-hint-warning/broadcast-join-hint-warning.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/broadcast-join-hint-warning/broadcast-join-hint-warning.3.adm
new file mode 100644
index 0000000..267992b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/broadcast-join-hint-warning/broadcast-join-hint-warning.3.adm
@@ -0,0 +1 @@
+{ "$1": 0 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/broadcast-join-hint-warning/broadcast-join-hint-warning.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/broadcast-join-hint-warning/broadcast-join-hint-warning.4.adm
new file mode 100644
index 0000000..267992b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/broadcast-join-hint-warning/broadcast-join-hint-warning.4.adm
@@ -0,0 +1 @@
+{ "$1": 0 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/cardinality-hint-warning/cardinality-hint-warning.02.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/cardinality-hint-warning/cardinality-hint-warning.02.adm
new file mode 100644
index 0000000..3ff59f6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/cardinality-hint-warning/cardinality-hint-warning.02.adm
@@ -0,0 +1 @@
+{ "$1": 0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/cardinality-hint-warning/cardinality-hint-warning.03.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/cardinality-hint-warning/cardinality-hint-warning.03.adm
new file mode 100644
index 0000000..3ff59f6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/cardinality-hint-warning/cardinality-hint-warning.03.adm
@@ -0,0 +1 @@
+{ "$1": 0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/cardinality-hint-warning/cardinality-hint-warning.04.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/cardinality-hint-warning/cardinality-hint-warning.04.adm
new file mode 100644
index 0000000..3ff59f6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/cardinality-hint-warning/cardinality-hint-warning.04.adm
@@ -0,0 +1 @@
+{ "$1": 0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/cardinality-hint-warning/cardinality-hint-warning.05.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/cardinality-hint-warning/cardinality-hint-warning.05.adm
new file mode 100644
index 0000000..3ff59f6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/cardinality-hint-warning/cardinality-hint-warning.05.adm
@@ -0,0 +1 @@
+{ "$1": 0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/cardinality-hint-warning/cardinality-hint-warning.06.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/cardinality-hint-warning/cardinality-hint-warning.06.adm
new file mode 100644
index 0000000..3ff59f6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/cardinality-hint-warning/cardinality-hint-warning.06.adm
@@ -0,0 +1 @@
+{ "$1": 0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/cardinality-hint-warning/cardinality-hint-warning.07.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/cardinality-hint-warning/cardinality-hint-warning.07.adm
new file mode 100644
index 0000000..3ff59f6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/cardinality-hint-warning/cardinality-hint-warning.07.adm
@@ -0,0 +1 @@
+{ "$1": 0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/cardinality-hint-warning/cardinality-hint-warning.08.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/cardinality-hint-warning/cardinality-hint-warning.08.adm
new file mode 100644
index 0000000..3ff59f6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/cardinality-hint-warning/cardinality-hint-warning.08.adm
@@ -0,0 +1 @@
+{ "$1": 0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/cardinality-hint-warning/cardinality-hint-warning.09.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/cardinality-hint-warning/cardinality-hint-warning.09.adm
new file mode 100644
index 0000000..3ff59f6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/cardinality-hint-warning/cardinality-hint-warning.09.adm
@@ -0,0 +1 @@
+{ "$1": 0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/cardinality-hint-warning/cardinality-hint-warning.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/cardinality-hint-warning/cardinality-hint-warning.10.adm
new file mode 100644
index 0000000..3ff59f6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/cardinality-hint-warning/cardinality-hint-warning.10.adm
@@ -0,0 +1 @@
+{ "$1": 0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/cardinality-hint-warning/cardinality-hint-warning.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/cardinality-hint-warning/cardinality-hint-warning.11.adm
new file mode 100644
index 0000000..3ff59f6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/cardinality-hint-warning/cardinality-hint-warning.11.adm
@@ -0,0 +1 @@
+{ "$1": 0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/cardinality-hint-warning/cardinality-hint-warning.12.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/cardinality-hint-warning/cardinality-hint-warning.12.adm
new file mode 100644
index 0000000..3ff59f6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/cardinality-hint-warning/cardinality-hint-warning.12.adm
@@ -0,0 +1 @@
+{ "$1": 0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/cardinality-hint-warning/cardinality-hint-warning.13.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/cardinality-hint-warning/cardinality-hint-warning.13.adm
new file mode 100644
index 0000000..3ff59f6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/cardinality-hint-warning/cardinality-hint-warning.13.adm
@@ -0,0 +1 @@
+{ "$1": 0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/cardinality-hint-warning/cardinality-hint-warning.14.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/cardinality-hint-warning/cardinality-hint-warning.14.adm
new file mode 100644
index 0000000..3ff59f6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/cardinality-hint-warning/cardinality-hint-warning.14.adm
@@ -0,0 +1 @@
+{ "$1": 0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/hashjoin-hint-warning/hashjoin-hint-warning.02.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/hashjoin-hint-warning/hashjoin-hint-warning.02.adm
new file mode 100644
index 0000000..267992b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/hashjoin-hint-warning/hashjoin-hint-warning.02.adm
@@ -0,0 +1 @@
+{ "$1": 0 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/hashjoin-hint-warning/hashjoin-hint-warning.03.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/hashjoin-hint-warning/hashjoin-hint-warning.03.adm
new file mode 100644
index 0000000..267992b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/hashjoin-hint-warning/hashjoin-hint-warning.03.adm
@@ -0,0 +1 @@
+{ "$1": 0 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/hashjoin-hint-warning/hashjoin-hint-warning.04.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/hashjoin-hint-warning/hashjoin-hint-warning.04.adm
new file mode 100644
index 0000000..267992b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/hashjoin-hint-warning/hashjoin-hint-warning.04.adm
@@ -0,0 +1 @@
+{ "$1": 0 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/hashjoin-hint-warning/hashjoin-hint-warning.05.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/hashjoin-hint-warning/hashjoin-hint-warning.05.adm
new file mode 100644
index 0000000..267992b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/hashjoin-hint-warning/hashjoin-hint-warning.05.adm
@@ -0,0 +1 @@
+{ "$1": 0 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/hashjoin-hint-warning/hashjoin-hint-warning.06.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/hashjoin-hint-warning/hashjoin-hint-warning.06.adm
new file mode 100644
index 0000000..267992b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/hashjoin-hint-warning/hashjoin-hint-warning.06.adm
@@ -0,0 +1 @@
+{ "$1": 0 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/hashjoin-hint-warning/hashjoin-hint-warning.07.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/hashjoin-hint-warning/hashjoin-hint-warning.07.adm
new file mode 100644
index 0000000..267992b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/hashjoin-hint-warning/hashjoin-hint-warning.07.adm
@@ -0,0 +1 @@
+{ "$1": 0 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/hashjoin-hint-warning/hashjoin-hint-warning.08.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/hashjoin-hint-warning/hashjoin-hint-warning.08.adm
new file mode 100644
index 0000000..267992b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/hashjoin-hint-warning/hashjoin-hint-warning.08.adm
@@ -0,0 +1 @@
+{ "$1": 0 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/hashjoin-hint-warning/hashjoin-hint-warning.09.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/hashjoin-hint-warning/hashjoin-hint-warning.09.adm
new file mode 100644
index 0000000..267992b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/hashjoin-hint-warning/hashjoin-hint-warning.09.adm
@@ -0,0 +1 @@
+{ "$1": 0 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/hashjoin-hint-warning/hashjoin-hint-warning.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/hashjoin-hint-warning/hashjoin-hint-warning.10.adm
new file mode 100644
index 0000000..267992b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/hashjoin-hint-warning/hashjoin-hint-warning.10.adm
@@ -0,0 +1 @@
+{ "$1": 0 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/hashjoin-hint-warning/hashjoin-hint-warning.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/hashjoin-hint-warning/hashjoin-hint-warning.11.adm
new file mode 100644
index 0000000..267992b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/hashjoin-hint-warning/hashjoin-hint-warning.11.adm
@@ -0,0 +1 @@
+{ "$1": 0 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/hashjoin-hint-warning/hashjoin-hint-warning.12.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/hashjoin-hint-warning/hashjoin-hint-warning.12.adm
new file mode 100644
index 0000000..267992b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/hashjoin-hint-warning/hashjoin-hint-warning.12.adm
@@ -0,0 +1 @@
+{ "$1": 0 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/multiple-hints-warning/multiple-hints-warning.02.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/multiple-hints-warning/multiple-hints-warning.02.adm
new file mode 100644
index 0000000..3ff59f6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/multiple-hints-warning/multiple-hints-warning.02.adm
@@ -0,0 +1 @@
+{ "$1": 0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/multiple-hints-warning/multiple-hints-warning.03.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/multiple-hints-warning/multiple-hints-warning.03.adm
new file mode 100644
index 0000000..3ff59f6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/multiple-hints-warning/multiple-hints-warning.03.adm
@@ -0,0 +1 @@
+{ "$1": 0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/multiple-hints-warning/multiple-hints-warning.04.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/multiple-hints-warning/multiple-hints-warning.04.adm
new file mode 100644
index 0000000..3ff59f6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/multiple-hints-warning/multiple-hints-warning.04.adm
@@ -0,0 +1 @@
+{ "$1": 0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/open-closed/query-issue453-2/query-issue453-2.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/open-closed/query-issue453-2/query-issue453-2.3.ast
index ffedb2d..057e63a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/open-closed/query-issue453-2/query-issue453-2.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/open-closed/query-issue453-2/query-issue453-2.3.ast
@@ -1,7 +1,26 @@
DataverseUse test
Query:
-SELECT ELEMENT [
-Variable [ Name=$d ]
+SELECT [
+FieldAccessor [
+ Variable [ Name=$d ]
+ Field=int_m
+]
+int_m
+FieldAccessor [
+ Variable [ Name=$d ]
+ Field=int_o
+]
+int_o
+FieldAccessor [
+ Variable [ Name=$d ]
+ Field=string_m
+]
+string_m
+FieldAccessor [
+ Variable [ Name=$d ]
+ Field=string_o
+]
+string_o
]
FROM [ FunctionCall asterix.dataset@1[
LiteralExpr [STRING] [test.DataOpen]
@@ -11,7 +30,22 @@
Orderby
FieldAccessor [
Variable [ Name=$d ]
- Field=id
+ Field=int_m
+ ]
+ ASC
+ FieldAccessor [
+ Variable [ Name=$d ]
+ Field=int_o
+ ]
+ ASC
+ FieldAccessor [
+ Variable [ Name=$d ]
+ Field=string_m
+ ]
+ ASC
+ FieldAccessor [
+ Variable [ Name=$d ]
+ Field=string_o
]
ASC
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/sqlpp_queries.xml b/asterixdb/asterix-app/src/test/resources/runtimets/sqlpp_queries.xml
new file mode 100644
index 0000000..b0c4c44
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/sqlpp_queries.xml
@@ -0,0 +1,16090 @@
+<!--
+ ! Licensed to the Apache Software Foundation (ASF) under one
+ ! or more contributor license agreements. See the NOTICE file
+ ! distributed with this work for additional information
+ ! regarding copyright ownership. The ASF licenses this file
+ ! to you under the Apache License, Version 2.0 (the
+ ! "License"); you may not use this file except in compliance
+ ! with the License. You may obtain a copy of the License at
+ !
+ ! http://www.apache.org/licenses/LICENSE-2.0
+ !
+ ! Unless required by applicable law or agreed to in writing,
+ ! software distributed under the License is distributed on an
+ ! "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ ! KIND, either express or implied. See the License for the
+ ! specific language governing permissions and limitations
+ ! under the License.
+ !-->
+<test-group name="sqlpp_queries">
+ <test-group name="api">
+ <test-case FilePath="api">
+ <compilation-unit name="compileonly">
+ <output-dir compare="Text">compileonly</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="api">
+ <compilation-unit name="compileonly-2">
+ <output-dir compare="Text">compileonly-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="api">
+ <compilation-unit name="multiple-param-values">
+ <output-dir compare="Text">multiple-param-values</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="api">
+ <compilation-unit name="readonly-request">
+ <output-dir compare="Text">readonly-request</output-dir>
+ <expected-error>ASX0044: CREATE_DATAVERSE statement is not supported in read-only mode</expected-error>
+ <expected-error>ASX0044: DATAVERSE_DROP statement is not supported in read-only mode</expected-error>
+ <expected-error>ASX0044: DATASET_DECL statement is not supported in read-only mode</expected-error>
+ <expected-error>ASX0044: DATASET_DROP statement is not supported in read-only mode</expected-error>
+ <expected-error>ASX0044: CREATE_INDEX statement is not supported in read-only mode</expected-error>
+ <expected-error>ASX0044: INDEX_DROP statement is not supported in read-only mode</expected-error>
+ <expected-error>ASX0044: INSERT statement is not supported in read-only mode</expected-error>
+ <expected-error>ASX0044: UPSERT statement is not supported in read-only mode</expected-error>
+ <expected-error>ASX0044: DELETE statement is not supported in read-only mode</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="api" check-warnings="true">
+ <compilation-unit name="request-dataverse">
+ <output-dir compare="Text">request-dataverse</output-dir>
+ <expected-warn>ASX1063: Cannot find dataverse with name testUnknown</expected-warn>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="api">
+ <compilation-unit name="request-param-validation">
+ <output-dir compare="Text">request-param-validation</output-dir>
+ <expected-error>Invalid value for parameter 'format': foo</expected-error>
+ <expected-error>Invalid value for parameter 'pretty': bar</expected-error>
+ <expected-error>Invalid value for parameter 'plan-format': blah</expected-error>
+ <expected-error>Invalid value for parameter 'max-result-reads': foo</expected-error>
+ <expected-error>Invalid value for parameter 'max-result-reads': 9999999999999999999999999999999999999999</expected-error>
+ <expected-error>Invalid value for parameter 'max-warnings': baz</expected-error>
+ <expected-error>Invalid value for parameter 'max-warnings': 1.5</expected-error>
+ <expected-error>Invalid value for parameter 'mode': asyn</expected-error>
+ <expected-error>Invalid value for parameter 'timeout': 12</expected-error>
+ <expected-error>Invalid value for parameter 'args': 12</expected-error>
+ <expected-error>Unable to process JSON content in request</expected-error>
+ <expected-error>Unable to process JSON content in request</expected-error>
+ <expected-error>Invalid value for parameter 'format': foo</expected-error>
+ <expected-error>Invalid value for parameter 'pretty': bar</expected-error>
+ <expected-error>Invalid value for parameter 'plan-format': blah</expected-error>
+ <expected-error>Invalid value for parameter 'max-result-reads': foo</expected-error>
+ <expected-error>Invalid value for parameter 'max-warnings': baz</expected-error>
+ <expected-error>Invalid value for parameter 'mode': asyn</expected-error>
+ <expected-error>Invalid value for parameter 'args': 12</expected-error>
+ <expected-error>Unable to process JSON content in request</expected-error>
+ <expected-error>Unable to process JSON content in request</expected-error>
+ <expected-error>Invalid value for parameter 'profile': true</expected-error>
+ <expected-error>Invalid value for parameter 'profile': true</expected-error>
+ <expected-error>Invalid value for parameter 'profile': foo</expected-error>
+ <expected-error>Invalid value for parameter 'profile': foo</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="api">
+ <compilation-unit name="request-param-validation-400-BAD">
+ <output-dir compare="Text">request-param-validation-400-BAD</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="api">
+ <compilation-unit name="request-param">
+ <output-dir compare="Text">request-param</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="api">
+ <compilation-unit name="format-param-in-accept-01">
+ <output-dir compare="Text">format-param-in-accept-01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="api">
+ <compilation-unit name="format-param-in-accept-02">
+ <output-dir compare="Clean-JSON">format-param-in-accept-02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="api">
+ <compilation-unit name="format-param-in-accept-03">
+ <output-dir compare="Lossless-JSON">format-param-in-accept-03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="api">
+ <compilation-unit name="format-param-in-accept-04">
+ <output-dir compare="Text">format-param-in-accept-04</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="api">
+ <compilation-unit name="format-param-in-accept-05">
+ <output-dir compare="AST">format-param-in-accept-05</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="api">
+ <compilation-unit name="get-non-query">
+ <output-dir compare="Text">get-non-query</output-dir>
+ <expected-error>CREATE_DATAVERSE statement is not supported in read-only mode</expected-error>
+ <expected-error>TYPE_DECL statement is not supported in read-only mode</expected-error>
+ <expected-error>CREATE_FUNCTION statement is not supported in read-only mode</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="api">
+ <compilation-unit name="get-query">
+ <output-dir compare="Text">get-query</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="api">
+ <compilation-unit name="post-non-query">
+ <output-dir compare="Text">post-non-query</output-dir>
+ <expected-error>TYPE_DECL statement is not supported in read-only mode</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="api">
+ <compilation-unit name="signature">
+ <output-dir compare="Text">signature</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="api">
+ <compilation-unit name="ignore-body-for-get">
+ <output-dir compare="Text">ignore-body-for-get</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="flwor">
+ <test-case FilePath="flwor">
+ <compilation-unit name="at00">
+ <output-dir compare="Text">at00</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="at01">
+ <output-dir compare="Text">at01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="at02">
+ <output-dir compare="Text">at02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="at03">
+ <output-dir compare="Text">at03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="at04">
+ <output-dir compare="Text">at04</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="at05">
+ <output-dir compare="Text">at05</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="at06">
+ <output-dir compare="Text">at06</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="at07">
+ <output-dir compare="Text">at07</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="order-by-13">
+ <output-dir compare="Text">order-by-13</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="query-issue550">
+ <output-dir compare="Text">query-issue550</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="query-ASTERIXDB-883">
+ <output-dir compare="Text">query-ASTERIXDB-883</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="query-ASTERIXDB-1576">
+ <output-dir compare="Text">query-ASTERIXDB-883</output-dir>
+ <expected-error>Duplicate alias definitions: samptable1</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="query-ASTERIXDB-1576-2">
+ <output-dir compare="Text">query-ASTERIXDB-883</output-dir>
+ <expected-error>Duplicate alias definitions: s2</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="query-ASTERIXDB-1576-3">
+ <output-dir compare="Text">query-ASTERIXDB-883</output-dir>
+ <expected-error>Duplicate alias definitions: s1</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor" check-warnings="true">
+ <compilation-unit name="query-ASTERIXDB-2446">
+ <output-dir compare="Text">query-ASTERIXDB-2446</output-dir>
+ <expected-error>ASX0013: Duplicate field name 'a'</expected-error>
+ <expected-warn>Duplicate field name 'c' (in line 28, at column 84)</expected-warn>
+ <expected-warn>Duplicate field name 'e' (in line 28, at column 116)</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="query-ASTERIXDB-2446-2">
+ <output-dir compare="Text">query-ASTERIXDB-883</output-dir>
+ <expected-error>ASX0013: Duplicate field name 'a' (in line 27, at column 20)</expected-error>
+ <expected-error>ASX0013: Duplicate field name 'b' (in line 27, at column 20)</expected-error>
+ <expected-error>ASX0013: Duplicate field name 'c' (in line 27, at column 11)</expected-error>
+ <expected-error>ASX0013: Duplicate field name 'd' (in line 27, at column 11)</expected-error>
+ <expected-error>ASX0013: Duplicate field name 'e' (in line 27, at column 14)</expected-error>
+ <expected-error>ASX0013: Duplicate field name 'f' (in line 27, at column 11)</expected-error>
+ <expected-error>ASX0013: Duplicate field name 'g' (in line 27, at column 11)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="select-let">
+ <output-dir compare="Text">select-let</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="let33">
+ <output-dir compare="Text">let33</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="at00">
+ <output-dir compare="Text">at00</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="query-ASTERIXDB-1463">
+ <output-dir compare="Text">query-ASTERIXDB-1485</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="query-ASTERIXDB-1485">
+ <output-dir compare="Text">query-ASTERIXDB-1485</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="sorting">
+ <test-case FilePath="sorting">
+ <compilation-unit name="arrays">
+ <output-dir compare="Text">arrays</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="sorting">
+ <compilation-unit name="range_hint">
+ <output-dir compare="Text">range_hint</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="sorting">
+ <compilation-unit name="records">
+ <output-dir compare="Text">records</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="explain">
+ <test-case FilePath="explain">
+ <compilation-unit name="explain_simple">
+ <output-dir compare="Text">explain_simple</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="explain">
+ <compilation-unit name="explain_caps">
+ <output-dir compare="Text">explain_simple</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="explain">
+ <compilation-unit name="explain_pretty">
+ <parameter name="pretty" value="true" />
+ <output-dir compare="Text">explain_simple</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="explain">
+ <compilation-unit name="explain_object_constructor-01">
+ <output-dir compare="Text">explain_object_constructor-01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="explain">
+ <compilation-unit name="explain_object_constructor-02">
+ <output-dir compare="Text">explain_object_constructor-02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="explain">
+ <compilation-unit name="explain_object_constructor-03">
+ <output-dir compare="Text">explain_object_constructor-03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="explain">
+ <compilation-unit name="explain_field_access">
+ <output-dir compare="Text">explain_field_access</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="explain">
+ <compilation-unit name="explain_field_access_closed">
+ <output-dir compare="Text">explain_field_access_closed</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="explain">
+ <compilation-unit name="explain_negative">
+ <output-dir compare="Text">explain_simple</output-dir>
+ <expected-error>ASX1001: Syntax error: EXPLAIN is not supported for this kind of statement</expected-error>
+ <expected-error>ASX1001: Syntax error: EXPLAIN is not supported for this kind of statement</expected-error>
+ <expected-error>ASX1001: Syntax error: EXPLAIN is not supported for this kind of statement</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="explain">
+ <compilation-unit name="explain_same_datasource_function_different_arguments">
+ <output-dir compare="Text">explain_same_datasource_function_different_arguments</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="aggregate">
+ <test-case FilePath="aggregate">
+ <compilation-unit name="issue531_string_min_max">
+ <output-dir compare="Text">issue531_string_min_max</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="agg_null">
+ <output-dir compare="Text">agg_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="agg_null_rec">
+ <output-dir compare="Text">agg_null_rec</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="agg_null_rec_1">
+ <output-dir compare="Text">agg_null_rec_1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="agg_number_rec">
+ <output-dir compare="Text">agg_number_rec</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate" check-warnings="true">
+ <compilation-unit name="avg_mixed">
+ <output-dir compare="Text">avg_mixed</output-dir>
+ <expected-warn>Unsupported type: agg-avg cannot process input type string (in line 26, at column 16)</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate" check-warnings="true">
+ <compilation-unit name="serial_avg_mixed">
+ <output-dir compare="Text">serial_avg_mixed</output-dir>
+ <expected-warn>Unsupported type: agg-avg cannot process input type string (in line 29, at column 39)</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="group_only">
+ <output-dir compare="Text">group_only</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="min_mixed">
+ <output-dir compare="Text">min_mixed</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="stddev_mixed">
+ <output-dir compare="Text">stddev_mixed</output-dir>
+ <expected-error>Type incompatibility: function agg-stddev_samp gets incompatible input values: string and float</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_stddev_mixed">
+ <output-dir compare="Text">serial_stddev_mixed</output-dir>
+ <expected-error>Type incompatibility: function agg-stddev_samp gets incompatible input values: string and float</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="stddev_pop_mixed">
+ <output-dir compare="Text">stddev_pop_mixed</output-dir>
+ <expected-error>Type incompatibility: function agg-stddev_pop gets incompatible input values: string and float</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_stddev_pop_mixed">
+ <output-dir compare="Text">serial_stddev_pop_mixed</output-dir>
+ <expected-error>Type incompatibility: function agg-stddev_pop gets incompatible input values: string and float</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate" check-warnings="true">
+ <compilation-unit name="sum/sum_mixed">
+ <output-dir compare="Text">sum/sum_mixed</output-dir>
+ <expected-warn>Unsupported type: agg-sum cannot process input type string (in line 27, at column 16)</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate" check-warnings="true">
+ <compilation-unit name="sum/serial_sum_mixed">
+ <output-dir compare="Text">sum/serial_sum_mixed</output-dir>
+ <expected-warn>Unsupported type: agg-sum cannot process input type string (in line 29, at column 39)</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="var_mixed">
+ <output-dir compare="Text">var_mixed</output-dir>
+ <expected-error>Type incompatibility: function agg-var_samp gets incompatible input values: string and float</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_var_mixed">
+ <output-dir compare="Text">serial_var_mixed</output-dir>
+ <expected-error>Type incompatibility: function agg-var_samp gets incompatible input values: string and float</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="var_pop_mixed">
+ <output-dir compare="Text">var_pop_mixed</output-dir>
+ <expected-error>Type incompatibility: function agg-var_pop gets incompatible input values: string and float</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_var_pop_mixed">
+ <output-dir compare="Text">serial_var_pop_mixed</output-dir>
+ <expected-error>Type incompatibility: function agg-var_pop gets incompatible input values: string and float</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="kurtosis_mixed">
+ <output-dir compare="Text">kurtosis_mixed</output-dir>
+ <expected-error>Type incompatibility: function agg-kurtosis gets incompatible input values: string and float</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_kurtosis_mixed">
+ <output-dir compare="Text">serial_kurtosis_mixed</output-dir>
+ <expected-error>Type incompatibility: function agg-kurtosis gets incompatible input values: string and float</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="skewness_mixed">
+ <output-dir compare="Text">skewness_mixed</output-dir>
+ <expected-error>Type incompatibility: function agg-skewness gets incompatible input values: string and float</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_skewness_mixed">
+ <output-dir compare="Text">serial_skewness_mixed</output-dir>
+ <expected-error>Type incompatibility: function agg-skewness gets incompatible input values: string and float</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="agg_number">
+ <output-dir compare="Text">agg_number</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="issue425_min_hetero_list_1">
+ <output-dir compare="Text">issue425_min_hetero_list_1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="issue425_min_hetero_list">
+ <output-dir compare="Text">issue425_min_hetero_list</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="sum/issue425_sum_hetero_list_1">
+ <output-dir compare="Text">sum/issue425_sum_hetero_list_1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="sum/issue425_sum_hetero_list">
+ <output-dir compare="Text">sum/issue425_sum_hetero_list</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="query-issue400">
+ <output-dir compare="Text">query-issue400</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="issue395">
+ <output-dir compare="Text">issue395</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="issue412_0">
+ <output-dir compare="Text">issue412_0</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="issue412_1">
+ <output-dir compare="Text">issue412_1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="avg_double">
+ <output-dir compare="Text">avg_double</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="avg_double_null">
+ <output-dir compare="Text">avg_double_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="avg_empty_01">
+ <output-dir compare="Text">avg_empty_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="avg_empty_02">
+ <output-dir compare="Text">avg_empty_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="avg_float">
+ <output-dir compare="Text">avg_float</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="avg_float_null">
+ <output-dir compare="Text">avg_float_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="avg_int16">
+ <output-dir compare="Text">avg_int16</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="avg_int16_null">
+ <output-dir compare="Text">avg_int16_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="avg_int32">
+ <output-dir compare="Text">avg_int32</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="avg_int32_null">
+ <output-dir compare="Text">avg_int32_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="avg_int64">
+ <output-dir compare="Text">avg_int64</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="avg_int64_null">
+ <output-dir compare="Text">avg_int64_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="avg_int8">
+ <output-dir compare="Text">avg_int8</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="avg_int8_null">
+ <output-dir compare="Text">avg_int8_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="avg_distinct">
+ <output-dir compare="Text">avg_distinct</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="count_01">
+ <output-dir compare="Text">count_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="count_dataset">
+ <output-dir compare="Text">count_dataset</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="count_empty_01">
+ <output-dir compare="Text">count_empty_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="count_empty_02">
+ <output-dir compare="Text">count_empty_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="count_null">
+ <output-dir compare="Text">count_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="count_distinct">
+ <output-dir compare="Text">count_distinct</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="kurtosis_double">
+ <output-dir compare="Text">kurtosis_double</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="kurtosis_double_null">
+ <output-dir compare="Text">kurtosis_double_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="kurtosis_empty_01">
+ <output-dir compare="Text">kurtosis_empty_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="kurtosis_empty_02">
+ <output-dir compare="Text">kurtosis_empty_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="kurtosis_float">
+ <output-dir compare="Text">kurtosis_float</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="kurtosis_float_null">
+ <output-dir compare="Text">kurtosis_float_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="kurtosis_int16">
+ <output-dir compare="Text">kurtosis_int16</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="kurtosis_int16_null">
+ <output-dir compare="Text">kurtosis_int16_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="kurtosis_int32">
+ <output-dir compare="Text">kurtosis_int32</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="kurtosis_int32_null">
+ <output-dir compare="Text">kurtosis_int32_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="kurtosis_int64">
+ <output-dir compare="Text">kurtosis_int64</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="kurtosis_int64_null">
+ <output-dir compare="Text">kurtosis_int64_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="kurtosis_int8">
+ <output-dir compare="Text">kurtosis_int8</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="kurtosis_int8_null">
+ <output-dir compare="Text">kurtosis_int8_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="kurtosis_distinct">
+ <output-dir compare="Text">kurtosis_distinct</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="max_empty_01">
+ <output-dir compare="Text">max_empty_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="max_empty_02">
+ <output-dir compare="Text">max_empty_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="max_distinct">
+ <output-dir compare="Text">max_distinct</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="min_empty_01">
+ <output-dir compare="Text">min_empty_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="min_empty_02">
+ <output-dir compare="Text">min_empty_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="min_distinct">
+ <output-dir compare="Text">min_distinct</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="scalar_avg">
+ <output-dir compare="Text">scalar_avg</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="scalar_avg_empty">
+ <output-dir compare="Text">scalar_avg_empty</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="scalar_avg_null">
+ <output-dir compare="Text">scalar_avg_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="scalar_count">
+ <output-dir compare="Text">scalar_count</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="scalar_count_empty">
+ <output-dir compare="Text">scalar_count_empty</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="scalar_count_null">
+ <output-dir compare="Text">scalar_count_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="scalar_kurtosis">
+ <output-dir compare="Text">scalar_kurtosis</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="scalar_kurtosis_empty">
+ <output-dir compare="Text">scalar_kurtosis_empty</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="scalar_kurtosis_null">
+ <output-dir compare="Text">scalar_kurtosis_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="scalar_max">
+ <output-dir compare="Text">scalar_max</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="scalar_max_empty">
+ <output-dir compare="Text">scalar_max_empty</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="scalar_max_null">
+ <output-dir compare="Text">scalar_max_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="scalar_min">
+ <output-dir compare="Text">scalar_min</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="scalar_min_empty">
+ <output-dir compare="Text">scalar_min_empty</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="scalar_min_null">
+ <output-dir compare="Text">scalar_min_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="scalar_skewness">
+ <output-dir compare="Text">scalar_skewness</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="scalar_skewness_empty">
+ <output-dir compare="Text">scalar_skewness_empty</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="scalar_skewness_null">
+ <output-dir compare="Text">scalar_skewness_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="scalar_stddev">
+ <output-dir compare="Text">scalar_stddev</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="scalar_stddev_empty">
+ <output-dir compare="Text">scalar_stddev_empty</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="scalar_stddev_null">
+ <output-dir compare="Text">scalar_stddev_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="sum/scalar_sum">
+ <output-dir compare="Text">sum/scalar_sum</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="sum/scalar_sum_empty">
+ <output-dir compare="Text">sum/scalar_sum_empty</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="sum/scalar_sum_null">
+ <output-dir compare="Text">sum/scalar_sum_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="sum/scalar_sum_type">
+ <output-dir compare="Text">sum/scalar_sum_type</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="scalar_var">
+ <output-dir compare="Text">scalar_var</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="scalar_var_empty">
+ <output-dir compare="Text">scalar_var_empty</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="scalar_var_null">
+ <output-dir compare="Text">scalar_var_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_avg_empty">
+ <output-dir compare="Text">serial_avg_empty</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_avg_int8">
+ <output-dir compare="Text">serial_avg_int8</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_avg_int8_null">
+ <output-dir compare="Text">serial_avg_int8_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_avg_int16">
+ <output-dir compare="Text">serial_avg_int16</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_avg_int16_null">
+ <output-dir compare="Text">serial_avg_int16_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_avg_int32">
+ <output-dir compare="Text">serial_avg_int32</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_avg_int32_null">
+ <output-dir compare="Text">serial_avg_int32_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_avg_int64">
+ <output-dir compare="Text">serial_avg_int64</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_avg_int64_null">
+ <output-dir compare="Text">serial_avg_int64_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_avg_float">
+ <output-dir compare="Text">serial_avg_float</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_avg_float_null">
+ <output-dir compare="Text">serial_avg_float_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_avg_double">
+ <output-dir compare="Text">serial_avg_double</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_avg_double_null">
+ <output-dir compare="Text">serial_avg_double_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_kurtosis_double">
+ <output-dir compare="Text">serial_kurtosis_double</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_kurtosis_double_null">
+ <output-dir compare="Text">serial_kurtosis_double_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_kurtosis_empty">
+ <output-dir compare="Text">serial_kurtosis_empty</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_kurtosis_float">
+ <output-dir compare="Text">serial_kurtosis_float</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_kurtosis_float_null">
+ <output-dir compare="Text">serial_kurtosis_float_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_kurtosis_int16">
+ <output-dir compare="Text">serial_kurtosis_int16</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_kurtosis_int16_null">
+ <output-dir compare="Text">serial_kurtosis_int16_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_kurtosis_int32">
+ <output-dir compare="Text">serial_kurtosis_int32</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_kurtosis_int32_null">
+ <output-dir compare="Text">serial_kurtosis_int32_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_kurtosis_int64">
+ <output-dir compare="Text">serial_kurtosis_int64</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_kurtosis_int64_null">
+ <output-dir compare="Text">serial_kurtosis_int64_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_kurtosis_int8">
+ <output-dir compare="Text">serial_kurtosis_int8</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_kurtosis_int8_null">
+ <output-dir compare="Text">serial_kurtosis_int8_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_skewness_double">
+ <output-dir compare="Text">serial_skewness_double</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_skewness_double_null">
+ <output-dir compare="Text">serial_skewness_double_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_skewness_empty">
+ <output-dir compare="Text">serial_skewness_empty</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_skewness_float">
+ <output-dir compare="Text">serial_skewness_float</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_skewness_float_null">
+ <output-dir compare="Text">serial_skewness_float_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_skewness_int16">
+ <output-dir compare="Text">serial_skewness_int16</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_skewness_int16_null">
+ <output-dir compare="Text">serial_skewness_int16_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_skewness_int32">
+ <output-dir compare="Text">serial_skewness_int32</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_skewness_int32_null">
+ <output-dir compare="Text">serial_skewness_int32_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_skewness_int64">
+ <output-dir compare="Text">serial_skewness_int64</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_skewness_int64_null">
+ <output-dir compare="Text">serial_skewness_int64_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_skewness_int8">
+ <output-dir compare="Text">serial_skewness_int8</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_skewness_int8_null">
+ <output-dir compare="Text">serial_skewness_int8_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_stddev_double">
+ <output-dir compare="Text">serial_stddev_double</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_stddev_double_null">
+ <output-dir compare="Text">serial_stddev_double_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_stddev_empty">
+ <output-dir compare="Text">serial_stddev_empty</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_stddev_float">
+ <output-dir compare="Text">serial_stddev_float</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_stddev_float_null">
+ <output-dir compare="Text">serial_stddev_float_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_stddev_int16">
+ <output-dir compare="Text">serial_stddev_int16</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_stddev_int16_null">
+ <output-dir compare="Text">serial_stddev_int16_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_stddev_int32">
+ <output-dir compare="Text">serial_stddev_int32</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_stddev_int32_null">
+ <output-dir compare="Text">serial_stddev_int32_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_stddev_int64">
+ <output-dir compare="Text">serial_stddev_int64</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_stddev_int64_null">
+ <output-dir compare="Text">serial_stddev_int64_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_stddev_int8">
+ <output-dir compare="Text">serial_stddev_int8</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_stddev_int8_null">
+ <output-dir compare="Text">serial_stddev_int8_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="sum/serial_sum_double">
+ <output-dir compare="Text">sum/serial_sum_double</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="sum/serial_sum_double_null">
+ <output-dir compare="Text">sum/serial_sum_double_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="sum/serial_sum_empty">
+ <output-dir compare="Text">sum/serial_sum_empty</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="sum/serial_sum_float">
+ <output-dir compare="Text">sum/serial_sum_float</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="sum/serial_sum_float_null">
+ <output-dir compare="Text">sum/serial_sum_float_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="sum/serial_sum_int16">
+ <output-dir compare="Text">sum/serial_sum_int16</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="sum/serial_sum_int16_null">
+ <output-dir compare="Text">sum/serial_sum_int16_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="sum/serial_sum_int32">
+ <output-dir compare="Text">sum/serial_sum_int32</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="sum/serial_sum_int32_null">
+ <output-dir compare="Text">sum/serial_sum_int32_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="sum/serial_sum_int64">
+ <output-dir compare="Text">sum/serial_sum_int64</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="sum/serial_sum_int64_null">
+ <output-dir compare="Text">sum/serial_sum_int64_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="sum/serial_sum_int64_overflow">
+ <output-dir compare="Text">sum/serial_sum_int64_overflow</output-dir>
+ <expected-error>Overflow in agg-sum</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="sum/serial_sum_int8">
+ <output-dir compare="Text">sum/serial_sum_int8</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="sum/serial_sum_int8_null">
+ <output-dir compare="Text">sum/serial_sum_int8_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_var_double">
+ <output-dir compare="Text">serial_var_double</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_var_double_null">
+ <output-dir compare="Text">serial_var_double_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_var_empty">
+ <output-dir compare="Text">serial_var_empty</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_var_float">
+ <output-dir compare="Text">serial_var_float</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_var_float_null">
+ <output-dir compare="Text">serial_var_float_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_var_int16">
+ <output-dir compare="Text">serial_var_int16</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_var_int16_null">
+ <output-dir compare="Text">serial_var_int16_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_var_int32">
+ <output-dir compare="Text">serial_var_int32</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_var_int32_null">
+ <output-dir compare="Text">serial_var_int32_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_var_int64">
+ <output-dir compare="Text">serial_var_int64</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_var_int64_null">
+ <output-dir compare="Text">serial_var_int64_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_var_int8">
+ <output-dir compare="Text">serial_var_int8</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_var_int8_null">
+ <output-dir compare="Text">serial_var_int8_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="skewness_double">
+ <output-dir compare="Text">skewness_double</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="skewness_double_null">
+ <output-dir compare="Text">skewness_double_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="skewness_empty_01">
+ <output-dir compare="Text">skewness_empty_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="skewness_empty_02">
+ <output-dir compare="Text">skewness_empty_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="skewness_float">
+ <output-dir compare="Text">skewness_float</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="skewness_float_null">
+ <output-dir compare="Text">skewness_float_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="skewness_int16">
+ <output-dir compare="Text">skewness_int16</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="skewness_int16_null">
+ <output-dir compare="Text">skewness_int16_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="skewness_int32">
+ <output-dir compare="Text">skewness_int32</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="skewness_int32_null">
+ <output-dir compare="Text">skewness_int32_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="skewness_int64">
+ <output-dir compare="Text">skewness_int64</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="skewness_int64_null">
+ <output-dir compare="Text">skewness_int64_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="skewness_int8">
+ <output-dir compare="Text">skewness_int8</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="skewness_int8_null">
+ <output-dir compare="Text">skewness_int8_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="skewness_distinct">
+ <output-dir compare="Text">skewness_distinct</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="stddev_double">
+ <output-dir compare="Text">stddev_double</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="stddev_double_null">
+ <output-dir compare="Text">stddev_double_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="stddev_empty_01">
+ <output-dir compare="Text">stddev_empty_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="stddev_empty_02">
+ <output-dir compare="Text">stddev_empty_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="stddev_float">
+ <output-dir compare="Text">stddev_float</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="stddev_float_null">
+ <output-dir compare="Text">stddev_float_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="stddev_int16">
+ <output-dir compare="Text">stddev_int16</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="stddev_int16_null">
+ <output-dir compare="Text">stddev_int16_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="stddev_int32">
+ <output-dir compare="Text">stddev_int32</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="stddev_int32_null">
+ <output-dir compare="Text">stddev_int32_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="stddev_int64">
+ <output-dir compare="Text">stddev_int64</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="stddev_int64_null">
+ <output-dir compare="Text">stddev_int64_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="stddev_int8">
+ <output-dir compare="Text">stddev_int8</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="stddev_int8_null">
+ <output-dir compare="Text">stddev_int8_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="stddev_distinct">
+ <output-dir compare="Text">stddev_distinct</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="stddev_misc">
+ <output-dir compare="Text">stddev_misc</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="stddev_pop_misc">
+ <output-dir compare="Text">stddev_pop_misc</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="sum/sum_double">
+ <output-dir compare="Text">sum/sum_double</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="sum/sum_double_null">
+ <output-dir compare="Text">sum/sum_double_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="sum/sum_empty_01">
+ <output-dir compare="Text">sum/sum_empty_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="sum/sum_empty_02">
+ <output-dir compare="Text">sum/sum_empty_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="sum/sum_float">
+ <output-dir compare="Text">sum/sum_float</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="sum/sum_float_null">
+ <output-dir compare="Text">sum/sum_float_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="sum/sum_int16">
+ <output-dir compare="Text">sum/sum_int16</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="sum/sum_int16_null">
+ <output-dir compare="Text">sum/sum_int16_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="sum/sum_int32">
+ <output-dir compare="Text">sum/sum_int32</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="sum/sum_int32_null">
+ <output-dir compare="Text">sum/sum_int32_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="sum/sum_int64">
+ <output-dir compare="Text">sum/sum_int64</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="sum/sum_int64_null">
+ <output-dir compare="Text">sum/sum_int64_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="sum/sum_int64_null">
+ <output-dir compare="Text">sum/sum_int64_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="sum/sum_int64_overflow">
+ <output-dir compare="Text">sum/sum_int64_overflow</output-dir>
+ <expected-error>Overflow in agg-sum</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="sum/sum_int8">
+ <output-dir compare="Text">sum/sum_int8</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="sum/sum_int8_null">
+ <output-dir compare="Text">sum/sum_int8_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="sum/sum_null-with-pred">
+ <output-dir compare="Text">sum/sum_null-with-pred</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="sum/sum_numeric_null">
+ <output-dir compare="Text">sum/sum_numeric_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="sum/sum_distinct">
+ <output-dir compare="Text">sum/sum_distinct</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="var_double">
+ <output-dir compare="Text">var_double</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="var_double_null">
+ <output-dir compare="Text">var_double_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="var_empty_01">
+ <output-dir compare="Text">var_empty_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="var_empty_02">
+ <output-dir compare="Text">var_empty_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="var_float">
+ <output-dir compare="Text">var_float</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="var_float_null">
+ <output-dir compare="Text">var_float_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="var_int16">
+ <output-dir compare="Text">var_int16</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="var_int16_null">
+ <output-dir compare="Text">var_int16_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="var_int32">
+ <output-dir compare="Text">var_int32</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="var_int32_null">
+ <output-dir compare="Text">var_int32_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="var_int64">
+ <output-dir compare="Text">var_int64</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="var_int64_null">
+ <output-dir compare="Text">var_int64_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="var_int8">
+ <output-dir compare="Text">var_int8</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="var_int8_null">
+ <output-dir compare="Text">var_int8_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="var_distinct">
+ <output-dir compare="Text">var_distinct</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="var_misc">
+ <output-dir compare="Text">var_misc</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="var_pop_misc">
+ <output-dir compare="Text">var_pop_misc</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="min_max_arrays">
+ <output-dir compare="Text">min_max_arrays</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="non-pure-function">
+ <output-dir compare="Text">non-pure-function</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="aggregate-sql">
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="median">
+ <output-dir compare="Text">median</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="min_max_arrays">
+ <output-dir compare="Text">min_max_arrays</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="issue531_string_min_max">
+ <output-dir compare="Text">issue531_string_min_max</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="agg_null">
+ <output-dir compare="Text">agg_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="agg_null_rec">
+ <output-dir compare="Text">agg_null_rec</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="agg_null_rec_1">
+ <output-dir compare="Text">agg_null_rec_1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="agg_number_rec">
+ <output-dir compare="Text">agg_number_rec</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql" check-warnings="true">
+ <compilation-unit name="avg_mixed">
+ <output-dir compare="Text">avg_mixed</output-dir>
+ <expected-warn>Unsupported type: agg-avg cannot process input type string (in line 26, at column 16)</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql" check-warnings="true">
+ <compilation-unit name="serial_avg_mixed">
+ <output-dir compare="Text">serial_avg_mixed</output-dir>
+ <expected-warn>Unsupported type: agg-avg cannot process input type string (in line 29, at column 38)</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="min_mixed">
+ <output-dir compare="Text">min_mixed</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_stddev_mixed">
+ <output-dir compare="Text">serial_stddev_mixed</output-dir>
+ <expected-error>Invalid item type: function agg-stddev_samp cannot process item type string in an input array (or multiset)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="stddev_mixed">
+ <output-dir compare="Text">stddev_mixed</output-dir>
+ <expected-error>Type incompatibility: function agg-stddev_samp gets incompatible input values: string and float</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_stddev_pop_mixed">
+ <output-dir compare="Text">serial_stddev_pop_mixed</output-dir>
+ <expected-error>Invalid item type: function agg-stddev_pop cannot process item type string in an input array (or multiset)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="stddev_pop_mixed">
+ <output-dir compare="Text">stddev_pop_mixed</output-dir>
+ <expected-error>Type incompatibility: function agg-stddev_pop gets incompatible input values: string and float</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql" check-warnings="true">
+ <compilation-unit name="sum/sum_mixed">
+ <output-dir compare="Text">sum/sum_mixed</output-dir>
+ <expected-warn>Unsupported type: agg-sum cannot process input type string (in line 27, at column 16)</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql" check-warnings="true">
+ <compilation-unit name="sum/serial_sum_mixed">
+ <output-dir compare="Text">sum/serial_sum_mixed</output-dir>
+ <expected-warn>Unsupported type: agg-sum cannot process input type string (in line 29, at column 38)</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_var_mixed">
+ <output-dir compare="Text">serial_var_mixed</output-dir>
+ <expected-error>Invalid item type: function agg-var_samp cannot process item type string in an input array (or multiset)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="var_mixed">
+ <output-dir compare="Text">var_mixed</output-dir>
+ <expected-error>Type incompatibility: function agg-var_samp gets incompatible input values: string and float</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_var_pop_mixed">
+ <output-dir compare="Text">serial_var_pop_mixed</output-dir>
+ <expected-error>Invalid item type: function agg-var_pop cannot process item type string in an input array (or multiset)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="var_pop_mixed">
+ <output-dir compare="Text">var_pop_mixed</output-dir>
+ <expected-error>Type incompatibility: function agg-var_pop gets incompatible input values: string and float</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_skewness_mixed">
+ <output-dir compare="Text">serial_skewness_mixed</output-dir>
+ <expected-error>Invalid item type: function agg-skewness cannot process item type string in an input array (or multiset)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="skewness_mixed">
+ <output-dir compare="Text">skewness_mixed</output-dir>
+ <expected-error>Type incompatibility: function agg-skewness gets incompatible input values: string and float</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="agg_number">
+ <output-dir compare="Text">agg_number</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="issue425_min_hetero_list_1">
+ <output-dir compare="Text">issue425_min_hetero_list_1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="issue425_min_hetero_list">
+ <output-dir compare="Text">issue425_min_hetero_list</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="sum/issue425_sum_hetero_list_1">
+ <output-dir compare="Text">sum/issue425_sum_hetero_list_1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="sum/issue425_sum_hetero_list">
+ <output-dir compare="Text">sum/issue425_sum_hetero_list</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="query-issue400">
+ <output-dir compare="Text">query-issue400</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="issue395">
+ <output-dir compare="Text">issue395</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="issue412_0">
+ <output-dir compare="Text">issue412_0</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="issue412_1">
+ <output-dir compare="Text">issue412_1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="issue2348">
+ <output-dir compare="Text">issue2348</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="issue2411">
+ <output-dir compare="Text">issue2411</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="avg_double">
+ <output-dir compare="Text">avg_double</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="avg_double_null">
+ <output-dir compare="Text">avg_double_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="avg_empty_01">
+ <output-dir compare="Text">avg_empty_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="avg_empty_02">
+ <output-dir compare="Text">avg_empty_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="avg_float">
+ <output-dir compare="Text">avg_float</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="avg_float_null">
+ <output-dir compare="Text">avg_float_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="avg_int16">
+ <output-dir compare="Text">avg_int16</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="avg_int16_null">
+ <output-dir compare="Text">avg_int16_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="avg_int32">
+ <output-dir compare="Text">avg_int32</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="avg_int32_null">
+ <output-dir compare="Text">avg_int32_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="avg_int64">
+ <output-dir compare="Text">avg_int64</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="avg_int64_null">
+ <output-dir compare="Text">avg_int64_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="avg_int8">
+ <output-dir compare="Text">avg_int8</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="avg_int8_null">
+ <output-dir compare="Text">avg_int8_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="avg_distinct">
+ <output-dir compare="Text">avg_distinct</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="count_01">
+ <output-dir compare="Text">count_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="count_dataset">
+ <output-dir compare="Text">count_dataset</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="count_empty_01">
+ <output-dir compare="Text">count_empty_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="count_empty_02">
+ <output-dir compare="Text">count_empty_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="count_null">
+ <output-dir compare="Text">count_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="count_distinct">
+ <output-dir compare="Text">count_distinct</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="kurtosis_double">
+ <output-dir compare="Text">kurtosis_double</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="kurtosis_double_null">
+ <output-dir compare="Text">kurtosis_double_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="kurtosis_empty_01">
+ <output-dir compare="Text">kurtosis_empty_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="kurtosis_empty_02">
+ <output-dir compare="Text">kurtosis_empty_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="kurtosis_float">
+ <output-dir compare="Text">kurtosis_float</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="kurtosis_float_null">
+ <output-dir compare="Text">kurtosis_float_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="kurtosis_int16">
+ <output-dir compare="Text">kurtosis_int16</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="kurtosis_int16_null">
+ <output-dir compare="Text">kurtosis_int16_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="kurtosis_int32">
+ <output-dir compare="Text">kurtosis_int32</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="kurtosis_int32_null">
+ <output-dir compare="Text">kurtosis_int32_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="kurtosis_int64">
+ <output-dir compare="Text">kurtosis_int64</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="kurtosis_int64_null">
+ <output-dir compare="Text">kurtosis_int64_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="kurtosis_int8">
+ <output-dir compare="Text">kurtosis_int8</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="kurtosis_int8_null">
+ <output-dir compare="Text">kurtosis_int8_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="kurtosis_distinct">
+ <output-dir compare="Text">kurtosis_distinct</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="max_empty_01">
+ <output-dir compare="Text">max_empty_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="max_empty_02">
+ <output-dir compare="Text">max_empty_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="max_distinct">
+ <output-dir compare="Text">max_distinct</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="min_empty_01">
+ <output-dir compare="Text">min_empty_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="min_empty_02">
+ <output-dir compare="Text">min_empty_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="min_distinct">
+ <output-dir compare="Text">min_distinct</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="scalar_avg">
+ <output-dir compare="Text">scalar_avg</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="scalar_avg_empty">
+ <output-dir compare="Text">scalar_avg_empty</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="scalar_avg_null">
+ <output-dir compare="Text">scalar_avg_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="scalar_count">
+ <output-dir compare="Text">scalar_count</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="scalar_count_empty">
+ <output-dir compare="Text">scalar_count_empty</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="scalar_count_null">
+ <output-dir compare="Text">scalar_count_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="scalar_kurtosis">
+ <output-dir compare="Text">scalar_kurtosis</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="scalar_kurtosis_empty">
+ <output-dir compare="Text">scalar_kurtosis_empty</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="scalar_kurtosis_null">
+ <output-dir compare="Text">scalar_kurtosis_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="scalar_max">
+ <output-dir compare="Text">scalar_max</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="scalar_max_empty">
+ <output-dir compare="Text">scalar_max_empty</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="scalar_max_null">
+ <output-dir compare="Text">scalar_max_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="scalar_min">
+ <output-dir compare="Text">scalar_min</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="scalar_min_empty">
+ <output-dir compare="Text">scalar_min_empty</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="scalar_min_null">
+ <output-dir compare="Text">scalar_min_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="scalar_skewness">
+ <output-dir compare="Text">scalar_skewness</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="scalar_skewness_empty">
+ <output-dir compare="Text">scalar_skewness_empty</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="scalar_skewness_null">
+ <output-dir compare="Text">scalar_skewness_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="scalar_stddev">
+ <output-dir compare="Text">scalar_stddev</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="scalar_stddev_empty">
+ <output-dir compare="Text">scalar_stddev_empty</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="scalar_stddev_null">
+ <output-dir compare="Text">scalar_stddev_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="sum/scalar_sum">
+ <output-dir compare="Text">sum/scalar_sum</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="sum/scalar_sum_empty">
+ <output-dir compare="Text">sum/scalar_sum_empty</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="sum/scalar_sum_null">
+ <output-dir compare="Text">sum/scalar_sum_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="sum/scalar_sum_type">
+ <output-dir compare="Text">sum/scalar_sum_type</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="scalar_var">
+ <output-dir compare="Text">scalar_var</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="scalar_var_empty">
+ <output-dir compare="Text">scalar_var_empty</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="scalar_var_null">
+ <output-dir compare="Text">scalar_var_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_avg_empty">
+ <output-dir compare="Text">serial_avg_empty</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_avg_int8">
+ <output-dir compare="Text">serial_avg_int8</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_avg_int8_null">
+ <output-dir compare="Text">serial_avg_int8_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_avg_int16">
+ <output-dir compare="Text">serial_avg_int16</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_avg_int16_null">
+ <output-dir compare="Text">serial_avg_int16_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_avg_int32">
+ <output-dir compare="Text">serial_avg_int32</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_avg_int32_null">
+ <output-dir compare="Text">serial_avg_int32_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_avg_int64">
+ <output-dir compare="Text">serial_avg_int64</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_avg_int64_null">
+ <output-dir compare="Text">serial_avg_int64_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_avg_float">
+ <output-dir compare="Text">serial_avg_float</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_avg_float_null">
+ <output-dir compare="Text">serial_avg_float_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_avg_double">
+ <output-dir compare="Text">serial_avg_double</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_avg_double_null">
+ <output-dir compare="Text">serial_avg_double_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_kurtosis_double">
+ <output-dir compare="Text">serial_kurtosis_double</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_kurtosis_double_null">
+ <output-dir compare="Text">serial_kurtosis_double_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_kurtosis_empty">
+ <output-dir compare="Text">serial_kurtosis_empty</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_kurtosis_float">
+ <output-dir compare="Text">serial_kurtosis_float</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_kurtosis_float_null">
+ <output-dir compare="Text">serial_kurtosis_float_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_kurtosis_int16">
+ <output-dir compare="Text">serial_kurtosis_int16</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_kurtosis_int16_null">
+ <output-dir compare="Text">serial_kurtosis_int16_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_kurtosis_int32">
+ <output-dir compare="Text">serial_kurtosis_int32</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_kurtosis_int32_null">
+ <output-dir compare="Text">serial_kurtosis_int32_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_kurtosis_int64">
+ <output-dir compare="Text">serial_kurtosis_int64</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_kurtosis_int64_null">
+ <output-dir compare="Text">serial_kurtosis_int64_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_kurtosis_int8">
+ <output-dir compare="Text">serial_kurtosis_int8</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_kurtosis_int8_null">
+ <output-dir compare="Text">serial_kurtosis_int8_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_skewness_double">
+ <output-dir compare="Text">serial_skewness_double</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_skewness_double_null">
+ <output-dir compare="Text">serial_skewness_double_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_skewness_empty">
+ <output-dir compare="Text">serial_skewness_empty</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_skewness_float">
+ <output-dir compare="Text">serial_skewness_float</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_skewness_float_null">
+ <output-dir compare="Text">serial_skewness_float_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_skewness_int16">
+ <output-dir compare="Text">serial_skewness_int16</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_skewness_int16_null">
+ <output-dir compare="Text">serial_skewness_int16_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_skewness_int32">
+ <output-dir compare="Text">serial_skewness_int32</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_skewness_int32_null">
+ <output-dir compare="Text">serial_skewness_int32_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_skewness_int64">
+ <output-dir compare="Text">serial_skewness_int64</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_skewness_int64_null">
+ <output-dir compare="Text">serial_skewness_int64_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_skewness_int8">
+ <output-dir compare="Text">serial_skewness_int8</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_skewness_int8_null">
+ <output-dir compare="Text">serial_skewness_int8_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_stddev_double">
+ <output-dir compare="Text">serial_stddev_double</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_stddev_double_null">
+ <output-dir compare="Text">serial_stddev_double_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_stddev_empty">
+ <output-dir compare="Text">serial_stddev_empty</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_stddev_float">
+ <output-dir compare="Text">serial_stddev_float</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_stddev_float_null">
+ <output-dir compare="Text">serial_stddev_float_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_stddev_int16">
+ <output-dir compare="Text">serial_stddev_int16</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_stddev_int16_null">
+ <output-dir compare="Text">serial_stddev_int16_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_stddev_int32">
+ <output-dir compare="Text">serial_stddev_int32</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_stddev_int32_null">
+ <output-dir compare="Text">serial_stddev_int32_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_stddev_int64">
+ <output-dir compare="Text">serial_stddev_int64</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_stddev_int64_null">
+ <output-dir compare="Text">serial_stddev_int64_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_stddev_int8">
+ <output-dir compare="Text">serial_stddev_int8</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_stddev_int8_null">
+ <output-dir compare="Text">serial_stddev_int8_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="sum/serial_sum_double">
+ <output-dir compare="Text">sum/serial_sum_double</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="sum/serial_sum_double_null">
+ <output-dir compare="Text">sum/serial_sum_double_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="sum/serial_sum_empty">
+ <output-dir compare="Text">sum/serial_sum_empty</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="sum/serial_sum_float">
+ <output-dir compare="Text">sum/serial_sum_float</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="sum/serial_sum_float_null">
+ <output-dir compare="Text">sum/serial_sum_float_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="sum/serial_sum_int16">
+ <output-dir compare="Text">sum/serial_sum_int16</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="sum/serial_sum_int16_null">
+ <output-dir compare="Text">sum/serial_sum_int16_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="sum/serial_sum_int32">
+ <output-dir compare="Text">sum/serial_sum_int32</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="sum/serial_sum_int32_null">
+ <output-dir compare="Text">sum/serial_sum_int32_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="sum/serial_sum_int64">
+ <output-dir compare="Text">sum/serial_sum_int64</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="sum/serial_sum_int64_null">
+ <output-dir compare="Text">sum/serial_sum_int64_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="sum/serial_sum_int64_overflow">
+ <output-dir compare="Text">sum/serial_sum_int64_overflow</output-dir>
+ <expected-error>Overflow in agg-sum</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="sum/serial_sum_int8">
+ <output-dir compare="Text">sum/serial_sum_int8</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="sum/serial_sum_int8_null">
+ <output-dir compare="Text">sum/serial_sum_int8_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_var_double">
+ <output-dir compare="Text">serial_var_double</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_var_double_null">
+ <output-dir compare="Text">serial_var_double_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_var_empty">
+ <output-dir compare="Text">serial_var_empty</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_var_float">
+ <output-dir compare="Text">serial_var_float</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_var_float_null">
+ <output-dir compare="Text">serial_var_float_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_var_int16">
+ <output-dir compare="Text">serial_var_int16</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_var_int16_null">
+ <output-dir compare="Text">serial_var_int16_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_var_int32">
+ <output-dir compare="Text">serial_var_int32</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_var_int32_null">
+ <output-dir compare="Text">serial_var_int32_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_var_int64">
+ <output-dir compare="Text">serial_var_int64</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_var_int64_null">
+ <output-dir compare="Text">serial_var_int64_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_var_int8">
+ <output-dir compare="Text">serial_var_int8</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_var_int8_null">
+ <output-dir compare="Text">serial_var_int8_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="skewness_double">
+ <output-dir compare="Text">skewness_double</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="skewness_double_null">
+ <output-dir compare="Text">skewness_double_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="skewness_empty_01">
+ <output-dir compare="Text">skewness_empty_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="skewness_empty_02">
+ <output-dir compare="Text">skewness_empty_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="skewness_float">
+ <output-dir compare="Text">skewness_float</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="skewness_float_null">
+ <output-dir compare="Text">skewness_float_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="skewness_int16">
+ <output-dir compare="Text">skewness_int16</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="skewness_int16_null">
+ <output-dir compare="Text">skewness_int16_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="skewness_int32">
+ <output-dir compare="Text">skewness_int32</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="skewness_int32_null">
+ <output-dir compare="Text">skewness_int32_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="skewness_int64">
+ <output-dir compare="Text">skewness_int64</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="skewness_int64_null">
+ <output-dir compare="Text">skewness_int64_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="skewness_int8">
+ <output-dir compare="Text">skewness_int8</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="skewness_int8_null">
+ <output-dir compare="Text">skewness_int8_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="skewness_distinct">
+ <output-dir compare="Text">skewness_distinct</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="stddev_double">
+ <output-dir compare="Text">stddev_double</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="stddev_double_null">
+ <output-dir compare="Text">stddev_double_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="stddev_empty_01">
+ <output-dir compare="Text">stddev_empty_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="stddev_empty_02">
+ <output-dir compare="Text">stddev_empty_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="stddev_float">
+ <output-dir compare="Text">stddev_float</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="stddev_float_null">
+ <output-dir compare="Text">stddev_float_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="stddev_int16">
+ <output-dir compare="Text">stddev_int16</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="stddev_int16_null">
+ <output-dir compare="Text">stddev_int16_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="stddev_int32">
+ <output-dir compare="Text">stddev_int32</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="stddev_int32_null">
+ <output-dir compare="Text">stddev_int32_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="stddev_int64">
+ <output-dir compare="Text">stddev_int64</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="stddev_int64_null">
+ <output-dir compare="Text">stddev_int64_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="stddev_int8">
+ <output-dir compare="Text">stddev_int8</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="stddev_int8_null">
+ <output-dir compare="Text">stddev_int8_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="stddev_distinct">
+ <output-dir compare="Text">stddev_distinct</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="stddev_misc">
+ <output-dir compare="Text">stddev_misc</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="stddev_pop_misc">
+ <output-dir compare="Text">stddev_pop_misc</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="sum/sum_double">
+ <output-dir compare="Text">sum/sum_double</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="sum/sum_double_null">
+ <output-dir compare="Text">sum/sum_double_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="sum/sum_empty_01">
+ <output-dir compare="Text">sum/sum_empty_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="sum/sum_empty_02">
+ <output-dir compare="Text">sum/sum_empty_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="sum/sum_float">
+ <output-dir compare="Text">sum/sum_float</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="sum/sum_float_null">
+ <output-dir compare="Text">sum/sum_float_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="sum/sum_int16">
+ <output-dir compare="Text">sum/sum_int16</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="sum/sum_int16_null">
+ <output-dir compare="Text">sum/sum_int16_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="sum/sum_int32">
+ <output-dir compare="Text">sum/sum_int32</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="sum/sum_int32_null">
+ <output-dir compare="Text">sum/sum_int32_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="sum/sum_int64">
+ <output-dir compare="Text">sum/sum_int64</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="sum/sum_int64_null">
+ <output-dir compare="Text">sum/sum_int64_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="sum/sum_int64_overflow">
+ <output-dir compare="Text">sum/sum_int64_overflow</output-dir>
+ <expected-error>Overflow in agg-sum</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="sum/sum_int8">
+ <output-dir compare="Text">sum/sum_int8</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="sum/sum_int8_null">
+ <output-dir compare="Text">sum/sum_int8_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="sum/sum_null-with-pred">
+ <output-dir compare="Text">sum/sum_null-with-pred</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="sum/sum_numeric_null">
+ <output-dir compare="Text">sum/sum_numeric_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="sum/sum_distinct">
+ <output-dir compare="Text">sum/sum_distinct</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="var_double">
+ <output-dir compare="Text">var_double</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="var_double_null">
+ <output-dir compare="Text">var_double_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="var_empty_01">
+ <output-dir compare="Text">var_empty_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="var_empty_02">
+ <output-dir compare="Text">var_empty_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="var_float">
+ <output-dir compare="Text">var_float</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="var_float_null">
+ <output-dir compare="Text">var_float_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="var_int16">
+ <output-dir compare="Text">var_int16</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="var_int16_null">
+ <output-dir compare="Text">var_int16_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="var_int32">
+ <output-dir compare="Text">var_int32</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="var_int32_null">
+ <output-dir compare="Text">var_int32_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="var_int64">
+ <output-dir compare="Text">var_int64</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="var_int64_null">
+ <output-dir compare="Text">var_int64_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="var_int8">
+ <output-dir compare="Text">var_int8</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="var_int8_null">
+ <output-dir compare="Text">var_int8_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="var_distinct">
+ <output-dir compare="Text">var_distinct</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="var_misc">
+ <output-dir compare="Text">var_misc</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="var_pop_misc">
+ <output-dir compare="Text">var_pop_misc</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="aggregate-sql-sugar">
+ <test-case FilePath="aggregate-sql-sugar">
+ <compilation-unit name="array_agg">
+ <output-dir compare="Text">array_agg</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql-sugar">
+ <compilation-unit name="array_agg_negative">
+ <output-dir compare="Text">array_agg</output-dir>
+ <expected-error>ASX1079: Compilation error: arrayagg is a SQL-92 aggregate function. The SQL++ core aggregate function strict_arrayagg could potentially express the intent.</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql-sugar" check-warnings="true">
+ <compilation-unit name="avg_mixed">
+ <output-dir compare="Text">avg_mixed</output-dir>
+ <expected-warn>ASX0004: Unsupported type: agg-avg cannot process input type string (in line 26, at column 12)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: agg-avg cannot process input type string (in line 28, at column 19)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: agg-avg cannot process input type string (in line 29, at column 19)</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql-sugar">
+ <compilation-unit name="distinct_mixed">
+ <output-dir compare="Text">distinct_mixed</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql-sugar">
+ <compilation-unit name="stddev">
+ <output-dir compare="Text">stddev</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql-sugar">
+ <compilation-unit name="var">
+ <output-dir compare="Text">var</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="aggregate-subclause">
+ <test-case FilePath="aggregate-subclause">
+ <compilation-unit name="agg_filter_01">
+ <output-dir compare="Text">agg_filter_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-subclause">
+ <compilation-unit name="agg_filter_02_neg">
+ <output-dir compare="Text">agg_filter_01</output-dir>
+ <expected-error>ASX1121: Illegal use of aggregate FILTER clause</expected-error>
+ <expected-error>ASX1121: Illegal use of aggregate FILTER clause</expected-error>
+ <expected-error>ASX1073: Cannot resolve alias reference for undefined identifier r</expected-error>
+ <expected-error>ASX1121: Illegal use of aggregate FILTER clause</expected-error>
+ <expected-error>ASX1121: Illegal use of aggregate FILTER clause</expected-error>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="array_fun">
+ <test-case FilePath="array_fun">
+ <compilation-unit name="array_prepend">
+ <output-dir compare="Text">array_prepend</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array_fun">
+ <compilation-unit name="array_append">
+ <output-dir compare="Text">array_append</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array_fun">
+ <compilation-unit name="array_position">
+ <output-dir compare="Text">array_position</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array_fun">
+ <compilation-unit name="array_repeat">
+ <output-dir compare="Text">array_repeat</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array_fun">
+ <compilation-unit name="array_reverse">
+ <output-dir compare="Text">array_reverse</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array_fun">
+ <compilation-unit name="array_contains">
+ <output-dir compare="Text">array_contains</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array_fun">
+ <compilation-unit name="array_insert">
+ <output-dir compare="Text">array_insert</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array_fun">
+ <compilation-unit name="array_put">
+ <output-dir compare="Text">array_put</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array_fun">
+ <compilation-unit name="array_remove">
+ <output-dir compare="Text">array_remove</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array_fun">
+ <compilation-unit name="array_distinct">
+ <output-dir compare="Text">array_distinct</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array_fun">
+ <compilation-unit name="array_sort">
+ <output-dir compare="Text">array_sort</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array_fun">
+ <compilation-unit name="array_concat">
+ <output-dir compare="Text">array_concat</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array_fun">
+ <compilation-unit name="array_flatten">
+ <output-dir compare="Text">array_flatten</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array_fun">
+ <compilation-unit name="array_ifnull">
+ <output-dir compare="Text">array_ifnull</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array_fun">
+ <compilation-unit name="array_intersect">
+ <output-dir compare="Text">array_intersect</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array_fun">
+ <compilation-unit name="array_range">
+ <output-dir compare="Text">array_range</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array_fun">
+ <compilation-unit name="array_replace">
+ <output-dir compare="Text">array_replace</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array_fun">
+ <compilation-unit name="array_star">
+ <output-dir compare="Text">array_star</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array_fun">
+ <compilation-unit name="array_symdiff">
+ <output-dir compare="Text">array_symdiff</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array_fun">
+ <compilation-unit name="array_symdiffn">
+ <output-dir compare="Text">array_symdiffn</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array_fun">
+ <compilation-unit name="array_union">
+ <output-dir compare="Text">array_union</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array_fun">
+ <compilation-unit name="array_slice/array_slice_double_argument">
+ <output-dir compare="Text">array_slice/array_slice_double_argument</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array_fun">
+ <compilation-unit name="array_slice/array_slice_int_argument">
+ <output-dir compare="Text">array_slice/array_slice_int_argument</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array_fun">
+ <compilation-unit name="array_slice/array_slice_negative_argument">
+ <output-dir compare="Text">array_slice/array_slice_negative_argument</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array_fun">
+ <compilation-unit name="array_slice/array_slice_missing_result">
+ <output-dir compare="Text">array_slice/array_slice_missing_result</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array_fun">
+ <compilation-unit name="array_slice/array_slice_null_result">
+ <output-dir compare="Text">array_slice/array_slice_null_result</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array_fun">
+ <compilation-unit name="array_slice/array_slice_exception_result">
+ <output-dir compare="Text">array_slice/array_slice_exception_result</output-dir>
+ <expected-error>ASX1081: Cannot find function with signature TinySocial.array_slice()</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array_fun">
+ <compilation-unit name="array_except/001">
+ <output-dir compare="Text">array_except/001</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array_fun">
+ <compilation-unit name="array_except/002">
+ <output-dir compare="Text">array_except/002</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array_fun">
+ <compilation-unit name="array_except/003">
+ <output-dir compare="Text">array_except/003</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array_fun">
+ <compilation-unit name="array_except/004">
+ <output-dir compare="Text">array_except/004</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array_fun">
+ <compilation-unit name="array_except/005">
+ <output-dir compare="Text">array_except/005</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array_fun">
+ <compilation-unit name="array_except/006">
+ <output-dir compare="Text">array_except/006</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array_fun">
+ <compilation-unit name="array_errors">
+ <output-dir compare="Text">array_errors</output-dir>
+ <expected-error>Input contains different list types (in line 25, at column 8)</expected-error>
+ <expected-error>Input contains different list types (in line 25, at column 8)</expected-error>
+ <expected-error>Input contains different list types (in line 25, at column 8)</expected-error>
+ <expected-error>Input contains different list types (in line 25, at column 8)</expected-error>
+ <expected-error>Input contains different list types (in line 25, at column 8)</expected-error>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="bitwise">
+ <test-case FilePath="bitwise">
+ <compilation-unit name="bit_and_01">
+ <output-dir compare="Text">bit_and_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="bitwise">
+ <compilation-unit name="bit_and_02">
+ <output-dir compare="Text">bit_and_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="bitwise">
+ <compilation-unit name="bit_and_03">
+ <output-dir compare="Text">bit_and_03</output-dir>
+ <expected-error>Invalid number of arguments for function</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="bitwise">
+ <compilation-unit name="bit_and_04">
+ <output-dir compare="Text">bit_and_04</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="bitwise">
+ <compilation-unit name="bit_or_01">
+ <output-dir compare="Text">bit_or_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="bitwise">
+ <compilation-unit name="bit_or_02">
+ <output-dir compare="Text">bit_or_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="bitwise">
+ <compilation-unit name="bit_or_03">
+ <output-dir compare="Text">bit_or_03</output-dir>
+ <expected-error>Invalid number of arguments for function</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="bitwise">
+ <compilation-unit name="bit_or_04">
+ <output-dir compare="Text">bit_or_04</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="bitwise">
+ <compilation-unit name="bit_xor_01">
+ <output-dir compare="Text">bit_xor_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="bitwise">
+ <compilation-unit name="bit_xor_02">
+ <output-dir compare="Text">bit_xor_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="bitwise">
+ <compilation-unit name="bit_xor_03">
+ <output-dir compare="Text">bit_xor_03</output-dir>
+ <expected-error>Invalid number of arguments for function</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="bitwise">
+ <compilation-unit name="bit_xor_04">
+ <output-dir compare="Text">bit_xor_04</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="bitwise">
+ <compilation-unit name="bit_not_01">
+ <output-dir compare="Text">bit_not_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="bitwise">
+ <compilation-unit name="bit_not_02">
+ <output-dir compare="Text">bit_not_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="bitwise">
+ <compilation-unit name="bit_not_03">
+ <output-dir compare="Text">bit_not_03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="bitwise">
+ <compilation-unit name="bit_count_01">
+ <output-dir compare="Text">bit_count_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="bitwise">
+ <compilation-unit name="bit_count_02">
+ <output-dir compare="Text">bit_count_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="bitwise">
+ <compilation-unit name="bit_count_03">
+ <output-dir compare="Text">bit_count_03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="bitwise">
+ <compilation-unit name="bit_set_01">
+ <output-dir compare="Text">bit_set_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="bitwise">
+ <compilation-unit name="bit_set_02">
+ <output-dir compare="Text">bit_set_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="bitwise">
+ <compilation-unit name="bit_set_03">
+ <output-dir compare="Text">bit_set_03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="bitwise">
+ <compilation-unit name="bit_clear_01">
+ <output-dir compare="Text">bit_clear_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="bitwise">
+ <compilation-unit name="bit_clear_02">
+ <output-dir compare="Text">bit_clear_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="bitwise">
+ <compilation-unit name="bit_clear_03">
+ <output-dir compare="Text">bit_clear_03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="bitwise">
+ <compilation-unit name="bit_shift_01">
+ <output-dir compare="Text">bit_shift_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="bitwise">
+ <compilation-unit name="bit_shift_02">
+ <output-dir compare="Text">bit_shift_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="bitwise">
+ <compilation-unit name="bit_shift_03">
+ <output-dir compare="Text">bit_shift_03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="bitwise">
+ <compilation-unit name="bit_shift_04">
+ <output-dir compare="Text">bit_shift_04</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="bitwise">
+ <compilation-unit name="bit_test_01">
+ <output-dir compare="Text">bit_test_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="bitwise">
+ <compilation-unit name="bit_test_02">
+ <output-dir compare="Text">bit_test_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="bitwise">
+ <compilation-unit name="bit_test_03">
+ <output-dir compare="Text">bit_test_03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="bitwise">
+ <compilation-unit name="bit_test_04">
+ <output-dir compare="Text">bit_test_04</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="bitwise">
+ <compilation-unit name="is_bit_set_01">
+ <output-dir compare="Text">is_bit_set_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="bitwise">
+ <compilation-unit name="is_bit_set_02">
+ <output-dir compare="Text">is_bit_set_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="bitwise">
+ <compilation-unit name="is_bit_set_03">
+ <output-dir compare="Text">is_bit_set_03</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="boolean">
+ <test-case FilePath="boolean">
+ <compilation-unit name="and_01">
+ <output-dir compare="Text">and_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="boolean">
+ <compilation-unit name="and_null">
+ <output-dir compare="Text">and_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="boolean">
+ <compilation-unit name="and_null_false">
+ <output-dir compare="Text">and_null_false</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="boolean">
+ <compilation-unit name="not_01">
+ <output-dir compare="Text">not_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="comparison">
+ <test-case FilePath="comparison">
+ <compilation-unit name="secondary_idx_lookup">
+ <output-dir compare="Text">secondary_idx_lookup</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="year_month_duration_order">
+ <output-dir compare="Text">year_month_duration_order</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="datetime_order">
+ <output-dir compare="Text">datetime_order</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="datetime_range">
+ <output-dir compare="Text">datetime_range</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="datetime_range_between">
+ <output-dir compare="Text">datetime_range</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="datetime_tzeq">
+ <output-dir compare="Text">datetime_tzeq</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="double">
+ <output-dir compare="Text">double</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="double_gte_01">
+ <output-dir compare="Text">double_gte_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="double_missing">
+ <output-dir compare="Text">double_missing</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="double_null">
+ <output-dir compare="Text">double_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="eq_01">
+ <output-dir compare="Text">eq_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="float">
+ <output-dir compare="Text">float</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="float_missing">
+ <output-dir compare="Text">float_missing</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="float_null">
+ <output-dir compare="Text">float_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="gt_01">
+ <output-dir compare="Text">gt_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="gte_01">
+ <output-dir compare="Text">gte_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="incompatible">
+ <output-dir compare="Text">incompatible</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="int16">
+ <output-dir compare="Text">int16</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="int16_missing">
+ <output-dir compare="Text">int16_missing</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="int16_null">
+ <output-dir compare="Text">int16_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="int32">
+ <output-dir compare="Text">int32</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="int32_missing">
+ <output-dir compare="Text">int32_missing</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="int32_null">
+ <output-dir compare="Text">int32_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="int64">
+ <output-dir compare="Text">int64</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="int64_missing">
+ <output-dir compare="Text">int64_missing</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="int64_null">
+ <output-dir compare="Text">int64_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="int8">
+ <output-dir compare="Text">int8</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="int8_missing">
+ <output-dir compare="Text">int8_missing</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="int8_null">
+ <output-dir compare="Text">int8_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="int_not_between">
+ <output-dir compare="Text">int_not_between</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="is_distinct_01">
+ <output-dir compare="Text">is_distinct_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="like">
+ <output-dir compare="Text">like</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="lt_01">
+ <output-dir compare="Text">lt_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="lte_01">
+ <output-dir compare="Text">lte_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="neq_01">
+ <output-dir compare="Text">neq_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="neq_02">
+ <output-dir compare="Text">neq_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="numeric-comparison_01">
+ <output-dir compare="Text">numeric-comparison_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="string">
+ <output-dir compare="Text">string</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="string_missing">
+ <output-dir compare="Text">string_missing</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="string_null">
+ <output-dir compare="Text">string_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="issue363_equality">
+ <output-dir compare="Text">issue363_equality</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="issue363_inequality_duration">
+ <output-dir compare="Text">issue363_inequality_duration</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="issue363_inequality_interval">
+ <output-dir compare="Text">issue363_inequality_interval</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="issue363_inequality_point">
+ <output-dir compare="Text">issue363_inequality_point</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="issue363_inequality_line">
+ <output-dir compare="Text">issue363_inequality_line</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="issue363_inequality_polygon">
+ <output-dir compare="Text">issue363_inequality_polygon</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="issue363_inequality_rectangle">
+ <output-dir compare="Text">issue363_inequality_rectangle</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="issue363_inequality_circle">
+ <output-dir compare="Text">issue363_inequality_circle</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="binary">
+ <output-dir compare="Text">binary</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="binary_null">
+ <output-dir compare="Text">binary_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="greatest_mixed">
+ <output-dir compare="Text">greatest_mixed</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="least_mixed">
+ <output-dir compare="Text">least_mixed</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="missingif">
+ <output-dir compare="Text">missingif</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="nullif">
+ <output-dir compare="Text">nullif</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="nanif">
+ <output-dir compare="Text">nanif</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="posinfif">
+ <output-dir compare="Text">posinfif</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="neginfif">
+ <output-dir compare="Text">neginfif</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="arrays">
+ <output-dir compare="Text">arrays</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="circle-point">
+ <output-dir compare="Text">circle-point</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison" check-warnings="true">
+ <compilation-unit name="incomparable_types">
+ <output-dir compare="Text">incomparable_types</output-dir>
+ <expected-warn>Incomparable input types: string and bigint (in line 26, at column 13)</expected-warn>
+ <expected-warn>Incomparable input types: array and bigint (in line 23, at column 7)</expected-warn>
+ <expected-warn>Incomparable input types: point and point (in line 24, at column 18)</expected-warn>
+ <expected-warn>Incomparable input types: bigint and string (in line 25, at column 46)</expected-warn>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="constructor">
+ <test-case FilePath="constructor">
+ <compilation-unit name="binary_01">
+ <output-dir compare="Text">binary_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="constructor" check-warnings="true">
+ <compilation-unit name="binary_02">
+ <output-dir compare="Text">binary_02</output-dir>
+ <expected-warn>ASX0006: Invalid format for binary in @#!1 (in line 24, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: hex() cannot process input type date (in line 25, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: hex() cannot process input type array (in line 26, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: hex() cannot process input type object (in line 27, at column 13)</expected-warn>
+ <expected-warn>ASX0006: Invalid format for binary in @#!2 (in line 24, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: base64() cannot process input type date (in line 25, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: base64() cannot process input type array (in line 26, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: base64() cannot process input type object (in line 27, at column 13)</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="constructor">
+ <compilation-unit name="add-null">
+ <output-dir compare="Text">add-null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="constructor">
+ <compilation-unit name="boolean_01">
+ <output-dir compare="Text">boolean_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="constructor" check-warnings="true">
+ <compilation-unit name="boolean_02">
+ <output-dir compare="Text">boolean_02</output-dir>
+ <expected-warn>ASX0006: Invalid format for boolean in FALSE (in line 24, at column 13)</expected-warn>
+ <expected-warn>ASX0006: Invalid format for boolean in TRUE (in line 25, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: boolean() cannot process input type date (in line 26, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: boolean() cannot process input type array (in line 27, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: boolean() cannot process input type object (in line 28, at column 13)</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="constructor">
+ <compilation-unit name="circle_01">
+ <output-dir compare="Text">circle_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="constructor">
+ <compilation-unit name="date_01">
+ <output-dir compare="Text">date_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="constructor" check-warnings="true">
+ <compilation-unit name="date_02">
+ <output-dir compare="Text">date_02</output-dir>
+ <expected-warn>ASX0006: Invalid format for date in @#! (in line 24, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: date() cannot process input type boolean (in line 25, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: date() cannot process input type bigint (in line 26, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: date() cannot process input type time (in line 27, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: date() cannot process input type array (in line 28, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: date() cannot process input type object (in line 29, at column 13)</expected-warn>
+ <expected-warn>ASX0006: Invalid format for date in 12/31/2020 (in line 30, at column 13)</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="constructor">
+ <compilation-unit name="datetime_01">
+ <output-dir compare="Text">datetime_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="constructor" check-warnings="true">
+ <compilation-unit name="datetime_02">
+ <output-dir compare="Text">datetime_02</output-dir>
+ <expected-warn>ASX0006: Invalid format for datetime in @#! (in line 24, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: datetime() cannot process input type boolean (in line 25, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: datetime() cannot process input type bigint (in line 26, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: datetime() cannot process input type array (in line 27, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: datetime() cannot process input type object (in line 28, at column 13)</expected-warn>
+ <expected-warn>ASX0006: Invalid format for datetime in 1951-12-27T12:20:15Z (in line 29, at column 13)</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="constructor">
+ <compilation-unit name="double_01">
+ <output-dir compare="Text">double_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="constructor" check-warnings="true">
+ <compilation-unit name="double_02">
+ <output-dir compare="Text">double_02</output-dir>
+ <expected-warn>ASX0006: Invalid format for double in @#! (in line 24, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: double() cannot process input type datetime (in line 25, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: double() cannot process input type date (in line 26, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: double() cannot process input type time (in line 27, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: double() cannot process input type array (in line 28, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: double() cannot process input type object (in line 29, at column 13)</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="constructor">
+ <compilation-unit name="duration_01">
+ <output-dir compare="Text">duration_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="constructor" check-warnings="true">
+ <compilation-unit name="duration_02">
+ <output-dir compare="Text">duration_02</output-dir>
+ <expected-warn>ASX0006: Invalid format for duration in @#! (in line 24, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: duration() cannot process input type bigint (in line 25, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: duration() cannot process input type datetime (in line 26, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: duration() cannot process input type date (in line 27, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: duration() cannot process input type time (in line 28, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: duration() cannot process input type array (in line 29, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: duration() cannot process input type object (in line 30, at column 13)</expected-warn>
+ <expected-warn>ASX0006: Invalid format for yearmonthduration in @#! (in line 24, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: year-month-duration() cannot process input type bigint (in line 25, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: year-month-duration() cannot process input type datetime (in line 26, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: year-month-duration() cannot process input type date (in line 27, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: year-month-duration() cannot process input type time (in line 28, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: year-month-duration() cannot process input type array (in line 29, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: year-month-duration() cannot process input type object (in line 30, at column 13)</expected-warn>
+ <expected-warn>ASX0006: Invalid format for daytimeduration in @#! (in line 24, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: day-time-duration() cannot process input type bigint (in line 25, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: day-time-duration() cannot process input type datetime (in line 26, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: day-time-duration() cannot process input type date (in line 27, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: day-time-duration() cannot process input type time (in line 28, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: day-time-duration() cannot process input type array (in line 29, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: day-time-duration() cannot process input type object (in line 30, at column 13)</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="constructor">
+ <compilation-unit name="float_01">
+ <output-dir compare="Text">float_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="constructor" check-warnings="true">
+ <compilation-unit name="float_02">
+ <output-dir compare="Text">float_02</output-dir>
+ <expected-warn>ASX0006: Invalid format for float in @#! (in line 24, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: float() cannot process input type datetime (in line 25, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: float() cannot process input type date (in line 26, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: float() cannot process input type time (in line 27, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: float() cannot process input type array (in line 28, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: float() cannot process input type object (in line 29, at column 13)</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="constructor">
+ <compilation-unit name="int_01">
+ <output-dir compare="Text">int_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="constructor" check-warnings="true">
+ <compilation-unit name="int_02">
+ <output-dir compare="Text">int_02</output-dir>
+ <expected-warn>ASX0006: Invalid format for tinyint in @#! (in line 24, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: int8() cannot process input type datetime (in line 25, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: int8() cannot process input type date (in line 26, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: int8() cannot process input type time (in line 27, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: int8() cannot process input type array (in line 28, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: int8() cannot process input type object (in line 29, at column 13)</expected-warn>
+ <expected-warn>ASX0006: Invalid format for smallint in @#! (in line 24, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: int16() cannot process input type datetime (in line 25, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: int16() cannot process input type date (in line 26, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: int16() cannot process input type time (in line 27, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: int16() cannot process input type array (in line 28, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: int16() cannot process input type object (in line 29, at column 13)</expected-warn>
+ <expected-warn>ASX0006: Invalid format for integer in @#! (in line 24, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: int32() cannot process input type datetime (in line 25, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: int32() cannot process input type date (in line 26, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: int32() cannot process input type time (in line 27, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: int32() cannot process input type array (in line 28, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: int32() cannot process input type object (in line 29, at column 13)</expected-warn>
+ <expected-warn>ASX0006: Invalid format for bigint in @#! (in line 24, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: int64() cannot process input type datetime (in line 25, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: int64() cannot process input type date (in line 26, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: int64() cannot process input type time (in line 27, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: int64() cannot process input type array (in line 28, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: int64() cannot process input type object (in line 29, at column 13)</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="constructor">
+ <compilation-unit name="interval">
+ <output-dir compare="Text">interval</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="constructor">
+ <compilation-unit name="line_01">
+ <output-dir compare="Text">line_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="constructor">
+ <compilation-unit name="rectangle_01">
+ <output-dir compare="Text">rectangle_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="constructor">
+ <compilation-unit name="point_01">
+ <output-dir compare="Text">point_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="constructor">
+ <compilation-unit name="polygon_01">
+ <output-dir compare="Text">polygon_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="constructor">
+ <compilation-unit name="primitive-01">
+ <output-dir compare="Text">primitive-01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="constructor">
+ <compilation-unit name="primitive-02">
+ <output-dir compare="Text">primitive-02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="constructor">
+ <compilation-unit name="primitive-03">
+ <output-dir compare="Text">primitive-03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="constructor">
+ <compilation-unit name="primitive-04">
+ <output-dir compare="Text">primitive-04</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="constructor">
+ <compilation-unit name="string_01">
+ <output-dir compare="Text">string_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="constructor" check-warnings="true">
+ <compilation-unit name="string_02">
+ <output-dir compare="Text">string_02</output-dir>
+ <expected-warn>ASX0004: Unsupported type: string() cannot process input type array (in line 24, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: string() cannot process input type object (in line 25, at column 13)</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="constructor">
+ <compilation-unit name="time_01">
+ <output-dir compare="Text">time_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="constructor" check-warnings="true">
+ <compilation-unit name="time_02">
+ <output-dir compare="Text">time_02</output-dir>
+ <expected-warn>ASX0006: Invalid format for time in @#! (in line 24, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: time() cannot process input type boolean (in line 25, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: time() cannot process input type bigint (in line 26, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: time() cannot process input type date (in line 27, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: time() cannot process input type array (in line 28, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: time() cannot process input type object (in line 29, at column 13)</expected-warn>
+ <expected-warn>ASX0006: Invalid format for time in 11:58:59 (in line 30, at column 13)</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="constructor/uuid">
+ <compilation-unit name="uuid_01">
+ <output-dir compare="Text">uuid_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="constructor/uuid" check-warnings="true">
+ <compilation-unit name="uuid_02">
+ <output-dir compare="Text">uuid_02</output-dir>
+ <expected-warn>ASX0006: Invalid format for uuid in 02a199ca-bf58-412e-bd9f-60a0c975a8a- (in line 24, at column 13)</expected-warn>
+ <expected-warn>ASX0006: Invalid format for uuid in 12345 (in line 25, at column 13)</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="constructor">
+ <compilation-unit name="polygon-from-open-list_issue1627">
+ <output-dir compare="Text">polygon-from-open-list_issue1627</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="custord">
+ <!--
+ <test-case FilePath="custord">
+ <compilation-unit name="co">
+ <output-dir compare="Text">co</output-dir>
+ </compilation-unit>
+ </test-case>
+ -->
+ <test-case FilePath="custord">
+ <compilation-unit name="customer_q_01">
+ <output-dir compare="Text">customer_q_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="custord">
+ <compilation-unit name="customer_q_02">
+ <output-dir compare="Text">customer_q_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="custord">
+ <compilation-unit name="customer_q_03">
+ <output-dir compare="Text">customer_q_03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="custord">
+ <compilation-unit name="customer_q_04">
+ <output-dir compare="Text">customer_q_04</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="custord">
+ <compilation-unit name="customer_q_05">
+ <output-dir compare="Text">customer_q_05</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="custord">
+ <compilation-unit name="customer_q_06">
+ <output-dir compare="Text">customer_q_06</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="custord">
+ <compilation-unit name="customer_q_07">
+ <output-dir compare="Text">customer_q_07</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="custord">
+ <compilation-unit name="customer_q_08">
+ <output-dir compare="Text">customer_q_08</output-dir>
+ </compilation-unit>
+ </test-case>
+ <!--
+ <test-case FilePath="custord">
+ <compilation-unit name="denorm-cust-order_01">
+ <output-dir compare="Text">denorm-cust-order_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ -->
+ <test-case FilePath="custord">
+ <compilation-unit name="denorm-cust-order_02">
+ <output-dir compare="Text">denorm-cust-order_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <!--
+ <test-case FilePath="custord">
+ <compilation-unit name="denorm-cust-order_03">
+ <output-dir compare="Text">denorm-cust-order_03</output-dir>
+ </compilation-unit>
+ </test-case>
+ -->
+ <!--
+ <test-case FilePath="custord">
+ <compilation-unit name="freq-clerk">
+ <output-dir compare="Text">freq-clerk</output-dir>
+ </compilation-unit>
+ </test-case>
+ -->
+ <test-case FilePath="custord">
+ <compilation-unit name="join_q_01">
+ <output-dir compare="Text">join_q_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="custord">
+ <compilation-unit name="join_q_02">
+ <output-dir compare="Text">join_q_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="custord">
+ <compilation-unit name="join_q_03">
+ <output-dir compare="Text">join_q_03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="custord">
+ <compilation-unit name="join_q_04">
+ <output-dir compare="Text">join_q_04</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="custord">
+ <compilation-unit name="join_q_05">
+ <output-dir compare="Text">join_q_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="custord">
+ <compilation-unit name="join_q_06">
+ <output-dir compare="Text">join_q_06</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="custord">
+ <compilation-unit name="join_q_07">
+ <output-dir compare="Text">join_q_06</output-dir>
+ <expected-error>Cannot find dataset c in dataverse test nor an alias with name c</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="custord">
+ <compilation-unit name="join_q_08">
+ <output-dir compare="Text">join_q_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="custord">
+ <compilation-unit name="join_q_09">
+ <output-dir compare="Text">join_q_01</output-dir>
+ <expected-error>Cannot resolve ambiguous alias reference for identifier age</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="custord">
+ <compilation-unit name="query-ASTERIXDB-1754">
+ <output-dir compare="Text">query-ASTERIXDB-1754</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="custord">
+ <compilation-unit name="load-test">
+ <output-dir compare="Text">load-test</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="custord">
+ <compilation-unit name="order_q_01">
+ <output-dir compare="Text">order_q_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="custord">
+ <compilation-unit name="order_q_02">
+ <output-dir compare="Text">order_q_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="custord">
+ <compilation-unit name="order_q_03">
+ <output-dir compare="Text">order_q_03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="custord">
+ <compilation-unit name="order_q_04">
+ <output-dir compare="Text">order_q_04</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="custord">
+ <compilation-unit name="order_q_05">
+ <output-dir compare="Text">order_q_05</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="custord">
+ <compilation-unit name="order_q_06">
+ <output-dir compare="Text">order_q_06</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="dapd">
+ <test-case FilePath="dapd">
+ <compilation-unit name="q1">
+ <output-dir compare="Text">q1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dapd">
+ <compilation-unit name="q2">
+ <output-dir compare="Text">q2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dapd">
+ <compilation-unit name="q2-2">
+ <output-dir compare="Text">q2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dapd">
+ <compilation-unit name="q2-2-negative">
+ <output-dir compare="Text">q2</output-dir>
+ <expected-error>Cannot find dataset e in dataverse test nor an alias with name e</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dapd">
+ <compilation-unit name="q2-3">
+ <output-dir compare="Text">q2</output-dir>
+ <expected-error>Cannot resolve ambiguous alias reference for identifier sig_id</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dapd">
+ <compilation-unit name="q2-4">
+ <output-dir compare="Text">q2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dapd">
+ <compilation-unit name="q2-5">
+ <output-dir compare="Text">q2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dapd">
+ <compilation-unit name="q2-6">
+ <output-dir compare="Text">q2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dapd">
+ <compilation-unit name="q2-7">
+ <output-dir compare="Text">q2-7</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dapd">
+ <compilation-unit name="q2-8">
+ <output-dir compare="Text">q2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dapd">
+ <compilation-unit name="q2-9">
+ <output-dir compare="Text">q2-9</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dapd">
+ <compilation-unit name="q2-10">
+ <output-dir compare="Text">q2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dapd">
+ <compilation-unit name="q2-11">
+ <output-dir compare="Text">q2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dapd">
+ <compilation-unit name="q2-12">
+ <output-dir compare="Text">q2-12</output-dir>
+ </compilation-unit>
+ </test-case>
+ <!--
+ <test-case FilePath="dapd">
+ <compilation-unit name="q3">
+ <output-dir compare="Text">q3</output-dir>
+ </compilation-unit>
+ </test-case>
+ -->
+ </test-group>
+ <test-group name="ddl">
+ <test-case FilePath="ddl">
+ <compilation-unit name="analyze-dataset-1">
+ <output-dir compare="Text">analyze-dataset-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="ddl">
+ <compilation-unit name="analyze-dataset-with-indexes">
+ <output-dir compare="Text">analyze-dataset-with-indexes</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="ddl/create-index">
+ <compilation-unit name="create-index-1">
+ <output-dir compare="Text">create-index-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="ddl/create-index">
+ <compilation-unit name="create-index-2">
+ <output-dir compare="Text">create-index-2</output-dir>
+ <expected-error>Syntax error: In line 53 >>create primary index sec_primary_idx1 on LineItem type rtree;<< Encountered "rtree" at column 58.</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="ddl/create-index">
+ <compilation-unit name="create-index-3">
+ <output-dir compare="Text">create-index-3</output-dir>
+ <expected-error>Syntax error: In line 53 >>create primary sec_primary_idx1 on LineItem;<< Encountered <IDENTIFIER> "sec_primary_idx1" at column 18.</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="ddl/create-index">
+ <compilation-unit name="create-index-4">
+ <output-dir compare="Text">create-index-4</output-dir>
+ <expected-error>Syntax error: In line 53 >>create primary index if not exists sec_primary_idx1 if not exists on LineItem;<< Encountered <IDENTIFIER> "sec_primary_idx1" at column 37.</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="ddl/create-index">
+ <compilation-unit name="create-index-5">
+ <output-dir compare="Text">create-index-5</output-dir>
+ <expected-error>Syntax error: In line 53 >>create primary index if exists sec_primary_idx1 on LineItem;<< Encountered "exists" at column 26.</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="ddl/create-index">
+ <compilation-unit name="create-index-6">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>ASX1050: Cannot find dataset with name LineItemView1 in dataverse test (in line 55, at column 1)</expected-error>
+ <expected-error>ASX1050: Cannot find dataset with name LineItemView2 in dataverse test (in line 60, at column 1)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="ddl/create-index">
+ <compilation-unit name="create-inverted-index-with-variable-length-primary-key">
+ <output-dir compare="Text">create-inverted-index-with-variable-length-primary-key</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="ddl">
+ <compilation-unit name="bad-type-ddl">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>ASX1079: Compilation error: Reserved type name $x</expected-error>
+ <expected-error>ASX0013: Duplicate field name 'c' (in line 29, at column 19)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="ddl">
+ <compilation-unit name="create-dataset-inline-type-1">
+ <output-dir compare="Text">create-dataset-inline-type-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="ddl">
+ <compilation-unit name="create-dataset-inline-type-2">
+ <output-dir compare="Text">create-dataset-inline-type-2</output-dir>
+ <expected-error>ASX1082: Cannot find datatype with name test.$d$t$i$Cust1</expected-error>
+ <expected-error>ASX1082: Cannot find datatype with name test.$d$t$i$Cust2</expected-error>
+ <expected-error>ASX1082: Cannot find datatype with name my_unknown_type</expected-error>
+ <expected-error>ASX0013: Duplicate field name 'c_name' (in line 25, at column 22)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="ddl">
+ <compilation-unit name="drop-primary-index">
+ <output-dir compare="Text">drop-primary-index</output-dir>
+ <expected-error>Cannot drop index 'ds'. Drop dataset 'ds' to remove this index</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="ddl" check-warnings="true">
+ <compilation-unit name="invalid-dataverse">
+ <output-dir compare="Text">invalid-dataverse</output-dir>
+ <source-location>false</source-location>
+ <expected-warn>Cannot find dataverse with name fakeDataverse (in line 22, at column 1)</expected-warn>
+ <expected-error>Cannot find dataverse with name fakeDataverse (in line 27, at column 1)</expected-error>
+ <expected-warn>Cannot find dataverse with name fakeDataverse (in line 29, at column 1)</expected-warn>
+ <expected-error>Cannot find dataverse with name fakeDataverse (in line 30, at column 1)</expected-error>
+ <expected-error>Cannot find datatype with name fakeDataverse.myType</expected-error>
+ <expected-error>Cannot find dataverse with name fakeDataverse (in line 30, at column 1)</expected-error>
+ <expected-error>Cannot find dataverse with name fakeDataverse (in line 32, at column 1)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="ddl">
+ <compilation-unit name="invalid-dataverse-name">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>ASX1115: Invalid name for a database object: ''</expected-error>
+ <expected-error>ASX1115: Invalid name for a database object: ' a'</expected-error>
+ <expected-error>ASX1115: Invalid name for a database object: ' invalid'</expected-error>
+ <expected-error>ASX1079: Compilation error: Invalid operation - Cannot create dataverse: asterix</expected-error>
+ <expected-error>ASX1079: Compilation error: Invalid operation - Cannot create dataverse: algebricks</expected-error>
+ <expected-error>ASX1115: Invalid name for a database object: '' (in line 24, at column 16)</expected-error>
+ <expected-error>ASX1115: Invalid name for a database object: 'a/b' (in line 27, at column 16)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="ddl">
+ <compilation-unit name="invalid-dataset-name">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>ASX1115: Invalid name for a database object: ''</expected-error>
+ <expected-error>ASX1115: Invalid name for a database object: ' a'</expected-error>
+ <expected-error>ASX1115: Invalid name for a database object: ' invalid'</expected-error>
+ <expected-error>ASX1115: Invalid name for a database object: 'a/b' (in line 29, at column 16)</expected-error>
+ <expected-error>ASX1115: Invalid name for a database object: 'c/d' (in line 31, at column 14)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="ddl">
+ <compilation-unit name="invalid-feed-name">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>ASX1115: Invalid name for a database object: ''</expected-error>
+ <expected-error>ASX1115: Invalid name for a database object: ' a'</expected-error>
+ <expected-error>ASX1115: Invalid name for a database object: ' invalid'</expected-error>
+ <expected-error>ASX1115: Invalid name for a database object: 'a/b' (in line 34, at column 13)</expected-error>
+ <expected-error>ASX1115: Invalid name for a database object: 'c/d' (in line 42, at column 11)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="ddl">
+ <compilation-unit name="invalid-feed-policy-name">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>ASX1115: Invalid name for a database object: ''</expected-error>
+ <expected-error>ASX1115: Invalid name for a database object: ' a'</expected-error>
+ <expected-error>ASX1115: Invalid name for a database object: 'a/b' (in line 29, at column 1)</expected-error>
+ <expected-error>ASX1115: Invalid name for a database object: 'c/d' (in line 32, at column 23)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="ddl">
+ <compilation-unit name="invalid-index-name">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>ASX1115: Invalid name for a database object: ''</expected-error>
+ <expected-error>ASX1115: Invalid name for a database object: ' a'</expected-error>
+ <expected-error>ASX1115: Invalid name for a database object: ' invalid'</expected-error>
+ <expected-error>ASX1115: Invalid name for a database object: 'a/b' (in line 29, at column 19)</expected-error>
+ <expected-error>ASX1115: Invalid name for a database object: 'c/d' (in line 33, at column 12)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="ddl">
+ <compilation-unit name="invalid-nodegroup-name">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>ASX1115: Invalid name for a database object: ''</expected-error>
+ <expected-error>ASX1115: Invalid name for a database object: ' a'</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="ddl">
+ <compilation-unit name="invalid-type-name">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>ASX1115: Invalid name for a database object: ''</expected-error>
+ <expected-error>ASX1115: Invalid name for a database object: ' a'</expected-error>
+ <expected-error>ASX1115: Invalid name for a database object: ' invalid'</expected-error>
+ <expected-error>ASX1115: Invalid name for a database object: 'a/b' (in line 29, at column 13)</expected-error>
+ <expected-error>ASX1115: Invalid name for a database object: 'c/d' (in line 33, at column 11)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="ddl">
+ <compilation-unit name="invalid-udf-name">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>ASX1115: Invalid name for a database object: ''</expected-error>
+ <expected-error>ASX1115: Invalid name for a database object: ' a'</expected-error>
+ <expected-error>ASX1115: Invalid name for a database object: ' invalid'</expected-error>
+ <expected-error>ASX1115: Invalid name for a database object: 'a/b' (in line 29, at column 17)</expected-error>
+ <expected-error>ASX1115: Invalid name for a database object: 'c/d' (in line 33, at column 15)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="ddl">
+ <compilation-unit name="invalid-view-name">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>ASX1115: Invalid name for a database object: '' (in line 29, at column 1)</expected-error>
+ <expected-error>ASX1115: Invalid name for a database object: ' a' (in line 29, at column 1)</expected-error>
+ <expected-error>ASX1115: Invalid name for a database object: ' invalid' (in line 29, at column 1)</expected-error>
+ <expected-error>ASX1115: Invalid name for a database object: 'a/b' (in line 29, at column 13)</expected-error>
+ <expected-error>ASX1115: Invalid name for a database object: 'c/d' (in line 32, at column 11)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="ddl">
+ <compilation-unit name="dataset-and-index-same-dataverse">
+ <output-dir compare="Text">dataset-and-index-same-dataverse</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="ddl" check-warnings="true">
+ <compilation-unit name="drop_dataset_invalid_dataverse">
+ <output-dir compare="Text">drop_dataset_invalid_dataverse</output-dir>
+ <expected-error>ASX1063: Cannot find dataverse with name fakeDataverse (in line 22, at column 1)</expected-error>
+ <expected-error>ASX1063: Cannot find dataverse with name fakeDataverse (in line 22, at column 1)</expected-error>
+ <expected-error>ASX1063: Cannot find dataverse with name fakeDataverse (in line 22, at column 1)</expected-error>
+ <expected-error>ASX1063: Cannot find dataverse with name fakeDataverse (in line 22, at column 1)</expected-error>
+ <expected-error>ASX1050: Cannot find dataset with name fakeDataset1 in dataverse realDataverse (in line 22, at column 1)</expected-error>
+ <expected-warn>ASX1063: Cannot find dataverse with name fakeDataverse (in line 22, at column 1)</expected-warn>
+ <expected-warn>ASX1063: Cannot find dataverse with name fakeDataverse (in line 22, at column 1)</expected-warn>
+ <expected-warn>ASX1063: Cannot find dataverse with name fakeDataverse (in line 22, at column 1)</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="ddl">
+ <compilation-unit name="create_dataset_with_filter_on_meta">
+ <output-dir compare="Text">create_dataset_with_filter_on_meta</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="ddl">
+ <compilation-unit name="index-bad-fields">
+ <output-dir compare="Text">index-bad-fields</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="ddl">
+ <compilation-unit name="index-cast-null">
+ <placeholder name="with" value="" />
+ <output-dir compare="Text">index-cast-null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="ddl">
+ <compilation-unit name="index-cast-null">
+ <placeholder name="with" value="WITH {'merge-policy': {'name': 'correlated-prefix','parameters': { 'max-mergable-component-size': 16384, 'max-tolerance-component-count': 3 }}}" />
+ <output-dir compare="Text">index-cast-null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="ddl">
+ <compilation-unit name="index-cast-null-negative">
+ <output-dir compare="Text">index-cast-null-negative</output-dir>
+ <expected-error>CAST modifier is only allowed for B-Tree indexes</expected-error>
+ <expected-error>CAST modifier cannot be specified together with ENFORCED</expected-error>
+ <expected-error>CAST modifier is used without specifying the type of the indexed field</expected-error>
+ <expected-error>Typed index on 'typed_f2' field could be created only for open datatype</expected-error>
+ <expected-error>Parameter invalid_date cannot be set</expected-error>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="dml">
+ <test-case FilePath="dml">
+ <compilation-unit name="insert-with-autogenerated-pk_adm-with-sec-primary-index">
+ <output-dir compare="Text">insert-with-autogenerated-pk_adm-with-sec-primary-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="insert-returning-fieldname-qualified">
+ <output-dir compare="Text">insert-returning-fieldname</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="compact-dataset-and-its-indexes">
+ <output-dir compare="Text">compact-dataset-and-its-indexes</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="using-constant-merge-policy">
+ <output-dir compare="Text">using-constant-merge-policy</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="using-prefix-merge-policy">
+ <output-dir compare="Text">using-prefix-merge-policy</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="using-concurrent-merge-policy">
+ <output-dir compare="Text">using-concurrent-merge-policy</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="using-correlated-prefix-merge-policy">
+ <output-dir compare="Text">using-correlated-prefix-merge-policy</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="using-correlated-prefix-merge-policy-with-feed">
+ <output-dir compare="Text">using-correlated-prefix-merge-policy</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="using-no-merge-policy">
+ <output-dir compare="Text">using-no-merge-policy</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="query-issue382">
+ <output-dir compare="Text">query-issue382</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="query-issue433">
+ <output-dir compare="Text">query-issue433</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="query-issue288">
+ <output-dir compare="Text">query-issue288</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="query-issue205">
+ <output-dir compare="Text">query-issue205</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="delete-from-loaded-dataset-with-index">
+ <output-dir compare="Text">delete-from-loaded-dataset-with-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="delete-from-loaded-dataset">
+ <output-dir compare="Text">delete-from-loaded-dataset</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="delete-syntax-change">
+ <output-dir compare="Text">delete-syntax-change</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="drop-empty-secondary-indexes">
+ <output-dir compare="Text">drop-empty-secondary-indexes</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="drop-index">
+ <output-dir compare="Text">drop-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="create-drop-cltype">
+ <output-dir compare="Text">create-drop-cltype</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="create-drop-opntype">
+ <output-dir compare="Text">create-drop-opntype</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="empty-load-with-index">
+ <output-dir compare="Text">empty-load-with-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="insert-into-empty-dataset">
+ <output-dir compare="Text">insert-into-empty-dataset</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="insert-into-empty-dataset-with-index">
+ <output-dir compare="Text">insert-into-empty-dataset-with-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="insert-syntax">
+ <output-dir compare="Text">insert-syntax</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="insert-and-scan-dataset">
+ <output-dir compare="Text">insert-and-scan-dataset</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="insert-and-scan-dataset-with-index">
+ <output-dir compare="Text">insert-and-scan-dataset-with-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="recreate-index">
+ <output-dir compare="Text">recreate-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="insert-and-scan-joined-datasets">
+ <output-dir compare="Text">insert-and-scan-joined-datasets</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="insert-into-loaded-dataset-with-index_01">
+ <output-dir compare="Text">insert-into-loaded-dataset-with-index_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="insert-into-loaded-dataset-with-index_02">
+ <output-dir compare="Text">insert-into-loaded-dataset-with-index_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="insert-into-loaded-dataset_01">
+ <output-dir compare="Text">insert-into-loaded-dataset_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="insert-into-loaded-dataset_02">
+ <output-dir compare="Text">insert-into-loaded-dataset_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="insert-src-dst-01">
+ <output-dir compare="Text">insert-src-dst-01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="insert">
+ <output-dir compare="Text">insert</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="insert-duplicated-keys">
+ <output-dir compare="Text">insert-duplicated-keys</output-dir>
+ <expected-error>Inserting duplicate keys into the primary storage</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="insert-duplicated-keys-with-pk-index">
+ <output-dir compare="Text">insert-duplicated-keys-with-pk-index</output-dir>
+ <expected-error>Inserting duplicate keys into the primary storage</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="insert_less_nc">
+ <output-dir compare="Text">insert_less_nc</output-dir>
+ </compilation-unit>
+ </test-case>
+ <!--
+ <test-case FilePath="dml">
+ <compilation-unit name="load-from-hdfs">
+ <output-dir compare="Text">load-from-hdfs</output-dir>
+ </compilation-unit>
+ </test-case>
+ -->
+ <test-case FilePath="dml">
+ <compilation-unit name="insert-with-autogenerated-pk_adm-with-sec-primary-index">
+ <output-dir compare="Text">insert-with-autogenerated-pk_adm-with-sec-primary-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="insert-with-autogenerated-pk_adm_01">
+ <output-dir compare="Text">insert-with-autogenerated-pk_adm_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="insert-with-autogenerated-pk_adm_02">
+ <output-dir compare="Text">insert-with-autogenerated-pk_adm_02</output-dir>
+ <expected-error>Field type string cannot be promoted to type uuid</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="insert-with-autogenerated-pk_adm_03">
+ <output-dir compare="Text">insert-with-autogenerated-pk_adm_03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="load-with-autogenerated-pk_txt_01">
+ <output-dir compare="Text">load-with-autogenerated-pk_txt_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="load-with-autogenerated-pk_adm_01">
+ <output-dir compare="Text">load-with-autogenerated-pk_adm_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="load-with-autogenerated-pk_adm_02">
+ <output-dir compare="Text">load-with-autogenerated-pk_adm_02</output-dir>
+ <expected-error>ASX3058: This record is closed, you can not add extra fields! new field name: id</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="load-with-autogenerated-pk_adm_03">
+ <output-dir compare="Text">load-with-autogenerated-pk_adm_03</output-dir>
+ <expected-error>ASX3058: This record is closed, you can not add extra fields! new field name: id</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="load-with-autogenerated-pk_adm_04">
+ <output-dir compare="Text">load-with-autogenerated-pk_adm_04</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="load-with-autogenerated-pk_csv_01">
+ <output-dir compare="Text">load-with-autogenerated-pk_csv_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="load-with-autogenerated-pk_csv_02">
+ <output-dir compare="Text">load-with-autogenerated-pk_csv_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="load-with-autogenerated-no-field">
+ <output-dir compare="Text">load-with-autogenerated-no-field</output-dir>
+ <expected-error>ASX1014: Field 'not_id' is not found</expected-error>
+ <expected-error>ASX1014: Field 'not_id' is not found</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="insert-return-records">
+ <output-dir compare="Text">insert-return-records</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="insert-returning-udf">
+ <output-dir compare="Text">insert-returning-fieldname</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="insert-returning-fieldname">
+ <output-dir compare="Text">insert-returning-fieldname</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="insert-returning-fieldname-implicit">
+ <output-dir compare="Text">insert-returning-fieldname</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="insert-returning-fieldname-implicit-2">
+ <output-dir compare="Text">insert-returning-fieldname</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="insert_uuid_autogenerate">
+ <output-dir compare="Text">insert_uuid_autogenerate</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="upsert_uuid_autogenerate">
+ <output-dir compare="Text">upsert_uuid_autogenerate</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="insert_uuid_manual">
+ <output-dir compare="Text">insert_uuid_manual</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="upsert_uuid_manual">
+ <output-dir compare="Text">upsert_uuid_manual</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="insert_uuid_manual_exists">
+ <output-dir compare="Text">insert_uuid_manual_exists</output-dir>
+ <expected-error>Inserting duplicate keys into the primary storage</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="upsert_uuid_manual_exists">
+ <output-dir compare="Text">upsert_uuid_manual_exists</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="insert_uuid_manual_exists_select">
+ <output-dir compare="Text">insert_uuid_manual_exists_select</output-dir>
+ <expected-error>Inserting duplicate keys into the primary storage</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="upsert_uuid_manual_exists_select">
+ <output-dir compare="Text">upsert_uuid_manual_exists_select</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="insert_nested_uuid_autogenerate">
+ <output-dir compare="Text">insert_nested_uuid_autogenerate</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="upsert_nested_uuid_autogenerate">
+ <output-dir compare="Text">upsert_nested_uuid_autogenerate</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="insert_nested_uuid_manual">
+ <output-dir compare="Text">insert_nested_uuid_manual</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="upsert_nested_uuid_manual">
+ <output-dir compare="Text">upsert_nested_uuid_manual</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="insert_nested_uuid_manual_exists">
+ <output-dir compare="Text">insert_nested_uuid_manual_exists</output-dir>
+ <expected-error>Inserting duplicate keys into the primary storage</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="upsert_nested_uuid_manual_exists">
+ <output-dir compare="Text">upsert_nested_uuid_manual_exists</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="insert_nested_uuid_manual_exists_select">
+ <output-dir compare="Text">insert_nested_uuid_manual_exists_select</output-dir>
+ <expected-error>Inserting duplicate keys into the primary storage</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="upsert_nested_uuid_manual_exists_select">
+ <output-dir compare="Text">upsert_nested_uuid_manual_exists_select</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="insert-with-bad-return">
+ <output-dir compare="Text">insert-with-bad-return</output-dir>
+ <expected-error>A returning expression cannot contain dataset access</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="load-with-index">
+ <output-dir compare="Text">load-with-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="load-with-ngram-index">
+ <output-dir compare="Text">load-with-ngram-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="load-with-rtree-index">
+ <output-dir compare="Text">load-with-rtree-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="load-with-word-index">
+ <output-dir compare="Text">load-with-word-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="opentype-c2o-recursive">
+ <output-dir compare="Text">opentype-c2o-recursive</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="opentype-c2o">
+ <output-dir compare="Text">opentype-c2o</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="opentype-closed-optional">
+ <output-dir compare="Text">opentype-closed-optional</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="opentype-insert">
+ <output-dir compare="Text">opentype-insert</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="opentype-insert2">
+ <output-dir compare="Text">opentype-insert2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="opentype-noexpand">
+ <output-dir compare="Text">opentype-noexpand</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="opentype-o2c-recursive">
+ <output-dir compare="Text">opentype-o2c-recursive</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="opentype-o2c">
+ <output-dir compare="Text">opentype-o2c</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="opentype-o2o">
+ <output-dir compare="Text">opentype-o2o</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-delete-btree-secondary-index-nullable">
+ <output-dir compare="Text">scan-delete-btree-secondary-index-nullable</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit
+ name="scan-delete-btree-correlated-secondary-index-nullable">
+ <output-dir compare="Text">scan-delete-btree-secondary-index-nullable</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-delete-rtree-secondary-index-nullable">
+ <output-dir compare="Text">scan-delete-rtree-secondary-index-nullable</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-delete-rtree-correlated-secondary-index-nullable">
+ <output-dir compare="Text">scan-delete-rtree-secondary-index-nullable</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-delete-rtree-secondary-index">
+ <output-dir compare="Text">scan-delete-rtree-secondary-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-delete-rtree-correlated-secondary-index">
+ <output-dir compare="Text">scan-delete-rtree-secondary-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-insert-btree-secondary-index-nullable">
+ <output-dir compare="Text">scan-insert-btree-secondary-index-nullable</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-insert-btree-correlated-secondary-index-nullable">
+ <output-dir compare="Text">scan-delete-btree-secondary-index-nullable</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-insert-rtree-secondary-index-nullable">
+ <output-dir compare="Text">scan-insert-rtree-secondary-index-nullable</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-insert-rtree-correlated-secondary-index-nullable">
+ <output-dir compare="Text">scan-insert-rtree-secondary-index-nullable</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-insert-rtree-secondary-index">
+ <output-dir compare="Text">scan-insert-rtree-secondary-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-insert-rtree-correlated-secondary-index">
+ <output-dir compare="Text">scan-insert-rtree-secondary-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-insert-inverted-index-ngram-secondary-index">
+ <output-dir compare="Text">scan-insert-inverted-index-ngram-secondary-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-insert-inverted-index-ngram-secondary-index-string-as-primary-key">
+ <output-dir compare="Text">scan-insert-inverted-index-ngram-secondary-index-string-as-primary-key</output-dir>
+ </compilation-unit>
+ </test-case>
+
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-insert-inverted-index-ngram-correlated-secondary-index">
+ <output-dir compare="Text">scan-insert-inverted-index-ngram-secondary-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-insert-inverted-index-word-secondary-index">
+ <output-dir compare="Text">scan-insert-inverted-index-word-secondary-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-insert-inverted-index-word-secondary-index-string-as-primary-key">
+ <output-dir compare="Text">scan-insert-inverted-index-word-secondary-index-string-as-primary-key</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-insert-inverted-index-word-correlated-secondary-index">
+ <output-dir compare="Text">scan-insert-inverted-index-word-secondary-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-insert-inverted-index-word-correlated-secondary-index-string-as-primary-key">
+ <output-dir compare="Text">scan-insert-inverted-index-word-secondary-index-string-as-primary-key</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-insert-inverted-index-ngram-secondary-index-nullable">
+ <output-dir compare="Text">scan-insert-inverted-index-ngram-secondary-index-nullable</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-insert-inverted-index-ngram-secondary-index-nullable-string-as-primary-key">
+ <output-dir compare="Text">scan-insert-inverted-index-ngram-secondary-index-nullable-string-as-primary-key</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-insert-inverted-index-ngram-correlated-secondary-index-nullable">
+ <output-dir compare="Text">scan-insert-inverted-index-ngram-secondary-index-nullable</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-insert-inverted-index-ngram-correlated-secondary-index-nullable-string-as-primary-key">
+ <output-dir compare="Text">scan-insert-inverted-index-ngram-secondary-index-nullable-string-as-primary-key</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-insert-inverted-index-word-secondary-index-nullable">
+ <output-dir compare="Text">scan-insert-inverted-index-word-secondary-index-nullable</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-insert-inverted-index-word-secondary-index-nullable-string-as-primary-key">
+ <output-dir compare="Text">scan-insert-inverted-index-word-secondary-index-nullable-string-as-primary-key</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-insert-inverted-index-word-correlated-secondary-index-nullable">
+ <output-dir compare="Text">scan-insert-inverted-index-word-secondary-index-nullable</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-delete-inverted-index-ngram-secondary-index">
+ <output-dir compare="Text">scan-delete-inverted-index-ngram-secondary-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-delete-inverted-index-ngram-secondary-index-string-as-primary-key">
+ <output-dir compare="Text">scan-delete-inverted-index-ngram-secondary-index-string-as-primary-key</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-delete-inverted-index-ngram-correlated-secondary-index">
+ <output-dir compare="Text">scan-delete-inverted-index-ngram-secondary-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-delete-inverted-index-word-secondary-index">
+ <output-dir compare="Text">scan-delete-inverted-index-word-secondary-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-delete-inverted-index-word-correlated-secondary-index">
+ <output-dir compare="Text">scan-delete-inverted-index-word-secondary-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-delete-inverted-index-ngram-secondary-index-nullable">
+ <output-dir compare="Text">scan-delete-inverted-index-ngram-secondary-index-nullable</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-delete-inverted-index-ngram-correlated-secondary-index-nullable">
+ <output-dir compare="Text">scan-delete-inverted-index-ngram-secondary-index-nullable</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-delete-inverted-index-word-secondary-index-nullable">
+ <output-dir compare="Text">scan-delete-inverted-index-word-secondary-index-nullable</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-delete-inverted-index-word-correlated-secondary-index-nullable">
+ <output-dir compare="Text">scan-delete-inverted-index-word-secondary-index-nullable</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="load-with-index-open">
+ <output-dir compare="Text">load-with-index-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="load-with-ngram-index-open">
+ <output-dir compare="Text">load-with-ngram-index-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="load-with-rtree-index-open">
+ <output-dir compare="Text">load-with-rtree-index-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="load-with-word-index-open">
+ <output-dir compare="Text">load-with-word-index-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-delete-btree-secondary-index-open">
+ <output-dir compare="Text">scan-delete-btree-secondary-index-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-delete-btree-correlated-secondary-index-open">
+ <output-dir compare="Text">scan-delete-btree-secondary-index-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-delete-inverted-index-ngram-secondary-index-open">
+ <output-dir compare="Text">scan-delete-inverted-index-ngram-secondary-index-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-delete-inverted-index-ngram-correlated-secondary-index-open">
+ <output-dir compare="Text">scan-delete-inverted-index-ngram-secondary-index-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-delete-inverted-index-word-secondary-index-open">
+ <output-dir compare="Text">scan-delete-inverted-index-word-secondary-index-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-delete-inverted-index-word-correlated-secondary-index-open">
+ <output-dir compare="Text">scan-delete-inverted-index-word-secondary-index-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-delete-rtree-secondary-index-open">
+ <output-dir compare="Text">scan-delete-rtree-secondary-index-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-delete-rtree-correlated-secondary-index-open">
+ <output-dir compare="Text">scan-delete-rtree-secondary-index-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-insert-btree-secondary-index-open">
+ <output-dir compare="Text">scan-insert-btree-secondary-index-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-insert-btree-correlated-secondary-index-open">
+ <output-dir compare="Text">scan-delete-btree-secondary-index-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-insert-inverted-index-ngram-secondary-index-open">
+ <output-dir compare="Text">scan-insert-inverted-index-ngram-secondary-index-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-insert-inverted-index-ngram-correlated-secondary-index-open">
+ <output-dir compare="Text">scan-insert-inverted-index-ngram-secondary-index-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-insert-inverted-index-word-secondary-index-open">
+ <output-dir compare="Text">scan-insert-inverted-index-word-secondary-index-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-insert-inverted-index-word-correlated-secondary-index-open">
+ <output-dir compare="Text">scan-insert-inverted-index-word-secondary-index-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-insert-rtree-secondary-index-open">
+ <output-dir compare="Text">scan-insert-rtree-secondary-index-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-insert-rtree-correlated-secondary-index-open">
+ <output-dir compare="Text">scan-insert-rtree-secondary-index-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="delete-multi-statement">
+ <output-dir compare="Text">delete-multi-statement</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="upsert-dataset-with-meta">
+ <output-dir compare="Text">upsert-dataset-with-meta</output-dir>
+ <expected-error>upsert into dataset is not supported on datasets with meta records</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="upsert-return-custom-result">
+ <output-dir compare="Text">upsert-return-custom-result</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="upsert-returning-fieldname">
+ <output-dir compare="Text">insert-returning-fieldname</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="upsert-returning-fieldname-implicit">
+ <output-dir compare="Text">insert-returning-fieldname</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="upsert-returning-fieldname-implicit-2">
+ <output-dir compare="Text">insert-returning-fieldname</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="delete-dataset-with-meta">
+ <output-dir compare="Text">delete-dataset-with-meta</output-dir>
+ <expected-error>delete from dataset is not supported on datasets with meta records</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="insert-dataset-with-meta">
+ <output-dir compare="Text">insert-dataset-with-meta</output-dir>
+ <expected-error>insert into dataset is not supported on datasets with meta records</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="insert-duplicated-keys-from-query">
+ <output-dir compare="Text">insert-duplicated-keys-from-query</output-dir>
+ <expected-error>HYR0033: Inserting duplicate keys into the primary storage</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="load-with-autogenerated-no-field">
+ <output-dir compare="Text">load-with-autogenerated-no-field</output-dir>
+ <expected-error>ASX1014: Field 'not_id' is not found</expected-error>
+ <expected-error>ASX1014: Field 'not_id' is not found</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="load-with-index-open_02">
+ <output-dir compare="Text">load-with-index-open_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="query-ASTERIXDB-867">
+ <output-dir compare="Text">query-ASTERIXDB-867</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="query-ASTERIXDB-1406">
+ <output-dir compare="Text">query-ASTERIXDB-1406</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="query-issue382">
+ <output-dir compare="Text">query-issue382</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="create-index-unknown-key">
+ <output-dir compare="Text">index-unknown-key</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="insert-into-index-unknown-key">
+ <output-dir compare="Text">index-unknown-key</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="upsert-into-index-unknown-key">
+ <output-dir compare="Text">index-unknown-key</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="load-into-index-unknown-key">
+ <output-dir compare="Text">index-unknown-key</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="employee">
+ <test-case FilePath="employee">
+ <compilation-unit name="q_01">
+ <output-dir compare="Text">q_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="employee">
+ <compilation-unit name="q_02">
+ <output-dir compare="Text">q_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="failure">
+ <test-case FilePath="failure">
+ <compilation-unit name="group_by_failure">
+ <output-dir compare="Text">group_by_failure</output-dir>
+ <expected-error>Injected failure in inject-failure</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="failure">
+ <compilation-unit name="group_by_hash_failure">
+ <output-dir compare="Text">group_by_hash_failure</output-dir>
+ <expected-error>Injected failure in inject-failure</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="failure">
+ <compilation-unit name="q01_pricing_summary_report_failure">
+ <output-dir compare="Text">q01_pricing_summary_report_failure</output-dir>
+ <expected-error>Injected failure in inject-failure</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="failure">
+ <compilation-unit name="q18_large_volume_customer_failure">
+ <output-dir compare="Text">q18_large_volume_customer_failure</output-dir>
+ <expected-error>Injected failure in inject-failure</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="failure">
+ <compilation-unit name="order_by_failure">
+ <output-dir compare="Text">order_by_failure</output-dir>
+ <expected-error>Injected failure in inject-failure</expected-error>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="fuzzyjoin">
+ <test-case FilePath="fuzzyjoin">
+ <compilation-unit name="basic-1_1">
+ <output-dir compare="Text">basic-1_1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="fuzzyjoin">
+ <compilation-unit name="basic-1_1_2">
+ <output-dir compare="Text">basic-1_1_2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="fuzzyjoin">
+ <compilation-unit name="basic-1_1_3">
+ <output-dir compare="Text">basic-1_1_3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="fuzzyjoin">
+ <compilation-unit name="dblp-string-as-primary-key">
+ <output-dir compare="Text">dblp-string-as-primary-key</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <!--
+ <test-group name="flwor">
+ <test-case FilePath="flwor">
+ <compilation-unit name="for01">
+ <output-dir compare="Text">for01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="for02">
+ <output-dir compare="Text">for02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="for03">
+ <output-dir compare="Text">for03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="for04">
+ <output-dir compare="Text">for04</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="for05">
+ <output-dir compare="Text">for05</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="for06">
+ <output-dir compare="Text">for06</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="for07">
+ <output-dir compare="Text">for07</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="for08">
+ <output-dir compare="Text">for08</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="for09">
+ <output-dir compare="Text">for09</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="for10">
+ <output-dir compare="Text">for10</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="for11">
+ <output-dir compare="Text">for11</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="for12">
+ <output-dir compare="Text">for12</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="for13">
+ <output-dir compare="Text">for13</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="for14">
+ <output-dir compare="Text">for14</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="for15">
+ <output-dir compare="Text">for15</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="for16">
+ <output-dir compare="Text">for16</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="for17">
+ <output-dir compare="Text">for17</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="for18">
+ <output-dir compare="Text">for18</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="for19">
+ <output-dir compare="Text">for19</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="grpby01">
+ <output-dir compare="Text">grpby01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="grpby02">
+ <output-dir compare="Text">grpby02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="let01">
+ <output-dir compare="Text">let01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="let02">
+ <output-dir compare="Text">let02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="let03">
+ <output-dir compare="Text">let03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="let04">
+ <output-dir compare="Text">let04</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="let05">
+ <output-dir compare="Text">let05</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="let06">
+ <output-dir compare="Text">let06</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="let07">
+ <output-dir compare="Text">let07</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="let08">
+ <output-dir compare="Text">let08</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="let09">
+ <output-dir compare="Text">let09</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="let10">
+ <output-dir compare="Text">let10</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="let11">
+ <output-dir compare="Text">let11</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="let12">
+ <output-dir compare="Text">let12</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="let13">
+ <output-dir compare="Text">let13</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="let14">
+ <output-dir compare="Text">let14</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="let15">
+ <output-dir compare="Text">let15</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="let16">
+ <output-dir compare="Text">let16</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="let17">
+ <output-dir compare="Text">let17</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="let18">
+ <output-dir compare="Text">let18</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="let19">
+ <output-dir compare="Text">let19</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="let20">
+ <output-dir compare="Text">let20</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="let21">
+ <output-dir compare="Text">let21</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="let22">
+ <output-dir compare="Text">let22</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="let23">
+ <output-dir compare="Text">let23</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="let24">
+ <output-dir compare="Text">let24</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="let25">
+ <output-dir compare="Text">let25</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="let26">
+ <output-dir compare="Text">let26</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="let27">
+ <output-dir compare="Text">let27</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="let28">
+ <output-dir compare="Text">let28</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="let29">
+ <output-dir compare="Text">let29</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="let30">
+ <output-dir compare="Text">let30</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="let31">
+ <output-dir compare="Text">let31</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="let32">
+ <output-dir compare="Text">let32</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="order-by-01">
+ <output-dir compare="Text">order-by-01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="order-by-02">
+ <output-dir compare="Text">order-by-02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="order-by-03">
+ <output-dir compare="Text">order-by-03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="order-by-04">
+ <output-dir compare="Text">order-by-04</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="order-by-05">
+ <output-dir compare="Text">order-by-05</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="order-by-06">
+ <output-dir compare="Text">order-by-06</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="order-by-07">
+ <output-dir compare="Text">order-by-07</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="order-by-08">
+ <output-dir compare="Text">order-by-08</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="order-by-09">
+ <output-dir compare="Text">order-by-09</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="order-by-10">
+ <output-dir compare="Text">order-by-10</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="order-by-11">
+ <output-dir compare="Text">order-by-11</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="order-by-12">
+ <output-dir compare="Text">order-by-12</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="ret-01">
+ <output-dir compare="Text">ret-01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="ret-02">
+ <output-dir compare="Text">ret-02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="ret-03">
+ <output-dir compare="Text">ret-03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="ret-04">
+ <output-dir compare="Text">ret-04</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="ret-05">
+ <output-dir compare="Text">ret-05</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="ret-06">
+ <output-dir compare="Text">ret-06</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="ret-07">
+ <output-dir compare="Text">ret-07</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="ret-08">
+ <output-dir compare="Text">ret-08</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="ret-09">
+ <output-dir compare="Text">ret-09</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="ret-10">
+ <output-dir compare="Text">ret-10</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="ret-11">
+ <output-dir compare="Text">ret-11</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="ret-12">
+ <output-dir compare="Text">ret-12</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="ret-13">
+ <output-dir compare="Text">ret-13</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="ret-14">
+ <output-dir compare="Text">ret-14</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="ret-15">
+ <output-dir compare="Text">ret-15</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="writers">
+ <test-case FilePath="writers">
+ <compilation-unit name="print_01">
+ <output-dir compare="Text">print_01</output-dir>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="ret-16">
+ <output-dir compare="Text">ret-16</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="ret-17">
+ <output-dir compare="Text">ret-17</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="ret-18">
+ <output-dir compare="Text">ret-18</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="ret-19">
+ <output-dir compare="Text">ret-19</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ -->
+ <test-group name="fulltext">
+ <test-case FilePath="fulltext">
+ <compilation-unit name="fulltext-01">
+ <output-dir compare="Text">fulltext-01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="fulltext">
+ <compilation-unit name="fulltext-02">
+ <output-dir compare="Text">fulltext-02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="fulltext">
+ <compilation-unit name="fulltext-04">
+ <output-dir compare="Text">fulltext-04</output-dir>
+ <expected-error>ASX1010: Phrase search in Full-text is not yet supported. Only one keyword per expression is permitted</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="fulltext">
+ <compilation-unit name="fulltext-05">
+ <output-dir compare="Text">fulltext-05</output-dir>
+ <expected-error>ASX1010: Phrase search in Full-text is not yet supported. Only one keyword per expression is permitted</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="fulltext">
+ <compilation-unit name="fulltext-06">
+ <output-dir compare="Text">fulltext-06</output-dir>
+ <expected-error>ASX1010: Phrase search in Full-text is not yet supported. Only one keyword per expression is permitted</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="fulltext">
+ <compilation-unit name="fulltext-08">
+ <output-dir compare="Text">fulltext-08</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="fulltext">
+ <compilation-unit name="fulltext-09">
+ <output-dir compare="Text">fulltext-09</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="fulltext">
+ <compilation-unit name="fulltext-index-01">
+ <output-dir compare="Text">fulltext-index-01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="fulltext">
+ <compilation-unit name="fulltext-index-02">
+ <output-dir compare="Text">fulltext-index-02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="fulltext">
+ <compilation-unit name="fulltext-index-04">
+ <output-dir compare="Text">fulltext-index-04</output-dir>
+ <expected-error>ASX1010: Phrase search in Full-text is not yet supported. Only one keyword per expression is permitted</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="fulltext">
+ <compilation-unit name="fulltext-index-05">
+ <output-dir compare="Text">fulltext-index-05</output-dir>
+ <expected-error>ASX1010: Phrase search in Full-text is not yet supported. Only one keyword per expression is permitted</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="fulltext">
+ <compilation-unit name="fulltext-index-06">
+ <output-dir compare="Text">fulltext-index-06</output-dir>
+ <expected-error>ASX1010: Phrase search in Full-text is not yet supported. Only one keyword per expression is permitted</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="fulltext">
+ <compilation-unit name="fulltext-index-08">
+ <output-dir compare="Text">fulltext-index-08</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="fulltext">
+ <compilation-unit name="fulltext-index-09">
+ <output-dir compare="Text">fulltext-index-09</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="fulltext">
+ <compilation-unit name="fulltext-index-large-data">
+ <output-dir compare="Text">fulltext-index-large-data</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="fulltext">
+ <compilation-unit name="stopwords-full-text-filter-1">
+ <output-dir compare="Text">stopwords-full-text-filter-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="global-aggregate">
+ <test-case FilePath="global-aggregate">
+ <compilation-unit name="q01">
+ <output-dir compare="Text">q01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="global-aggregate">
+ <compilation-unit name="q02">
+ <output-dir compare="Text">q02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="global-aggregate">
+ <compilation-unit name="q03">
+ <output-dir compare="Text">q01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="global-aggregate">
+ <compilation-unit name="q04">
+ <output-dir compare="Text">q01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="global-aggregate">
+ <compilation-unit name="q05_error">
+ <output-dir compare="Text">q01</output-dir>
+ <expected-error>ASX1073: Cannot resolve alias reference for undefined identifier u (in line 22, at column 8)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="global-aggregate">
+ <compilation-unit name="q06_error">
+ <output-dir compare="Text">q06</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="global-aggregate">
+ <compilation-unit name="q07_error">
+ <output-dir compare="Text">q01</output-dir>
+ <expected-error>count is a SQL-92 aggregate function. The SQL++ core aggregate function array_count could potentially express the intent.</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="global-aggregate">
+ <compilation-unit name="q08">
+ <output-dir compare="Text">q08</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="global-aggregate">
+ <compilation-unit name="q09">
+ <output-dir compare="Text">q09</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="global-aggregate">
+ <compilation-unit name="q10">
+ <output-dir compare="Text">q09</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="global-aggregate">
+ <compilation-unit name="q11">
+ <output-dir compare="Text">q01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="global-aggregate">
+ <compilation-unit name="q12_error">
+ <output-dir compare="Text">q01</output-dir>
+ <expected-error>The parameter * can only be used in count().</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="global-aggregate">
+ <compilation-unit name="query-ASTERIXDB-159">
+ <output-dir compare="Text">query-ASTERIXDB-159</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="global-aggregate">
+ <compilation-unit name="query-ASTERIXDB-1626">
+ <output-dir compare="Text">query-ASTERIXDB-1626</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="global-aggregate">
+ <compilation-unit name="query-ASTERIXDB-1626-2">
+ <output-dir compare="Text">query-ASTERIXDB-1626-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="global-aggregate">
+ <compilation-unit name="query-ASTERIXDB-2525">
+ <output-dir compare="Text">query-ASTERIXDB-2525</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="group-by">
+ <test-case FilePath="group-by">
+ <compilation-unit name="gby-array">
+ <output-dir compare="Text">gby-array</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="group-by">
+ <compilation-unit name="gby-case-01">
+ <output-dir compare="Text">gby-case-01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="group-by">
+ <compilation-unit name="gby-cross-join">
+ <output-dir compare="Text">gby-cross-join</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="group-by">
+ <compilation-unit name="gby-nested-01">
+ <output-dir compare="Text">gby-nested-01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="group-by">
+ <compilation-unit name="gby-record">
+ <output-dir compare="Text">gby-record</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="group-by">
+ <compilation-unit name="core-01">
+ <output-dir compare="Text">core-01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="group-by">
+ <compilation-unit name="core-01-error">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>Cannot find dataset e in dataverse gby nor an alias with name e</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="group-by">
+ <compilation-unit name="core-02-error">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>Cannot find dataset f in dataverse gby nor an alias with name f</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="group-by">
+ <compilation-unit name="core-02">
+ <output-dir compare="Text">core-02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="group-by">
+ <compilation-unit name="core-03">
+ <output-dir compare="Text">core-03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="group-by">
+ <compilation-unit name="core-04">
+ <output-dir compare="Text">core-04</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="group-by">
+ <compilation-unit name="core-05">
+ <output-dir compare="Text">core-05</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="group-by">
+ <compilation-unit name="core-06">
+ <output-dir compare="Text">core-06</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="group-by">
+ <compilation-unit name="core-07-error">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error><![CDATA[ASX1001: Syntax error: In line 25 >>GROUP BY x, y;<< Encountered "GROUP" at column 1]]></expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="group-by">
+ <compilation-unit name="sugar-01">
+ <output-dir compare="Text">core-01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="group-by">
+ <compilation-unit name="sugar-01-negative">
+ <output-dir compare="Text">sugar-01-negative</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="group-by">
+ <compilation-unit name="sugar-01-2">
+ <output-dir compare="Text">core-01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="group-by">
+ <compilation-unit name="sugar-02">
+ <output-dir compare="Text">core-02</output-dir>
+ <expected-error>Cannot resolve ambiguous alias reference for identifier deptId</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="group-by">
+ <compilation-unit name="sugar-02-2">
+ <output-dir compare="Text">core-02</output-dir>
+ <expected-error>ASX1073: Cannot resolve alias reference for undefined identifier deptId (in line 28, at column 8)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="group-by">
+ <compilation-unit name="sugar-03">
+ <output-dir compare="Text">core-03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="group-by">
+ <compilation-unit name="sugar-03-2">
+ <output-dir compare="Text">core-03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="group-by">
+ <compilation-unit name="sugar-04">
+ <output-dir compare="Text">core-04</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="group-by">
+ <compilation-unit name="sugar-04-2">
+ <output-dir compare="Text">core-04</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="group-by">
+ <compilation-unit name="sugar-05">
+ <output-dir compare="Text">core-05</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="group-by">
+ <compilation-unit name="sugar-05-2">
+ <output-dir compare="Text">core-05</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="group-by">
+ <compilation-unit name="sugar-06-distinct">
+ <output-dir compare="Text">sugar-06-distinct</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="group-by">
+ <compilation-unit name="sugar-07-negative">
+ <output-dir compare="Text">core-01</output-dir>
+ <expected-error>Cannot resolve ambiguous alias reference for identifier IDENT</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="group-by">
+ <compilation-unit name="sugar-08-negative">
+ <output-dir compare="Text">core-01</output-dir>
+ <expected-error>ASX1103: Illegal use of identifier: x</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="group-by">
+ <compilation-unit name="sugar-09">
+ <output-dir compare="Text">sugar-09</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="group-by">
+ <compilation-unit name="two-step-agg-01">
+ <output-dir compare="Text">two-step-agg-01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="group-by">
+ <compilation-unit name="null">
+ <output-dir compare="Text">null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="group-by">
+ <compilation-unit name="gby-expr">
+ <output-dir compare="Text">gby-expr</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="group-by">
+ <compilation-unit name="grouping-sets-1">
+ <output-dir compare="Text">grouping-sets-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="group-by">
+ <compilation-unit name="grouping-sets-2">
+ <output-dir compare="Text">grouping-sets-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="group-by">
+ <compilation-unit name="grouping-sets-3-negative">
+ <output-dir compare="Text">grouping-sets-2</output-dir>
+ <expected-error>ASX1120: Unexpected alias: v21</expected-error>
+ <expected-error>ASX1120: Unexpected alias: v22</expected-error>
+ <expected-error>ASX1120: Unexpected alias: v23</expected-error>
+ <expected-error>ASX1087: Invalid number of arguments for function grouping</expected-error>
+ <expected-error>ASX1119: Invalid argument to grouping() function</expected-error>
+ <expected-error>ASX1119: Invalid argument to grouping() function</expected-error>
+ <expected-error>ASX1119: Invalid argument to grouping() function</expected-error>
+ <expected-error>ASX1118: Too many grouping sets in group by clause: 512. Maximum allowed: 128.</expected-error>
+ <expected-error>ASX1129: Cannot compile SELECT variable.* with GROUP BY GROUPING SETS/ROLLUP/CUBE followed by ORDER BY/LIMIT</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="group-by">
+ <compilation-unit name="having">
+ <output-dir compare="Text">core-02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="group-by">
+ <compilation-unit name="having-2">
+ <output-dir compare="Text">core-02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="group-by">
+ <compilation-unit name="policy">
+ <output-dir compare="Text">policy</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="group-by">
+ <compilation-unit name="policy-02">
+ <output-dir compare="Text">policy-02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="group-by">
+ <compilation-unit name="policy-03">
+ <output-dir compare="Text">policy-03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="group-by">
+ <compilation-unit name="policy-04">
+ <output-dir compare="Text">policy-04</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="group-by" check-warnings="true">
+ <compilation-unit name="policy-05">
+ <output-dir compare="Text">policy-05</output-dir>
+ <expected-warn>Unsupported type: agg-sum cannot process input type object (in line 29, at column 23)</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="group-by">
+ <compilation-unit name="listify">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>The byte size of a single group</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="group-by">
+ <compilation-unit name="listify-2">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>The byte size of a single group</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="group-by">
+ <compilation-unit name="listify-3">
+ <output-dir compare="Text">listify-3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="group-by">
+ <compilation-unit name="redundant-var-in-groupby">
+ <output-dir compare="Text">redundant-var-in-groupby</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="group-by">
+ <compilation-unit name="group-by-all-ASTERIXDB-2611">
+ <output-dir compare="Text">group-by-all-ASTERIXDB-2611</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="group-by">
+ <compilation-unit name="hash-group-by-decor">
+ <output-dir compare="Text">hash-group-by-decor</output-dir>
+ </compilation-unit>
+ </test-case>
+ <!--test-case FilePath="group-by">
+ <compilation-unit name="query-ASTERIXDB-3016">
+ <output-dir compare="Text">query-ASTERIXDB-3016</output-dir>
+ </compilation-unit>
+ </test-case-->
+ </test-group>
+ <test-group name="index-join">
+ <test-case FilePath="index-join">
+ <compilation-unit name="btree-index-nested-loop-join">
+ <output-dir compare="Text">btree-index-nested-loop-join</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-join">
+ <compilation-unit name="btree-primary-equi-join">
+ <output-dir compare="Text">btree-primary-equi-join</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-join">
+ <compilation-unit name="btree-pidx-to-sidx-idxonly-equi-join_01">
+ <output-dir compare="Text">btree-pidx-to-sidx-idxonly-equi-join_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-join">
+ <compilation-unit name="btree-secondary-equi-join_01">
+ <output-dir compare="Text">btree-secondary-equi-join_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-join">
+ <compilation-unit name="btree-secondary-equi-join_02">
+ <output-dir compare="Text">btree-secondary-equi-join_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-join">
+ <compilation-unit name="btree-secondary-equi-join_03">
+ <output-dir compare="Text">btree-secondary-equi-join_03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-join">
+ <compilation-unit name="btree-secondary-equi-join_04">
+ <output-dir compare="Text">btree-secondary-equi-join_04</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-join">
+ <compilation-unit name="btree-sidx-idxonly-to-pidx-equi-join_01">
+ <output-dir compare="Text">btree-sidx-idxonly-to-pidx-equi-join_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-join">
+ <compilation-unit name="btree-sidx-idxonly-to-sidx-idxonly-equi-join_01">
+ <output-dir compare="Text">btree-sidx-idxonly-to-sidx-idxonly-equi-join_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-join">
+ <compilation-unit name="btree-sidx-non-idxonly-to-pidx-equi-join_01">
+ <output-dir compare="Text">btree-sidx-non-idxonly-to-pidx-equi-join_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-join">
+ <compilation-unit name="btree-sidx-non-idxonly-to-sidx-idxonly-equi-join_01">
+ <output-dir compare="Text">btree-sidx-non-idxonly-to-sidx-idxonly-equi-join_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-join">
+ <compilation-unit name="btree-secondary-self-equi-join_01">
+ <output-dir compare="Text">btree-secondary-self-equi-join_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-join" check-warnings="true">
+ <compilation-unit name="hints-indexnl-params">
+ <output-dir compare="Text">hints-indexnl-params</output-dir>
+ <expected-warn><![CDATA[ASX1132: Invalid specification for hint indexnl. ASX1001: Syntax error: In line 1 >>(8, idx_tenk2_1k_2k)<< Encountered <INTEGER_LITERAL> "8" at column 2. (in line 35, at column 21)]]></expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-join">
+ <compilation-unit name="rtree-spatial-intersect-point_01">
+ <output-dir compare="Text">rtree-spatial-intersect-point_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-join">
+ <compilation-unit name="rtree-spatial-intersect-point_02">
+ <output-dir compare="Text">rtree-spatial-intersect-point_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-join">
+ <compilation-unit name="rtree-spatial-intersect-point_03">
+ <output-dir compare="Text">rtree-spatial-intersect-point_03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-join">
+ <compilation-unit name="rtree-spatial-intersect-point_04">
+ <output-dir compare="Text">rtree-spatial-intersect-point_04</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-join">
+ <compilation-unit name="rtree-spatial-intersect-point_05">
+ <output-dir compare="Text">rtree-spatial-intersect-point_05</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-join">
+ <compilation-unit name="rtree-spatial-self-intersect-point">
+ <output-dir compare="Text">rtree-spatial-self-intersect-point</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-join">
+ <compilation-unit name="btree-multiple-join">
+ <output-dir compare="Text">btree-multiple-join</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="index-selection">
+ <test-case FilePath="index-selection">
+ <compilation-unit name="btree-index-composite-key">
+ <output-dir compare="Text">btree-index-composite-key</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="btree-index-composite-key-02">
+ <output-dir compare="Text">btree-index-composite-key-02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="btree-index-composite-key-03">
+ <output-dir compare="Text">btree-index-composite-key-03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="btree-index-composite-key-04">
+ <output-dir compare="Text">btree-index-composite-key-04</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="btree-sec-primary-index">
+ <output-dir compare="Text">btree-sec-primary-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="btree-sec-primary-index-01">
+ <output-dir compare="Text">btree-sec-primary-index-01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="btree-sec-primary-index-02">
+ <output-dir compare="Text">btree-sec-primary-index-02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="btree-sec-primary-index-03">
+ <output-dir compare="Text">btree-sec-primary-index-03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="btree-sec-primary-index-04">
+ <output-dir compare="Text">btree-sec-primary-index-04</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="btree-index-composite-key-mixed-intervals">
+ <output-dir compare="Text">btree-index-composite-key-mixed-intervals</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="btree-index-rewrite-multiple">
+ <output-dir compare="Text">btree-index-rewrite-multiple</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="btree-sidx-composite-idxonly-01">
+ <output-dir compare="Text">btree-sidx-composite-idxonly-01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="btree-sidx-composite-idxonly-02">
+ <output-dir compare="Text">btree-sidx-composite-idxonly-02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="btree-sidx-composite-idxonly-03">
+ <output-dir compare="Text">btree-sidx-composite-idxonly-03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="btree-sidx-composite-idxonly-04">
+ <output-dir compare="Text">btree-sidx-composite-idxonly-04</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="btree-sidx-idxonly-01">
+ <output-dir compare="Text">btree-sidx-idxonly-01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="btree-sidx-non-idxonly-01">
+ <output-dir compare="Text">btree-sidx-non-idxonly-01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="cust-index-age-nullable">
+ <output-dir compare="Text">cust-index-age-nullable</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="intersection-misc-01">
+ <output-dir compare="Text">intersection-misc-01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="intersection-with-between">
+ <output-dir compare="Text">intersection-with-between</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="inverted-index-ngram-contains">
+ <output-dir compare="Text">inverted-index-ngram-contains</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="inverted-index-ngram-edit-distance-panic">
+ <output-dir compare="Text">inverted-index-ngram-edit-distance-panic</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="inverted-index-ngram-edit-distance">
+ <output-dir compare="Text">inverted-index-ngram-edit-distance</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="inverted-index-ngram-edit-distance-large-data">
+ <output-dir compare="Text">inverted-index-ngram-edit-distance-large-data</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="inverted-index-ngram-edit-distance-word-tokens">
+ <output-dir compare="Text">inverted-index-ngram-edit-distance-word-tokens</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="inverted-index-ngram-jaccard">
+ <output-dir compare="Text">inverted-index-ngram-jaccard</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="inverted-index-ngram-edit-distance-contains">
+ <output-dir compare="Text">inverted-index-ngram-edit-distance-contains</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="inverted-index-olist-edit-distance-panic">
+ <output-dir compare="Text">inverted-index-olist-edit-distance-panic</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="inverted-index-olist-edit-distance">
+ <output-dir compare="Text">inverted-index-olist-edit-distance</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="inverted-index-olist-jaccard">
+ <output-dir compare="Text">inverted-index-olist-jaccard</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="inverted-index-ulist-jaccard">
+ <output-dir compare="Text">inverted-index-ulist-jaccard</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="inverted-index-word-contains">
+ <output-dir compare="Text">inverted-index-word-contains</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="inverted-index-word-jaccard">
+ <output-dir compare="Text">inverted-index-word-jaccard</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="orders-index-custkey-conjunctive-open">
+ <output-dir compare="Text">orders-index-custkey-conjunctive-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="orders-index-custkey-conjunctive">
+ <output-dir compare="Text">orders-index-custkey-conjunctive</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="orders-index-custkey-open">
+ <output-dir compare="Text">orders-index-custkey-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="orders-index-custkey">
+ <output-dir compare="Text">orders-index-custkey</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="range-search-open">
+ <output-dir compare="Text">range-search-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="range-search">
+ <output-dir compare="Text">range-search</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="rtree-secondary-index-nullable">
+ <output-dir compare="Text">rtree-secondary-index-nullable</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="rtree-secondary-index-open">
+ <output-dir compare="Text">rtree-secondary-index-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="rtree-secondary-index">
+ <output-dir compare="Text">rtree-secondary-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="rtree-secondary-index-circular-query">
+ <output-dir compare="Text">rtree-secondary-index-circular-query</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="rtree-sidx-idxonly-01">
+ <output-dir compare="Text">rtree-sidx-idxonly-01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="rtree-sidx-non-idxonly-01">
+ <output-dir compare="Text">rtree-sidx-non-idxonly-01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="rtree-sidx-non-idxonly-02">
+ <output-dir compare="Text">rtree-sidx-non-idxonly-02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="disjunctive-predicate-1">
+ <output-dir compare="Text">disjunctive-predicate-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="dataset-with-meta">
+ <output-dir compare="Text">dataset-with-meta</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection" check-warnings="true">
+ <compilation-unit name="hints-skip-index">
+ <output-dir compare="Text">hints-skip-index</output-dir>
+ <expected-warn><![CDATA[ASX1132: Invalid specification for hint skip-index. ASX1001: Syntax error: In line 1 >>(13, idx_1k)<< Encountered <INTEGER_LITERAL> "13" at column 2. (in line 32, at column 19)]]></expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection" check-warnings="true">
+ <compilation-unit name="hints-use-index">
+ <output-dir compare="Text">hints-use-index</output-dir>
+ <expected-warn><![CDATA[ASX1132: Invalid specification for hint use-index. ASX1001: Syntax error: In line 1 >>(18, idx_1k_2k)<< Encountered <INTEGER_LITERAL> "18" at column 2. (in line 33, at column 15)]]></expected-warn>
+ <expected-warn><![CDATA[ASX1132: Invalid specification for hint use-index. ASX1001: Syntax error: In line 1 >>()<< Encountered ")" at column 2. (in line 33, at column 15)]]></expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="inverted-index-ngram-edit-distance-with-two-ngram-index">
+ <output-dir compare="Text">inverted-index-ngram-edit-distance-with-two-ngram-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="intersection">
+ <output-dir compare="Text">intersection</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="intersection-with-filter">
+ <output-dir compare="Text">intersection-with-filter</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="intersection_with_nodegroup">
+ <output-dir compare="Text">intersection</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="verify">
+ <output-dir compare="Text">verify</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="inverted-index-join">
+ <test-case FilePath="inverted-index-join">
+ <compilation-unit name="ngram-edit-distance">
+ <output-dir compare="Text">ngram-edit-distance</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="inverted-index-join">
+ <compilation-unit name="ngram-edit-distance-inline">
+ <output-dir compare="Text">ngram-edit-distance-inline</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="inverted-index-join">
+ <compilation-unit name="ngram-jaccard">
+ <output-dir compare="Text">ngram-jaccard</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="inverted-index-join">
+ <compilation-unit name="ngram-jaccard-inline">
+ <output-dir compare="Text">ngram-jaccard-inline</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="inverted-index-join">
+ <compilation-unit name="olist-edit-distance">
+ <output-dir compare="Text">olist-edit-distance</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="inverted-index-join">
+ <compilation-unit name="olist-edit-distance-inline">
+ <output-dir compare="Text">olist-edit-distance-inline</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="inverted-index-join">
+ <compilation-unit name="olist-jaccard">
+ <output-dir compare="Text">olist-jaccard</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="inverted-index-join">
+ <compilation-unit name="olist-jaccard-inline">
+ <output-dir compare="Text">olist-jaccard-inline</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="inverted-index-join">
+ <compilation-unit name="ulist-jaccard">
+ <output-dir compare="Text">ulist-jaccard</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="inverted-index-join">
+ <compilation-unit name="ulist-jaccard-inline">
+ <output-dir compare="Text">ulist-jaccard-inline</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="inverted-index-join">
+ <compilation-unit name="word-jaccard">
+ <output-dir compare="Text">word-jaccard</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="inverted-index-join">
+ <compilation-unit name="word-jaccard-inline">
+ <output-dir compare="Text">word-jaccard-inline</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="cast-default-null">
+ <placeholder name="with" value="" />
+ <output-dir compare="Text">cast-default-null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="cast-default-null">
+ <placeholder name="with" value="WITH {'merge-policy': {'name': 'correlated-prefix','parameters': { 'max-mergable-component-size': 16384, 'max-tolerance-component-count': 3 }}}" />
+ <output-dir compare="Text">cast-default-null</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="inverted-index-join-noeqjoin">
+ <test-case FilePath="inverted-index-join-noeqjoin">
+ <compilation-unit name="ngram-edit-distance">
+ <output-dir compare="Text">ngram-edit-distance</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="inverted-index-join-noeqjoin">
+ <compilation-unit name="ngram-edit-distance-inline">
+ <output-dir compare="Text">ngram-edit-distance-inline</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="inverted-index-join-noeqjoin">
+ <compilation-unit name="ngram-jaccard">
+ <output-dir compare="Text">ngram-jaccard</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="inverted-index-join-noeqjoin">
+ <compilation-unit name="ngram-jaccard-inline">
+ <output-dir compare="Text">ngram-jaccard-inline</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="inverted-index-join-noeqjoin">
+ <compilation-unit name="olist-edit-distance">
+ <output-dir compare="Text">olist-edit-distance</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="inverted-index-join-noeqjoin">
+ <compilation-unit name="olist-edit-distance-inline">
+ <output-dir compare="Text">olist-edit-distance-inline</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="inverted-index-join-noeqjoin">
+ <compilation-unit name="olist-jaccard">
+ <output-dir compare="Text">olist-jaccard</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="inverted-index-join-noeqjoin">
+ <compilation-unit name="olist-jaccard-inline">
+ <output-dir compare="Text">olist-jaccard-inline</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="inverted-index-join-noeqjoin">
+ <compilation-unit name="ulist-jaccard">
+ <output-dir compare="Text">ulist-jaccard</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="inverted-index-join-noeqjoin">
+ <compilation-unit name="ulist-jaccard-inline">
+ <output-dir compare="Text">ulist-jaccard-inline</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="inverted-index-join-noeqjoin">
+ <compilation-unit name="word-jaccard">
+ <output-dir compare="Text">word-jaccard</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="inverted-index-join-noeqjoin">
+ <compilation-unit name="word-jaccard-inline">
+ <output-dir compare="Text">word-jaccard-inline</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="join">
+ <test-case FilePath="join">
+ <compilation-unit name="cross-join-01">
+ <output-dir compare="Text">cross-join-01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="join">
+ <compilation-unit name="cross-join-02-negative">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>ASX1077: Cannot find dataset x in dataverse Default nor an alias with name x (in line 26, at column 39)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="join">
+ <compilation-unit name="hash-join-with-redundant-variable">
+ <output-dir compare="Text">hash-join-with-redundant-variable</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="join">
+ <compilation-unit name="ASTERIXDB-3502">
+ <output-dir compare="Text">ASTERIXDB-3502</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="join">
+ <compilation-unit name="hash_join_array">
+ <output-dir compare="Text">hash_join_array</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="join">
+ <compilation-unit name="hash_join_missing">
+ <output-dir compare="Text">hash_join_missing</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="join">
+ <compilation-unit name="hash_join_record">
+ <output-dir compare="Text">hash_join_record</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="join">
+ <compilation-unit name="join-with-empty-dataset">
+ <output-dir compare="Text">join-with-empty-dataset</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="join">
+ <compilation-unit name="nlj_partitioning_property">
+ <output-dir compare="Text">nlj_partitioning_property</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="join">
+ <compilation-unit name="inner_right_corr">
+ <output-dir compare="Text">inner_right_corr</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="list">
+ <test-case FilePath="list">
+ <compilation-unit name="any-collection-member_01">
+ <output-dir compare="Text">any-collection-member_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="list">
+ <compilation-unit name="array_length">
+ <output-dir compare="Text">array_length</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="list">
+ <compilation-unit name="enforcing_item_type">
+ <output-dir compare="Text">enforcing_item_type</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="list">
+ <compilation-unit name="exists">
+ <output-dir compare="Text">exists</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="list">
+ <compilation-unit name="get-item_01">
+ <output-dir compare="Text">get-item_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="list">
+ <compilation-unit name="get-item_02">
+ <output-dir compare="Text">get-item_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="list">
+ <compilation-unit name="get-item_03">
+ <output-dir compare="Text">get-item_03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="list">
+ <compilation-unit name="len_01">
+ <output-dir compare="Text">len_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="list">
+ <compilation-unit name="len_null_01">
+ <output-dir compare="Text">len_null_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="list">
+ <compilation-unit name="list-range">
+ <output-dir compare="Text">list-range</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="list">
+ <compilation-unit name="list-slice_01">
+ <output-dir compare="Text">list-slice_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="list">
+ <compilation-unit name="list-slice_02">
+ <output-dir compare="Text">list-slice_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="list">
+ <compilation-unit name="list-slice_03">
+ <output-dir compare="Text">list-slice_03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="list">
+ <compilation-unit name="list-star_01">
+ <output-dir compare="Text">list-star_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="list">
+ <compilation-unit name="listify_01">
+ <output-dir compare="Text">listify_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="list">
+ <compilation-unit name="listify_02">
+ <output-dir compare="Text">listify_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="list">
+ <compilation-unit name="listify_03">
+ <output-dir compare="Text">listify_03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="list">
+ <compilation-unit name="ordered-list-constructor_01">
+ <output-dir compare="Text">ordered-list-constructor_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="list">
+ <compilation-unit name="ordered-list-constructor_02">
+ <output-dir compare="Text">ordered-list-constructor_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="list">
+ <compilation-unit name="ordered-list-constructor_03">
+ <output-dir compare="Text">ordered-list-constructor_03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="list">
+ <compilation-unit name="ordered-list-constructor_04">
+ <output-dir compare="Text">ordered-list-constructor_04</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="list">
+ <compilation-unit name="ordered-list-constructor_05">
+ <output-dir compare="Text">ordered-list-constructor_05</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="list">
+ <compilation-unit name="ordered-list-constructor_06_negative">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error><![CDATA[ASX1001: Syntax error: In line 26 >> from [r, r+1] x select value count(x),<< Encountered "," at column 42]]></expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="list">
+ <compilation-unit name="scan-collection_01">
+ <output-dir compare="Text">scan-collection_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <!-- <test-case FilePath="list">
+ <compilation-unit name="union_01">
+ <output-dir compare="Text">union_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="list">
+ <compilation-unit name="union_02">
+ <output-dir compare="Text">union_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ -->
+ <test-case FilePath="list">
+ <compilation-unit name="unordered-list-constructor_01">
+ <output-dir compare="Text">unordered-list-constructor_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="list">
+ <compilation-unit name="unordered-list-constructor_02">
+ <output-dir compare="Text">unordered-list-constructor_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="list">
+ <compilation-unit name="unordered-list-constructor_03">
+ <output-dir compare="Text">unordered-list-constructor_03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="list">
+ <compilation-unit name="query-issue428">
+ <output-dir compare="Text">query-issue428</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="list">
+ <compilation-unit name="query-ASTERIXDB-159-2">
+ <output-dir compare="Text">query-ASTERIXDB-159-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="list">
+ <compilation-unit name="query-ASTERIXDB-159-3">
+ <output-dir compare="Text">query-ASTERIXDB-159-3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="list">
+ <compilation-unit name="query-ASTERIXDB-1131-2">
+ <output-dir compare="Text">query-ASTERIXDB-1131-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="list">
+ <compilation-unit name="query-ASTERIXDB-1131">
+ <output-dir compare="Text">query-ASTERIXDB-1131</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="list">
+ <compilation-unit name="query-ASTERIXDB-1212-2-open">
+ <output-dir compare="Text">query-ASTERIXDB-1212-2-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="list">
+ <compilation-unit name="query-ASTERIXDB-1212-2">
+ <output-dir compare="Text">query-ASTERIXDB-1212-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="list">
+ <compilation-unit name="query-ASTERIXDB-1212-open">
+ <output-dir compare="Text">query-ASTERIXDB-1212-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="list">
+ <compilation-unit name="query-ASTERIXDB-1212">
+ <output-dir compare="Text">query-ASTERIXDB-1212</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="list">
+ <compilation-unit name="query-ASTERIXDB-673">
+ <output-dir compare="Text">query-ASTERIXDB-673</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="list">
+ <compilation-unit name="var-in-list">
+ <output-dir compare="Text">var-in-list</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="misc">
+ <test-case FilePath="misc">
+ <compilation-unit name="big_in_list/000">
+ <output-dir compare="Text">big_in_list/000</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="big_in_list/001">
+ <output-dir compare="Text">big_in_list/001</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="multiple_and/000">
+ <output-dir compare="Text">multiple_and/000</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="record-serialization-ASTERIXDB-2567">
+ <output-dir compare="Text">record-serialization-ASTERIXDB-2567</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="record-serialization-ASTERIXDB-2613">
+ <output-dir compare="Text">record-serialization-ASTERIXDB-2613</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="field_access-ASTERIXDB-2289">
+ <output-dir compare="Text">field_access-ASTERIXDB-2289</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="comp-ASTERIXDB-2415">
+ <output-dir compare="Text">query-ASTERIXDB-1671</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="comp-ASTERIXDB-2412">
+ <output-dir compare="Text">comp-ASTERIXDB-2412</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="field_access_union-ASTERIXDB-2288">
+ <output-dir compare="Text">field_access_union-ASTERIXDB-2288</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="constant_folding">
+ <output-dir compare="Text">constant_folding</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="join-ASTERIXDB-2686">
+ <output-dir compare="Text">join-ASTERIXDB-2686</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="poll-dynamic">
+ <output-dir compare="Text">poll-dynamic</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="validate-expected">
+ <output-dir compare="Text">validate-expected</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="dataset-resources">
+ <output-dir compare="Text">dataset-resources</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="ping">
+ <output-dir compare="Text">ping</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="case_01">
+ <output-dir compare="Text">case_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="case_02">
+ <output-dir compare="Text">case_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="case_03">
+ <output-dir compare="Text">case_03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="case_04">
+ <output-dir compare="Text">case_04</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="case_05">
+ <output-dir compare="Text">case_05</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="case_06">
+ <output-dir compare="Text">case_06</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="case_07">
+ <output-dir compare="Text">case_05</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="case_08">
+ <output-dir compare="Text">case_08</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="case_09">
+ <output-dir compare="Text">case_09</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="dataset_nodegroup">
+ <output-dir compare="Text">dataset_nodegroup</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="ensure_result_numeric_type">
+ <output-dir compare="Text">ensure_result_numeric_type</output-dir>
+ <expected-error>expected < 3.0</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="partition-by-nonexistent-field">
+ <output-dir compare="Text">partition-by-nonexistent-field</output-dir>
+ <expected-error>Field 'id' is not found</expected-error>
+ <expected-error>Cannot find dataset with name testds in dataverse test</expected-error>
+ <expected-error>Cannot find dataset testds in dataverse test nor an alias with name testds</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="float_01">
+ <output-dir compare="Text">float_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="flushtest">
+ <output-dir compare="Text">flushtest</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="groupby-orderby-count">
+ <output-dir compare="Text">groupby-orderby-count</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="identifier_01">
+ <output-dir compare="Text">identifier_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="ifthenelse_01">
+ <output-dir compare="Text">ifthenelse_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="is-null_01">
+ <output-dir compare="Text">is-null_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="nested-loop-join_01">
+ <output-dir compare="Text">nested-loop-join_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="query_issue267">
+ <output-dir compare="Text">query_issue267</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="random">
+ <output-dir compare="Text">random</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="stable_sort">
+ <output-dir compare="Text">stable_sort</output-dir>
+ </compilation-unit>
+ </test-case>
+ <!--
+ <test-case FilePath="misc">
+ <compilation-unit name="range_01">
+ <output-dir compare="Text">range_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ -->
+ <!--
+ <test-case FilePath="misc">
+ <compilation-unit name="tid_01">
+ <output-dir compare="Text">tid_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ -->
+ <test-case FilePath="misc">
+ <compilation-unit name="year_01">
+ <output-dir compare="Text">year_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="string_eq_01">
+ <output-dir compare="Text">string_eq_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="prefix-search">
+ <output-dir compare="Text">prefix-search</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="query-ASTERIXDB-1490">
+ <output-dir compare="Text">query-ASTERIXDB-1490</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="query-ASTERIXDB-971">
+ <output-dir compare="Text">query-ASTERIXDB-971-sqlpp</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="query-ASTERIXDB-1531">
+ <output-dir compare="Text">query-ASTERIXDB-1531</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="query-ASTERIXDB-1577">
+ <output-dir compare="Text">query-ASTERIXDB-1577</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="query-ASTERIXDB-1671">
+ <output-dir compare="Text">query-ASTERIXDB-1671</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="query-ASTERIXDB-1671-2">
+ <output-dir compare="Text">query-ASTERIXDB-1671</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="query-ASTERIXDB-2354">
+ <output-dir compare="Text">query-ASTERIXDB-2354</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="query-ASTERIXDB-2355">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error><![CDATA[ASX1001: Syntax error: In line 22 >> %%%<< Encountered "%" at column 2.]]></expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="query-ASTERIXDB-2380">
+ <output-dir compare="Text">query-ASTERIXDB-2380</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="query-ASTERIXDB-2550">
+ <output-dir compare="Text">query-ASTERIXDB-2550</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="query-ASTERIXDB-2886">
+ <output-dir compare="Text">query-ASTERIXDB-2886</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="unsupported_parameter">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>Query parameter compiler.joinmem is not supported</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="unsupported_parameter_value">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>The given string: 4LS is not a byte unit string (e.g., 320KB or 1024)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="uuid">
+ <output-dir compare="Text">uuid</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="p_sort_join">
+ <output-dir compare="Text">p_sort_join</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="p_sort_seq_merge">
+ <output-dir compare="Text">p_sort_seq_merge</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="p_sort_num_samples">
+ <output-dir compare="Text">p_sort_num_samples</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="p_sort_static_range_map">
+ <output-dir compare="Text">p_sort_static_range_map</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="active_requests">
+ <output-dir compare="Text">active_requests</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="jobs">
+ <output-dir compare="Text">jobs</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="completed_requests">
+ <output-dir compare="Text">completed_requests</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="dump_index">
+ <output-dir compare="Text">dump_index</output-dir>
+ <expected-error>Cannot find index with name noindex</expected-error>
+ <expected-error>Cannot find dataset with name nodataset in dataverse test</expected-error>
+ <expected-error>Cannot find dataset with name ds in dataverse nodataverse</expected-error>
+ <expected-error>Unsupported type: dump-index cannot process input type null</expected-error>
+ <expected-error>Unsupported type: dump-index cannot process input type null</expected-error>
+ <expected-error>Unsupported type: dump-index cannot process input type null</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="metadata_only_01">
+ <output-dir compare="Text">metadata_only_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="metadata_only_02">
+ <output-dir compare="Text">metadata_only_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="cast-ASTERIXDB-2458">
+ <output-dir compare="Text">cast-ASTERIXDB-2458</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="insert_nulls_with_secondary_idx">
+ <output-dir compare="Text">insert_nulls_with_secondary_idx</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="query-ASTERIXDB-2700">
+ <output-dir compare="Text">query-ASTERIXDB-2700</output-dir>
+ </compilation-unit>
+ </test-case>
+ <!--
+ <test-case FilePath="misc">
+ <compilation-unit name="query-ASTERIXDB-1203">
+ <output-dir compare="Text">query-ASTERIXDB-1203</output-dir>
+ </compilation-unit>
+ </test-case>
+ -->
+ <test-case FilePath="misc">
+ <compilation-unit name="query-ASTERIXDB-819-2">
+ <output-dir compare="Text">query-ASTERIXDB-819-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="query-ASTERIXDB-819">
+ <output-dir compare="Text">query-ASTERIXDB-819</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="query-ASTERIXDB-865">
+ <output-dir compare="Text">query-ASTERIXDB-865</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="serialized_size_fun">
+ <output-dir compare="Text">serialized_size_fun</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="query-ASTERIXDB-3334">
+ <output-dir compare="Text">query-ASTERIXDB-3334</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="query-ASTERIXDB-3418">
+ <output-dir compare="Text">query-ASTERIXDB-3418</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="query-ASTERIXDB-3403">
+ <output-dir compare="Text">query-ASTERIXDB-3403</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="query-ASTERIXDB-3490">
+ <output-dir compare="Text">query-ASTERIXDB-3490</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="query-ASTERIXDB-3316">
+ <output-dir compare="Text">query-ASTERIXDB-3316</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="query-ASTERIXDB-3512">
+ <output-dir compare="Text">query-ASTERIXDB-3512</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
+ <compilation-unit name="remove_listify">
+ <output-dir compare="Text">remove_listify</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="multipart-dataverse">
+ <test-case FilePath="multipart-dataverse">
+ <compilation-unit name="index_1">
+ <output-dir compare="Text">index_1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="multipart-dataverse">
+ <compilation-unit name="resolution_1">
+ <output-dir compare="Text">resolution_1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="multipart-dataverse">
+ <compilation-unit name="special_chars_1">
+ <output-dir compare="Text">special_chars_1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="multipart-dataverse">
+ <compilation-unit name="special_chars_2">
+ <output-dir compare="Text">special_chars_2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="multipart-dataverse">
+ <compilation-unit name="udf_1">
+ <output-dir compare="Text">udf_1</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="index">
+ <test-group name="index/validations">
+ <test-case FilePath="index/validations">
+ <compilation-unit name="keys-same-as-pk-but-different-order">
+ <output-dir compare="Text">keys-same-as-pk-but-different-order</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index/validations">
+ <compilation-unit name="keys-same-as-pk-in-same-order">
+ <output-dir compare="Text">keys-same-as-pk-in-same-order</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index/validations">
+ <compilation-unit name="repetitive-keys">
+ <output-dir compare="Text">repetitive-keys</output-dir>
+ <expected-error>Cannot create index with the same field '[value]' specified more than once.</expected-error>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ </test-group>
+ <test-group name="open-index-enforced">
+ <test-group name="open-index-enforced/error-checking">
+ <test-case FilePath="open-index-enforced/error-checking">
+ <compilation-unit name="enforced-field-name-collision">
+ <output-dir compare="Text">enforced-field-name-collision</output-dir>
+ <expected-error>Cannot create enforced index on '[value]' field. The field is closed type.</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-enforced/error-checking">
+ <compilation-unit name="enforced-field-type-collision">
+ <output-dir compare="Text">enforced-field-type-collision</output-dir>
+ <expected-error>Cannot create enforced index on '[value]' field. The field is closed type.</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-enforced/error-checking">
+ <compilation-unit name="missing-enforce-statement">
+ <output-dir compare="Text">missing-enforce-statement</output-dir>
+ <expected-error>ASX1042: Cannot create non-enforced typed index of this kind: RTREE</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-enforced/error-checking">
+ <compilation-unit name="missing-optionality">
+ <output-dir compare="Text">missing-optionality</output-dir>
+ <expected-error>Cannot create enforced index on '[value]' field with non-optional type</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-enforced/error-checking">
+ <compilation-unit name="index-on-closed-type">
+ <output-dir compare="Text">index-on-closed-type</output-dir>
+ <expected-error>ASX1014: Field 'value' is not found (in line 33, at column 34)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-enforced/error-checking">
+ <compilation-unit name="index-type-collision">
+ <output-dir compare="Text">index-type-collision</output-dir>
+ <expected-error>Cannot create index testIdx2 , enforced index testIdx1 on field(s) 'value' is already defined with type(s) 'integer'</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-enforced/error-checking">
+ <compilation-unit name="index-type-promotion-collision">
+ <output-dir compare="Text">index-type-promotion-collision</output-dir>
+ <expected-error>Cannot create index testIdx2 , enforced index testIdx1 on field(s) 'value' is already defined with type(s) 'bigint'</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-enforced/error-checking">
+ <compilation-unit name="object-type-collision">
+ <output-dir compare="Text">object-type-collision</output-dir>
+ <expected-error>ASX1051: Cannot create enforced index on '[value]' field. The field is closed type.</expected-error>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="open-index-enforced/index-join">
+ <test-case FilePath="open-index-enforced/index-join">
+ <compilation-unit name="btree-secondary-equi-join">
+ <output-dir compare="Text">btree-secondary-equi-join</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-enforced/index-join">
+ <compilation-unit name="ngram-edit-distance">
+ <output-dir compare="Text">ngram-edit-distance</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-enforced/index-join">
+ <compilation-unit name="ngram-edit-distance-inline">
+ <output-dir compare="Text">ngram-edit-distance-inline</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-enforced/index-join">
+ <compilation-unit name="ngram-jaccard">
+ <output-dir compare="Text">ngram-jaccard</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-enforced/index-join">
+ <compilation-unit name="ngram-jaccard-inline">
+ <output-dir compare="Text">ngram-jaccard-inline</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-enforced/index-join">
+ <compilation-unit name="rtree-spatial-intersect-point">
+ <output-dir compare="Text">rtree-spatial-intersect-point</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-enforced/index-join">
+ <compilation-unit name="word-jaccard">
+ <output-dir compare="Text">word-jaccard</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-enforced/index-join">
+ <compilation-unit name="word-jaccard-inline">
+ <output-dir compare="Text">word-jaccard-inline</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="open-index-enforced/index-leftouterjoin">
+ <test-case FilePath="open-index-enforced/index-leftouterjoin">
+ <compilation-unit name="probe-pidx-with-join-btree-sidx1">
+ <output-dir compare="Text">probe-pidx-with-join-btree-sidx1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-enforced/index-leftouterjoin">
+ <compilation-unit name="probe-pidx-with-join-btree-sidx2">
+ <output-dir compare="Text">probe-pidx-with-join-btree-sidx2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-enforced/index-leftouterjoin">
+ <compilation-unit name="probe-pidx-with-join-invidx-sidx2">
+ <output-dir compare="Text">probe-pidx-with-join-invidx-sidx2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-enforced/index-leftouterjoin">
+ <compilation-unit name="probe-pidx-with-join-rtree-sidx1">
+ <output-dir compare="Text">probe-pidx-with-join-rtree-sidx1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-enforced/index-leftouterjoin">
+ <compilation-unit name="probe-pidx-with-join-rtree-sidx2">
+ <output-dir compare="Text">probe-pidx-with-join-rtree-sidx2</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="open-index-enforced/index-selection">
+ <test-case FilePath="open-index-enforced/index-selection">
+ <compilation-unit name="btree-index-composite-key">
+ <output-dir compare="Text">btree-index-composite-key</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-enforced/index-selection">
+ <compilation-unit name="btree-index-composite-key-mixed-intervals">
+ <output-dir compare="Text">btree-index-composite-key-mixed-intervals</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-enforced/index-selection">
+ <compilation-unit name="btree-index-rewrite-multiple">
+ <output-dir compare="Text">btree-index-rewrite-multiple</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-enforced/index-selection">
+ <compilation-unit name="inverted-index-ngram-contains">
+ <output-dir compare="Text">inverted-index-ngram-contains</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-enforced/index-selection">
+ <compilation-unit name="inverted-index-ngram-edit-distance">
+ <output-dir compare="Text">inverted-index-ngram-edit-distance</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-enforced/index-selection">
+ <compilation-unit name="inverted-index-ngram-edit-distance-contains">
+ <output-dir compare="Text">inverted-index-ngram-edit-distance-contains</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-enforced/index-selection">
+ <compilation-unit name="inverted-index-ngram-edit-distance-panic">
+ <output-dir compare="Text">inverted-index-ngram-edit-distance-panic</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-enforced/index-selection">
+ <compilation-unit name="inverted-index-ngram-edit-distance-word-tokens">
+ <output-dir compare="Text">inverted-index-ngram-edit-distance-word-tokens</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-enforced/index-selection">
+ <compilation-unit name="inverted-index-ngram-jaccard">
+ <output-dir compare="Text">inverted-index-ngram-jaccard</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-enforced/index-selection">
+ <compilation-unit name="inverted-index-word-contains">
+ <output-dir compare="Text">inverted-index-word-contains</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-enforced/index-selection">
+ <compilation-unit name="inverted-index-word-jaccard">
+ <output-dir compare="Text">inverted-index-word-jaccard</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-enforced/index-selection">
+ <compilation-unit name="orders-index-custkey">
+ <output-dir compare="Text">orders-index-custkey</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-enforced/index-selection">
+ <compilation-unit name="orders-index-custkey-conjunctive">
+ <output-dir compare="Text">orders-index-custkey-conjunctive</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-enforced/index-selection">
+ <compilation-unit name="range-search">
+ <output-dir compare="Text">range-search</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-enforced/index-selection">
+ <compilation-unit name="rtree-secondary-index">
+ <output-dir compare="Text">rtree-secondary-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-enforced/index-selection">
+ <compilation-unit name="multi-index-composite-key">
+ <output-dir compare="Text">multi-index-composite-key</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-enforced/index-selection">
+ <compilation-unit name="multi-index">
+ <output-dir compare="Text">multi-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="open-index-enforced/type-checking">
+ <test-case FilePath="open-index-enforced/type-checking">
+ <compilation-unit name="enforced-type-delete">
+ <output-dir compare="Text">enforced-type-delete</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-enforced/type-checking">
+ <compilation-unit name="enforced-type-upsert">
+ <output-dir compare="Text">enforced-type-upsert</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="open-index-non-enforced/index-selection">
+ <test-case FilePath="open-index-non-enforced/index-selection">
+ <compilation-unit name="btree-index-01">
+ <output-dir compare="Text">btree-index-01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-non-enforced/index-selection">
+ <compilation-unit name="btree-index-02">
+ <output-dir compare="Text">btree-index-02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-non-enforced/index-selection">
+ <compilation-unit name="btree-index-03">
+ <output-dir compare="Text">btree-index-03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-non-enforced/index-selection">
+ <compilation-unit name="btree-index-04">
+ <output-dir compare="Text">btree-index-04</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-non-enforced/index-selection">
+ <compilation-unit name="btree-index-composite-key-03">
+ <output-dir compare="Text">btree-index-composite-key-03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-non-enforced/index-selection">
+ <compilation-unit name="btree-index-composite-key-04">
+ <output-dir compare="Text">btree-index-composite-key-04</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-non-enforced/correlated-index-selection">
+ <compilation-unit name="btree-index-01">
+ <output-dir compare="Text">btree-index-01</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="open-index-non-enforced/index-join">
+ <test-case FilePath="open-index-non-enforced/index-join">
+ <compilation-unit name="btree-equi-join-01">
+ <output-dir compare="Text">btree-equi-join-01</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ </test-group>
+ <test-group name="nested-open-index">
+ <test-group name="nested-open-index/index-join">
+ <test-case FilePath="nested-open-index/index-join">
+ <compilation-unit name="btree-secondary-equi-join">
+ <output-dir compare="Text">btree-secondary-equi-join</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-join">
+ <compilation-unit name="ngram-edit-distance">
+ <output-dir compare="Text">ngram-edit-distance</output-dir>
+ </compilation-unit>
+ </test-case>
+ <!-- <test-case FilePath="nested-open-index/index-join">
+ <compilation-unit name="ngram-edit-distance-inline">
+ <output-dir compare="Text">ngram-edit-distance-inline</output-dir>
+ </compilation-unit>
+ </test-case> -->
+ <test-case FilePath="nested-open-index/index-join">
+ <compilation-unit name="ngram-jaccard">
+ <output-dir compare="Text">ngram-jaccard</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-join">
+ <compilation-unit name="ngram-jaccard-inline">
+ <output-dir compare="Text">ngram-jaccard-inline</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-join">
+ <compilation-unit name="rtree-spatial-intersect-point">
+ <output-dir compare="Text">rtree-spatial-intersect-point</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-join">
+ <compilation-unit name="word-jaccard">
+ <output-dir compare="Text">word-jaccard</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-join">
+ <compilation-unit name="word-jaccard-inline">
+ <output-dir compare="Text">word-jaccard-inline</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-join">
+ <compilation-unit name="btree-secondary-non-enforced-equi-join">
+ <output-dir compare="Text">btree-secondary-non-enforced-equi-join</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-join">
+ <compilation-unit name="btree-secondary-non-enforced-equi-join-2">
+ <output-dir compare="Text">btree-secondary-non-enforced-equi-join-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="nested-open-index/index-leftouterjoin">
+ <test-case FilePath="nested-open-index/index-leftouterjoin">
+ <compilation-unit name="probe-pidx-with-join-btree-sidx1">
+ <output-dir compare="Text">probe-pidx-with-join-btree-sidx1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-leftouterjoin">
+ <compilation-unit name="probe-pidx-with-join-btree-sidx2">
+ <output-dir compare="Text">probe-pidx-with-join-btree-sidx2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-leftouterjoin">
+ <compilation-unit name="probe-pidx-with-join-invidx-sidx2">
+ <output-dir compare="Text">probe-pidx-with-join-invidx-sidx2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-leftouterjoin">
+ <compilation-unit name="probe-pidx-with-join-rtree-sidx1">
+ <output-dir compare="Text">probe-pidx-with-join-rtree-sidx1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-leftouterjoin">
+ <compilation-unit name="probe-pidx-with-join-rtree-sidx2">
+ <output-dir compare="Text">probe-pidx-with-join-rtree-sidx2</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="nested-open-index/index-selection">
+ <test-case FilePath="nested-open-index/index-selection">
+ <compilation-unit name="btree-index-composite-key">
+ <output-dir compare="Text">btree-index-composite-key</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-selection">
+ <compilation-unit name="btree-index-composite-key-mixed-intervals">
+ <output-dir compare="Text">btree-index-composite-key-mixed-intervals</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-selection">
+ <compilation-unit name="btree-index-rewrite-multiple">
+ <output-dir compare="Text">btree-index-rewrite-multiple</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-selection">
+ <compilation-unit name="inverted-index-ngram-contains">
+ <output-dir compare="Text">inverted-index-ngram-contains</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-selection">
+ <compilation-unit name="inverted-index-ngram-edit-distance">
+ <output-dir compare="Text">inverted-index-ngram-edit-distance</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-selection">
+ <compilation-unit name="inverted-index-ngram-edit-distance-contains">
+ <output-dir compare="Text">inverted-index-ngram-edit-distance-contains</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-selection">
+ <compilation-unit name="inverted-index-ngram-edit-distance-panic">
+ <output-dir compare="Text">inverted-index-ngram-edit-distance-panic</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-selection">
+ <compilation-unit name="inverted-index-ngram-edit-distance-word-tokens">
+ <output-dir compare="Text">inverted-index-ngram-edit-distance-word-tokens</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-selection">
+ <compilation-unit name="inverted-index-ngram-jaccard">
+ <output-dir compare="Text">inverted-index-ngram-jaccard</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-selection">
+ <compilation-unit name="inverted-index-word-contains">
+ <output-dir compare="Text">inverted-index-word-contains</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-selection">
+ <compilation-unit name="inverted-index-word-jaccard">
+ <output-dir compare="Text">inverted-index-word-jaccard</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-selection">
+ <compilation-unit name="orders-index-custkey">
+ <output-dir compare="Text">orders-index-custkey</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-selection">
+ <compilation-unit name="orders-index-custkey-conjunctive">
+ <output-dir compare="Text">orders-index-custkey-conjunctive</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-selection">
+ <compilation-unit name="range-search">
+ <output-dir compare="Text">range-search</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-selection">
+ <compilation-unit name="rtree-secondary-index">
+ <output-dir compare="Text">rtree-secondary-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-selection">
+ <compilation-unit name="non-enforced-01">
+ <output-dir compare="Text">non-enforced-01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-selection">
+ <compilation-unit name="non-enforced-02">
+ <output-dir compare="Text">non-enforced-02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-selection">
+ <compilation-unit name="non-enforced-03">
+ <output-dir compare="Text">non-enforced-03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-selection">
+ <compilation-unit name="non-enforced-04">
+ <output-dir compare="Text">non-enforced-04</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="nested-open-index/highly-open-highly-nested">
+ <test-case FilePath="nested-open-index/highly-open-highly-nested">
+ <compilation-unit name="bottom-closed-top-closed">
+ <output-dir compare="Text">bottom-closed-top-closed</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/highly-open-highly-nested">
+ <compilation-unit name="bottom-closed-top-open">
+ <output-dir compare="Text">bottom-closed-top-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/highly-open-highly-nested">
+ <compilation-unit name="bottom-open-top-closed">
+ <output-dir compare="Text">bottom-open-top-closed</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/highly-open-highly-nested">
+ <compilation-unit name="bottom-open-top-open">
+ <output-dir compare="Text">bottom-open-top-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ </test-group>
+ <test-group name="nested-index">
+ <test-group name="nested-index/index-join">
+ <test-case FilePath="nested-index/index-join">
+ <compilation-unit name="btree-primary-equi-join">
+ <output-dir compare="Text">btree-primary-equi-join</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-join">
+ <compilation-unit name="btree-secondary-equi-join">
+ <output-dir compare="Text">btree-secondary-equi-join</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-join">
+ <compilation-unit name="ngram-edit-distance">
+ <output-dir compare="Text">ngram-edit-distance</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-join">
+ <compilation-unit name="ngram-edit-distance-inline">
+ <output-dir compare="Text">ngram-edit-distance-inline</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-join">
+ <compilation-unit name="ngram-jaccard">
+ <output-dir compare="Text">ngram-jaccard</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-join">
+ <compilation-unit name="ngram-jaccard-inline">
+ <output-dir compare="Text">ngram-jaccard-inline</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-join">
+ <compilation-unit name="rtree-spatial-intersect-point">
+ <output-dir compare="Text">rtree-spatial-intersect-point</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-join">
+ <compilation-unit name="word-jaccard">
+ <output-dir compare="Text">word-jaccard</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-join">
+ <compilation-unit name="word-jaccard-inline">
+ <output-dir compare="Text">word-jaccard-inline</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="nested-index/index-leftouterjoin">
+ <test-case FilePath="nested-index/index-leftouterjoin">
+ <compilation-unit name="probe-pidx-with-join-btree-sidx1">
+ <output-dir compare="Text">probe-pidx-with-join-btree-sidx1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-leftouterjoin">
+ <compilation-unit name="probe-pidx-with-join-btree-sidx2">
+ <output-dir compare="Text">probe-pidx-with-join-btree-sidx2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-leftouterjoin">
+ <compilation-unit name="probe-pidx-with-join-invidx-sidx1">
+ <output-dir compare="Text">probe-pidx-with-join-invidx-sidx1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-leftouterjoin">
+ <compilation-unit name="probe-pidx-with-join-invidx-sidx2">
+ <output-dir compare="Text">probe-pidx-with-join-invidx-sidx2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-leftouterjoin">
+ <compilation-unit name="probe-pidx-with-join-rtree-sidx1">
+ <output-dir compare="Text">probe-pidx-with-join-rtree-sidx1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-leftouterjoin">
+ <compilation-unit name="probe-pidx-with-join-rtree-sidx2">
+ <output-dir compare="Text">probe-pidx-with-join-rtree-sidx2</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="nested-index/index-selection">
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="btree-index-composite-key">
+ <output-dir compare="Text">btree-index-composite-key</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="btree-sec-primary-index">
+ <output-dir compare="Text">btree-sec-primary-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="btree-index-composite-key-mixed-intervals">
+ <output-dir compare="Text">btree-index-composite-key-mixed-intervals</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="btree-index-rewrite-multiple">
+ <output-dir compare="Text">btree-index-rewrite-multiple</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="cust-index-age-nullable">
+ <output-dir compare="Text">cust-index-age-nullable</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="inverted-index-ngram-contains">
+ <output-dir compare="Text">inverted-index-ngram-contains</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="inverted-index-ngram-edit-distance">
+ <output-dir compare="Text">inverted-index-ngram-edit-distance</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="inverted-index-ngram-edit-distance-contains">
+ <output-dir compare="Text">inverted-index-ngram-edit-distance-contains</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="inverted-index-ngram-edit-distance-panic">
+ <output-dir compare="Text">inverted-index-ngram-edit-distance-panic</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="inverted-index-ngram-edit-distance-word-tokens">
+ <output-dir compare="Text">inverted-index-ngram-edit-distance-word-tokens</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="inverted-index-ngram-jaccard">
+ <output-dir compare="Text">inverted-index-ngram-jaccard</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="inverted-index-olist-edit-distance">
+ <output-dir compare="Text">inverted-index-olist-edit-distance</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="inverted-index-olist-edit-distance-panic">
+ <output-dir compare="Text">inverted-index-olist-edit-distance-panic</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="inverted-index-olist-jaccard">
+ <output-dir compare="Text">inverted-index-olist-jaccard</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="inverted-index-ulist-jaccard">
+ <output-dir compare="Text">inverted-index-ulist-jaccard</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="inverted-index-word-contains">
+ <output-dir compare="Text">inverted-index-word-contains</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="inverted-index-word-jaccard">
+ <output-dir compare="Text">inverted-index-word-jaccard</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="orders-index-custkey">
+ <output-dir compare="Text">orders-index-custkey</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="orders-index-custkey-conjunctive">
+ <output-dir compare="Text">orders-index-custkey-conjunctive</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="orders-index-custkey-conjunctive-open">
+ <output-dir compare="Text">orders-index-custkey-conjunctive-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="orders-index-custkey-open">
+ <output-dir compare="Text">orders-index-custkey-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="range-search">
+ <output-dir compare="Text">range-search</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="range-search-open">
+ <output-dir compare="Text">range-search-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="rtree-secondary-index">
+ <output-dir compare="Text">rtree-secondary-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="rtree-secondary-index-nullable">
+ <output-dir compare="Text">rtree-secondary-index-nullable</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="rtree-secondary-index-open">
+ <output-dir compare="Text">rtree-secondary-index-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="rtree-secondary-index-optional">
+ <output-dir compare="Text">rtree-secondary-index-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ </test-group>
+ <test-group name="nested-index-dml">
+ <test-case FilePath="nested-index-dml">
+ <compilation-unit name="compact-dataset-and-its-indexes">
+ <output-dir compare="Text">compact-dataset-and-its-indexes</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index-dml">
+ <compilation-unit name="nested-uuid-load">
+ <output-dir compare="Text">nested-uuid-load</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index-dml">
+ <compilation-unit name="nested-uuid-insert">
+ <output-dir compare="Text">nested-uuid-insert</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index-dml">
+ <compilation-unit name="delete-from-loaded-dataset-with-index">
+ <output-dir compare="Text">delete-from-loaded-dataset-with-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index-dml">
+ <compilation-unit name="delete-from-loaded-dataset-with-sec-primary-index">
+ <output-dir compare="Text">delete-from-loaded-dataset-with-sec-primary-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index-dml">
+ <compilation-unit name="drop-index">
+ <output-dir compare="Text">drop-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index-dml">
+ <compilation-unit name="insert-into-empty-dataset-with-index">
+ <output-dir compare="Text">insert-into-empty-dataset-with-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index-dml">
+ <compilation-unit name="insert-into-empty-dataset-with-sec-primary-index">
+ <output-dir compare="Text">insert-into-empty-dataset-with-sec-primary-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index-dml">
+ <compilation-unit name="insert-into-loaded-dataset-with-index_01">
+ <output-dir compare="Text">insert-into-loaded-dataset-with-index_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index-dml">
+ <compilation-unit name="insert-into-loaded-dataset-with-index_02">
+ <output-dir compare="Text">insert-into-loaded-dataset-with-index_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index-dml">
+ <compilation-unit name="insert-into-loaded-dataset-with-sec-primary-index">
+ <output-dir compare="Text">insert-into-loaded-dataset-with-sec-primary-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index-dml">
+ <compilation-unit name="load-with-index">
+ <output-dir compare="Text">load-with-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index-dml">
+ <compilation-unit name="load-with-sec-primary-index">
+ <output-dir compare="Text">load-with-sec-primary-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index-dml">
+ <compilation-unit name="load-with-ngram-index">
+ <output-dir compare="Text">load-with-ngram-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index-dml">
+ <compilation-unit name="load-with-rtree-index">
+ <output-dir compare="Text">load-with-rtree-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index-dml">
+ <compilation-unit name="load-with-word-index">
+ <output-dir compare="Text">load-with-word-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index-dml">
+ <compilation-unit name="scan-delete-btree-secondary-index-nullable">
+ <output-dir compare="Text">scan-delete-btree-secondary-index-nullable</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index-dml">
+ <compilation-unit name="scan-delete-rtree-secondary-index-nullable">
+ <output-dir compare="Text">scan-delete-rtree-secondary-index-nullable</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index-dml">
+ <compilation-unit name="scan-delete-rtree-secondary-index">
+ <output-dir compare="Text">scan-delete-rtree-secondary-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index-dml">
+ <compilation-unit name="scan-insert-btree-secondary-index-nullable">
+ <output-dir compare="Text">scan-insert-btree-secondary-index-nullable</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index-dml">
+ <compilation-unit name="scan-insert-rtree-secondary-index-nullable">
+ <output-dir compare="Text">scan-insert-rtree-secondary-index-nullable</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index-dml">
+ <compilation-unit name="scan-insert-rtree-secondary-index">
+ <output-dir compare="Text">scan-insert-rtree-secondary-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index-dml">
+ <compilation-unit name="scan-insert-inverted-index-ngram-secondary-index">
+ <output-dir compare="Text">scan-insert-inverted-index-ngram-secondary-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index-dml">
+ <compilation-unit name="scan-insert-inverted-index-word-secondary-index">
+ <output-dir compare="Text">scan-insert-inverted-index-word-secondary-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index-dml">
+ <compilation-unit name="scan-insert-inverted-index-ngram-secondary-index-nullable">
+ <output-dir compare="Text">scan-insert-inverted-index-ngram-secondary-index-nullable</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index-dml">
+ <compilation-unit name="scan-insert-inverted-index-word-secondary-index-nullable">
+ <output-dir compare="Text">scan-insert-inverted-index-word-secondary-index-nullable</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index-dml">
+ <compilation-unit name="scan-delete-inverted-index-ngram-secondary-index">
+ <output-dir compare="Text">scan-delete-inverted-index-ngram-secondary-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index-dml">
+ <compilation-unit name="scan-delete-inverted-index-word-secondary-index">
+ <output-dir compare="Text">scan-delete-inverted-index-word-secondary-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index-dml">
+ <compilation-unit name="scan-delete-inverted-index-ngram-secondary-index-nullable">
+ <output-dir compare="Text">scan-delete-inverted-index-ngram-secondary-index-nullable</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index-dml">
+ <compilation-unit name="scan-delete-inverted-index-word-secondary-index-nullable">
+ <output-dir compare="Text">scan-delete-inverted-index-word-secondary-index-nullable</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="array-index">
+ <test-group name="array-index/error-handling">
+ <test-case FilePath="array-index/error-handling">
+ <compilation-unit name="index-include-unknown-key">
+ <output-dir compare="Text">index-include-unknown-key</output-dir>
+ <expected-error>ASX1079: Compilation error: Array indexes must specify EXCLUDE UNKNOWN KEY</expected-error>
+ <expected-error>ASX1079: Compilation error: Array indexes must specify EXCLUDE UNKNOWN KEY</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/error-handling">
+ <compilation-unit name="index-two-array-fields">
+ <output-dir compare="Text">index-two-array-fields</output-dir>
+ <expected-error>ASX1079: Compilation error: Cannot create composite index with multiple array fields using different arrays</expected-error>
+ <expected-error>ASX1079: Compilation error: Cannot create composite index with multiple array fields using different arrays</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/error-handling">
+ <compilation-unit name="invalid-array-path">
+ <output-dir compare="Text">invalid-array-path</output-dir>
+ <expected-error>ASX0037: Type mismatch: expected value of type array or multiset, but got the value of type CheckinType_checkin_time:</expected-error>
+ <expected-error>ASX0037: Type mismatch: expected value of type array or multiset, but got the value of type string</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/error-handling">
+ <compilation-unit name="index-on-closed-array">
+ <output-dir compare="Text">index-on-closed-array</output-dir>
+ <expected-error>ASX1014: Field 'date' is not found</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/error-handling">
+ <compilation-unit name="index-with-enforced-type">
+ <output-dir compare="Text">index-with-enforced-type</output-dir>
+ <expected-error>ASX1155: Incompatible index type ARRAY</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/error-handling">
+ <compilation-unit name="no-field-type">
+ <output-dir compare="Text">no-field-type</output-dir>
+ <expected-error>ASX1079: Compilation error: cannot find type of elements of field 'open_array_f'</expected-error>
+ <expected-error>ASX1079: Compilation error: cannot find type of elements of field 'nested_array'</expected-error>
+ <expected-error>ASX1079: Compilation error: cannot find type of elements of field 'nested_array'</expected-error>
+ <expected-error>ASX1079: Compilation error: cannot find type of field 'proj1'</expected-error>
+ <expected-error>ASX1079: Compilation error: cannot find type of field 'proj1'</expected-error>
+ <expected-error>ASX1079: Compilation error: cannot find type of field 'proj1'</expected-error>
+ <expected-error>ASX1079: Compilation error: cannot find type of field 'proj1'</expected-error>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="array-index/metadata">
+ <test-case FilePath="array-index/metadata/closed">
+ <compilation-unit name="use-case-1">
+ <output-dir compare="Text">use-case-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/metadata/open">
+ <compilation-unit name="use-case-1">
+ <output-dir compare="Text">use-case-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/metadata/closed">
+ <compilation-unit name="use-case-2">
+ <output-dir compare="Text">use-case-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/metadata/open">
+ <compilation-unit name="use-case-2">
+ <output-dir compare="Text">use-case-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/metadata/closed">
+ <compilation-unit name="use-case-3">
+ <output-dir compare="Text">use-case-3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/metadata/open">
+ <compilation-unit name="use-case-3">
+ <output-dir compare="Text">use-case-3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/metadata/closed">
+ <compilation-unit name="use-case-4">
+ <output-dir compare="Text">use-case-4</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/metadata/open">
+ <compilation-unit name="use-case-4">
+ <output-dir compare="Text">use-case-4</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/metadata/open">
+ <compilation-unit name="complex-structures">
+ <output-dir compare="Text">complex-structures</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/metadata/closed">
+ <compilation-unit name="with-composite-sk">
+ <output-dir compare="Text">with-composite-sk</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/metadata/open">
+ <compilation-unit name="with-composite-sk">
+ <output-dir compare="Text">with-composite-sk</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/metadata/closed">
+ <compilation-unit name="with-composite-array-different-indicators">
+ <output-dir compare="Text">with-composite-array-different-indicators</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/metadata/open">
+ <compilation-unit name="with-composite-array-different-indicators">
+ <output-dir compare="Text">with-composite-array-different-indicators</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/metadata/closed">
+ <compilation-unit name="with-3-level-record-path">
+ <output-dir compare="Text">with-3-level-record-path</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/metadata/open">
+ <compilation-unit name="with-3-level-record-path">
+ <output-dir compare="Text">with-3-level-record-path</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="array-index/bulk-loading/on-index-creation">
+ <test-case FilePath="array-index/bulk-loading/on-index-creation/closed">
+ <compilation-unit name="use-case-1">
+ <output-dir compare="Text">use-case-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/bulk-loading/on-index-creation/open">
+ <compilation-unit name="use-case-1">
+ <output-dir compare="Text">use-case-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/bulk-loading/on-index-creation/closed">
+ <compilation-unit name="use-case-2">
+ <output-dir compare="Text">use-case-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/bulk-loading/on-index-creation/open">
+ <compilation-unit name="use-case-2">
+ <output-dir compare="Text">use-case-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/bulk-loading/on-index-creation/closed">
+ <compilation-unit name="use-case-3">
+ <output-dir compare="Text">use-case-3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/bulk-loading/on-index-creation/open">
+ <compilation-unit name="use-case-3">
+ <output-dir compare="Text">use-case-3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/bulk-loading/on-index-creation/closed">
+ <compilation-unit name="use-case-4">
+ <output-dir compare="Text">use-case-4</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/bulk-loading/on-index-creation/open">
+ <compilation-unit name="use-case-4">
+ <output-dir compare="Text">use-case-4</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/bulk-loading/on-index-creation/closed">
+ <compilation-unit name="with-composite-pk">
+ <output-dir compare="Text">with-composite-pk</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/bulk-loading/on-index-creation/closed">
+ <compilation-unit name="with-filter-fields">
+ <output-dir compare="Text">with-filter-fields</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/bulk-loading/on-index-creation/closed">
+ <compilation-unit name="with-3-level-record-path">
+ <output-dir compare="Text">with-3-level-record-path</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/bulk-loading/on-index-creation/open">
+ <compilation-unit name="composite-atomic">
+ <output-dir compare="Text">composite-atomic</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="array-index/bulk-loading/after-index-creation">
+ <test-case FilePath="array-index/bulk-loading/after-index-creation">
+ <compilation-unit name="use-case-1">
+ <output-dir compare="Text">use-case-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/bulk-loading/after-index-creation">
+ <compilation-unit name="use-case-2">
+ <output-dir compare="Text">use-case-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/bulk-loading/after-index-creation">
+ <compilation-unit name="use-case-3">
+ <output-dir compare="Text">use-case-3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/bulk-loading/after-index-creation">
+ <compilation-unit name="use-case-4">
+ <output-dir compare="Text">use-case-4</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/bulk-loading/after-index-creation">
+ <compilation-unit name="with-3-level-record-path">
+ <output-dir compare="Text">with-3-level-record-path</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/bulk-loading/after-index-creation">
+ <compilation-unit name="with-composite-pk">
+ <output-dir compare="Text">with-composite-pk</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/bulk-loading/after-index-creation">
+ <compilation-unit name="with-filter-fields">
+ <output-dir compare="Text">with-filter-fields</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/bulk-loading/after-index-creation">
+ <compilation-unit name="with-open-index">
+ <output-dir compare="Text">with-open-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="array-index/insert-upsert-delete">
+ <test-case FilePath="array-index/insert-upsert-delete/closed">
+ <compilation-unit name="use-case-1">
+ <output-dir compare="Text">use-case-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/insert-upsert-delete/open">
+ <compilation-unit name="use-case-1">
+ <output-dir compare="Text">use-case-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/insert-upsert-delete/closed">
+ <compilation-unit name="use-case-2">
+ <output-dir compare="Text">use-case-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/insert-upsert-delete/open">
+ <compilation-unit name="use-case-2">
+ <output-dir compare="Text">use-case-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/insert-upsert-delete/closed">
+ <compilation-unit name="use-case-3">
+ <output-dir compare="Text">use-case-3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/insert-upsert-delete/open">
+ <compilation-unit name="use-case-3">
+ <output-dir compare="Text">use-case-3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/insert-upsert-delete/closed">
+ <compilation-unit name="use-case-4">
+ <output-dir compare="Text">use-case-4</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/insert-upsert-delete/open">
+ <compilation-unit name="use-case-4">
+ <output-dir compare="Text">use-case-4</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/insert-upsert-delete/closed">
+ <compilation-unit name="with-composite-sk">
+ <output-dir compare="Text">with-composite-sk</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/insert-upsert-delete/open">
+ <compilation-unit name="with-composite-sk">
+ <output-dir compare="Text">with-composite-sk</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/insert-upsert-delete/closed">
+ <compilation-unit name="with-additional-atomic-index">
+ <output-dir compare="Text">with-additional-atomic-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/insert-upsert-delete/open">
+ <compilation-unit name="with-additional-atomic-index">
+ <output-dir compare="Text">with-additional-atomic-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/insert-upsert-delete/closed">
+ <compilation-unit name="with-filter-fields">
+ <output-dir compare="Text">with-filter-fields</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="array-index/join-quantified-queries">
+ <test-case FilePath="array-index/join-quantified-queries">
+ <compilation-unit name="use-case-1">
+ <output-dir compare="Text">use-case-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/join-quantified-queries">
+ <compilation-unit name="use-case-2">
+ <output-dir compare="Text">use-case-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/join-quantified-queries">
+ <compilation-unit name="use-case-3">
+ <output-dir compare="Text">use-case-3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/join-quantified-queries">
+ <compilation-unit name="use-case-4">
+ <output-dir compare="Text">use-case-4</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="array-index/join-unnest-queries">
+ <test-case FilePath="array-index/join-unnest-queries">
+ <compilation-unit name="use-case-1">
+ <output-dir compare="Text">use-case-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/join-unnest-queries">
+ <compilation-unit name="use-case-2">
+ <output-dir compare="Text">use-case-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/join-unnest-queries">
+ <compilation-unit name="use-case-3">
+ <output-dir compare="Text">use-case-3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/join-unnest-queries">
+ <compilation-unit name="use-case-4">
+ <output-dir compare="Text">use-case-4</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/join-unnest-queries">
+ <compilation-unit name="with-open-index">
+ <output-dir compare="Text">with-open-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/join-unnest-queries">
+ <compilation-unit name="loj-subquery">
+ <output-dir compare="Text">loj-subquery</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="array-index/select-unnest-queries">
+ <test-case FilePath="array-index/select-unnest-queries/closed">
+ <compilation-unit name="use-case-1">
+ <output-dir compare="Text">use-case-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/open">
+ <compilation-unit name="use-case-1">
+ <output-dir compare="Text">use-case-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/closed">
+ <compilation-unit name="use-case-2">
+ <output-dir compare="Text">use-case-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/open">
+ <compilation-unit name="use-case-2">
+ <output-dir compare="Text">use-case-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/closed">
+ <compilation-unit name="use-case-3">
+ <output-dir compare="Text">use-case-3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/open">
+ <compilation-unit name="use-case-3">
+ <output-dir compare="Text">use-case-3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/closed">
+ <compilation-unit name="use-case-4">
+ <output-dir compare="Text">use-case-4</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/open">
+ <compilation-unit name="use-case-4">
+ <output-dir compare="Text">use-case-4</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/closed">
+ <compilation-unit name="with-3-level-record-path">
+ <output-dir compare="Text">with-3-level-record-path</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/open">
+ <compilation-unit name="with-3-level-record-path">
+ <output-dir compare="Text">with-3-level-record-path</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/closed">
+ <compilation-unit name="with-composite-sk">
+ <output-dir compare="Text">with-composite-sk</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/open">
+ <compilation-unit name="with-composite-sk">
+ <output-dir compare="Text">with-composite-sk</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/closed">
+ <compilation-unit name="with-composite-pk">
+ <output-dir compare="Text">with-composite-pk</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/closed">
+ <compilation-unit name="with-filter-fields">
+ <output-dir compare="Text">with-filter-fields</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/open">
+ <compilation-unit name="using-feed-new-index">
+ <output-dir compare="Text">using-feed</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/open">
+ <compilation-unit name="using-feed-old-index">
+ <output-dir compare="Text">using-feed</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/open">
+ <compilation-unit name="multiple-indexes">
+ <output-dir compare="Text">multiple-indexes</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="array-index/select-quantified-queries">
+ <test-case FilePath="array-index/select-quantified-queries">
+ <compilation-unit name="use-case-1">
+ <output-dir compare="Text">use-case-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-quantified-queries">
+ <compilation-unit name="use-case-2">
+ <output-dir compare="Text">use-case-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-quantified-queries">
+ <compilation-unit name="use-case-3">
+ <output-dir compare="Text">use-case-3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-quantified-queries">
+ <compilation-unit name="use-case-4">
+ <output-dir compare="Text">use-case-4</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-quantified-queries">
+ <compilation-unit name="with-composite-pk">
+ <output-dir compare="Text">with-composite-pk</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-quantified-queries">
+ <compilation-unit name="with-open-index">
+ <output-dir compare="Text">with-open-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-quantified-queries">
+ <compilation-unit name="multiple-indexes">
+ <output-dir compare="Text">multiple-indexes</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="array-index/composite-index-queries">
+ <test-case FilePath="array-index">
+ <compilation-unit name="composite-index-queries">
+ <output-dir compare="Text">composite-index-queries</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="array-index/index-bad-fields">
+ <test-case FilePath="array-index">
+ <compilation-unit name="index-bad-fields">
+ <output-dir compare="Text">index-bad-fields</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ </test-group>
+ <test-group name="nestrecords">
+ <test-case FilePath="nestrecords">
+ <compilation-unit name="nestrecord">
+ <output-dir compare="Text">nestrecord</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nestrecords">
+ <compilation-unit name="nested-optional-pk">
+ <output-dir compare="Text">nested-optional-pk</output-dir>
+ <expected-error>ASX1021: The primary key field 'nested.id' cannot be nullable</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nestrecords">
+ <compilation-unit name="query-ASTERIXDB-1025">
+ <output-dir compare="Text">query-ASTERIXDB-1025</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="null-missing">
+ <test-case FilePath="null-missing">
+ <compilation-unit name="array">
+ <output-dir compare="Text">array</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="null-missing">
+ <compilation-unit name="boolean">
+ <output-dir compare="Text">boolean</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="null-missing">
+ <compilation-unit name="ceil">
+ <output-dir compare="Text">ceil</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="null-missing">
+ <compilation-unit name="comparison">
+ <output-dir compare="Text">comparison</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="null-missing">
+ <compilation-unit name="field-access">
+ <output-dir compare="Text">field-access</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="null-missing">
+ <compilation-unit name="group-by">
+ <output-dir compare="Text">group-by</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="null-missing">
+ <compilation-unit name="group-by-from-dataset">
+ <output-dir compare="Text">group-by-from-dataset</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="null-missing">
+ <compilation-unit name="length">
+ <output-dir compare="Text">length</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="null-missing">
+ <compilation-unit name="lower">
+ <output-dir compare="Text">lower</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="null-missing">
+ <compilation-unit name="order-by">
+ <output-dir compare="Text">order-by</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="null-missing">
+ <compilation-unit name="order-by-2">
+ <output-dir compare="Text">order-by-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="null-missing">
+ <compilation-unit name="order-by-3-negative">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error><![CDATA[ASX1001: Syntax error: In line 33 >>ORDER BY d.a NULLS;<< Encountered ";" at column 19]]></expected-error>
+ <expected-error><![CDATA[ASX1001: Syntax error: Unexpected token: NULLS (in line 33, at column 20)]]></expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="null-missing">
+ <compilation-unit name="order-by-from-dataset">
+ <output-dir compare="Text">order-by-from-dataset</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="null-missing">
+ <compilation-unit name="order-by-from-dataset-2">
+ <output-dir compare="Text">order-by-from-dataset-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="null-missing">
+ <compilation-unit name="scan-collection">
+ <output-dir compare="Text">scan-collection</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="null-missing">
+ <compilation-unit name="spatial-intersect">
+ <output-dir compare="Text">spatial-intersect</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="null-missing">
+ <compilation-unit name="substr">
+ <output-dir compare="Text">substr</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="null-missing">
+ <compilation-unit name="upper">
+ <output-dir compare="Text">upper</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="null-missing">
+ <compilation-unit name="is">
+ <output-dir compare="Text">is</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="null-missing">
+ <compilation-unit name="ifmissing">
+ <output-dir compare="Text">ifmissing</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="null-missing">
+ <compilation-unit name="ifnull">
+ <output-dir compare="Text">ifnull</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="null-missing">
+ <compilation-unit name="ifmissingornull">
+ <output-dir compare="Text">ifmissingornull</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="null-missing">
+ <compilation-unit name="coalesce">
+ <output-dir compare="Text">coalesce</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="null-missing">
+ <compilation-unit name="query-ASTERIXDB-1689">
+ <output-dir compare="Text">query-ASTERIXDB-1689</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="null-missing">
+ <compilation-unit name="print-ASTERIXDB-1885">
+ <output-dir compare="Text">print-ASTERIXDB-1885</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="numeric">
+ <test-case FilePath="numeric">
+ <compilation-unit name="caret0">
+ <output-dir compare="Text">caret0</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="caret1">
+ <output-dir compare="Text">caret1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="abs0">
+ <output-dir compare="Text">abs0</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="abs1">
+ <output-dir compare="Text">abs1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="abs2">
+ <output-dir compare="Text">abs2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="abs3">
+ <output-dir compare="Text">abs3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="abs4">
+ <output-dir compare="Text">abs4</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="add_double">
+ <output-dir compare="Text">add_double</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="add_float">
+ <output-dir compare="Text">add_float</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="add_int16">
+ <output-dir compare="Text">add_int16</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="add_int32">
+ <output-dir compare="Text">add_int32</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="add_int64">
+ <output-dir compare="Text">add_int64</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="add_int8">
+ <output-dir compare="Text">add_int8</output-dir>
+ </compilation-unit>
+ </test-case>
+ <!--
+ <test-case FilePath="numeric">
+ <compilation-unit name="issue_1166">
+ <output-dir compare="Text">issue_1166</output-dir>
+ </compilation-unit>
+ </test-case>
+ -->
+ <test-case FilePath="numeric">
+ <compilation-unit name="ceiling0">
+ <output-dir compare="Text">ceiling0</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="ceiling1">
+ <output-dir compare="Text">ceiling1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="ceiling2">
+ <output-dir compare="Text">ceiling2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="ceiling3">
+ <output-dir compare="Text">ceiling3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="ceiling4">
+ <output-dir compare="Text">ceiling4</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="const">
+ <output-dir compare="Text">const</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="degrees">
+ <output-dir compare="Text">degrees</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="divide_double">
+ <output-dir compare="Text">divide_double</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="divide_float">
+ <output-dir compare="Text">divide_float</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="divide_int16">
+ <output-dir compare="Text">divide_int16</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="divide_int32">
+ <output-dir compare="Text">divide_int32</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="divide_int64">
+ <output-dir compare="Text">divide_int64</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="divide_int8">
+ <output-dir compare="Text">divide_int8</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="div_mod_case_insensitive">
+ <output-dir compare="Text">div_mod_case_insensitive</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="floor0">
+ <output-dir compare="Text">floor0</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="floor1">
+ <output-dir compare="Text">floor1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="floor2">
+ <output-dir compare="Text">floor2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="floor3">
+ <output-dir compare="Text">floor3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="floor4">
+ <output-dir compare="Text">floor4</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="ifinf">
+ <output-dir compare="Text">ifinf</output-dir>
+ <expected-error>Invalid number of arguments for function if-inf (in line 25, at column 14)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="ifnan">
+ <output-dir compare="Text">ifnan</output-dir>
+ <expected-error>Invalid number of arguments for function if-nan (in line 25, at column 14)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="ifnanorinf">
+ <output-dir compare="Text">ifnanorinf</output-dir>
+ <expected-error>Invalid number of arguments for function if-nan-or-inf (in line 25, at column 14)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="infinity">
+ <output-dir compare="Text">infinity</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="multiply_double">
+ <output-dir compare="Text">multiply_double</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="multiply_float">
+ <output-dir compare="Text">multiply_float</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="multiply_int16">
+ <output-dir compare="Text">multiply_int16</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="multiply_int32">
+ <output-dir compare="Text">multiply_int32</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="multiply_int64">
+ <output-dir compare="Text">multiply_int64</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="multiply_int8">
+ <output-dir compare="Text">multiply_int8</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="power">
+ <output-dir compare="Text">power</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="query-ASTERIXDB-2530">
+ <output-dir compare="Text">query-ASTERIXDB-2530</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="radians">
+ <output-dir compare="Text">radians</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="round-half-to-even0">
+ <output-dir compare="Text">round-half-to-even0</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="round-half-to-even1">
+ <output-dir compare="Text">round-half-to-even1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="round-half-to-even2">
+ <output-dir compare="Text">round-half-to-even2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="round-half-to-even20">
+ <output-dir compare="Text">round-half-to-even20</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="round-half-to-even21">
+ <output-dir compare="Text">round-half-to-even21</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="round-half-to-even22">
+ <output-dir compare="Text">round-half-to-even22</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="round-half-to-even23">
+ <output-dir compare="Text">round-half-to-even23</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="round-half-to-even24">
+ <output-dir compare="Text">round-half-to-even24</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="round-half-to-even3">
+ <output-dir compare="Text">round-half-to-even3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="round-half-to-even4">
+ <output-dir compare="Text">round-half-to-even4</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="round-half-to-even5">
+ <output-dir compare="Text">round-half-to-even5</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="round-half-up">
+ <output-dir compare="Text">round-half-up</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="round0">
+ <output-dir compare="Text">round0</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="round1">
+ <output-dir compare="Text">round1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="round2">
+ <output-dir compare="Text">round2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="round3">
+ <output-dir compare="Text">round3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="round4">
+ <output-dir compare="Text">round4</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="round5_with_digit_int8">
+ <output-dir compare="Text">round5_with_digit_int8</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="round6_with_digit_int16">
+ <output-dir compare="Text">round6_with_digit_int16</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="round7_with_digit_int32">
+ <output-dir compare="Text">round7_with_digit_int32</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="round8_with_digit_int64">
+ <output-dir compare="Text">round8_with_digit_int64</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="round9_with_digit_float">
+ <output-dir compare="Text">round9_with_digit_float</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="round10_with_digit_double">
+ <output-dir compare="Text">round10_with_digit_double</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="round11_invalid">
+ <output-dir compare="Text">round11_invalid</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="subtract_double">
+ <output-dir compare="Text">subtract_double</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="subtract_float">
+ <output-dir compare="Text">subtract_float</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="subtract_int16">
+ <output-dir compare="Text">subtract_int16</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="subtract_int32">
+ <output-dir compare="Text">subtract_int32</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="subtract_int64">
+ <output-dir compare="Text">subtract_int64</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="subtract_int8">
+ <output-dir compare="Text">subtract_int8</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="trunc">
+ <output-dir compare="Text">trunc</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="unary-minus_double_02">
+ <output-dir compare="Text">unary-minus_double_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="unary-minus_float_02">
+ <output-dir compare="Text">unary-minus_float_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="unary-minus_int_02">
+ <output-dir compare="Text">unary-minus_int_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="unary-minus_null">
+ <output-dir compare="Text">unary-minus_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="query-issue355"> <!-- @Yingyi, This one fails in the aql tests!! -->
+ <output-dir compare="Text">query-issue355</output-dir>
+ <!-- <expected-error>org.apache.asterix.common.exceptions.AsterixException</expected-error> -->
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="acos">
+ <output-dir compare="Text">acos</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="asin">
+ <output-dir compare="Text">asin</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="atan">
+ <output-dir compare="Text">atan</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="atan2">
+ <output-dir compare="Text">atan2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="cos">
+ <output-dir compare="Text">cos</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="cosh">
+ <output-dir compare="Text">cosh</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="sin">
+ <output-dir compare="Text">sin</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="sinh">
+ <output-dir compare="Text">sinh</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="tan">
+ <output-dir compare="Text">tan</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="tanh">
+ <output-dir compare="Text">tanh</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="exp">
+ <output-dir compare="Text">exp</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="ln">
+ <output-dir compare="Text">ln</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="log">
+ <output-dir compare="Text">log</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="sqrt">
+ <output-dir compare="Text">sqrt</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="scientific">
+ <output-dir compare="Text">scientific</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="scientific_error">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>ASX1073: Cannot resolve alias reference for undefined identifier e (in line 24, at column 10)</expected-error>
+ <expected-error>ASX1073: Cannot resolve alias reference for undefined identifier e (in line 24, at column 8)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="sign">
+ <output-dir compare="Text">sign</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="non-finite">
+ <output-dir compare="Clean-JSON">non-finite</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="add_int8">
+ <output-dir compare="Text">add_int8</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="multiply_int8">
+ <output-dir compare="Text">multiply_int8</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="unary-minus_double_02">
+ <output-dir compare="Text">unary-minus_double_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="open-closed">
+ <!--
+ <test-case FilePath="open-closed">
+ <compilation-unit name="c2c-w-optional">
+ <output-dir compare="Text">c2c-w-optional</output-dir>
+ </compilation-unit>
+ </test-case>
+ -->
+ <!--
+ <test-case FilePath="open-closed">
+ <compilation-unit name="c2c-wo-optional">
+ <output-dir compare="Text">c2c-wo-optional</output-dir>
+ </compilation-unit>
+ </test-case>
+ -->
+ <!--
+ <test-case FilePath="open-closed">
+ <compilation-unit name="c2c">
+ <output-dir compare="Text">c2c</output-dir>
+ </compilation-unit>
+ </test-case>
+ -->
+ <test-case FilePath="open-closed">
+ <compilation-unit name="heterog-list-ordered01">
+ <output-dir compare="Text">heterog-list-ordered01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-closed">
+ <compilation-unit name="heterog-list01">
+ <output-dir compare="Text">heterog-list01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <!--
+ <test-case FilePath="open-closed">
+ <compilation-unit name="heterog-list02">
+ <output-dir compare="Text">heterog-list02</output-dir>
+ </compilation-unit>
+ </test-case>
+ -->
+ <!--
+ <test-case FilePath="open-closed">
+ <compilation-unit name="heterog-list03">
+ <output-dir compare="Text">heterog-list03</output-dir>
+ </compilation-unit>
+ </test-case>
+ -->
+ <test-case FilePath="open-closed">
+ <compilation-unit name="open-closed-01">
+ <output-dir compare="Text">open-closed-01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-closed">
+ <compilation-unit name="open-closed-12">
+ <output-dir compare="Text">open-closed-12</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-closed">
+ <compilation-unit name="open-closed-14">
+ <output-dir compare="Text">open-closed-14</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-closed">
+ <compilation-unit name="query-issue134">
+ <output-dir compare="Text">query-issue134</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-closed">
+ <compilation-unit name="query-issue55">
+ <output-dir compare="Text">query-issue55</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-closed">
+ <compilation-unit name="query-issue55-1">
+ <output-dir compare="Text">query-issue55-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-closed">
+ <compilation-unit name="query-issue166">
+ <output-dir compare="Text">query-issue166</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-closed">
+ <compilation-unit name="query-issue208">
+ <output-dir compare="Text">query-issue208</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-closed">
+ <compilation-unit name="query-issue236">
+ <output-dir compare="Text">query-issue236</output-dir>
+ </compilation-unit>
+ </test-case>
+ <!--
+ <test-case FilePath="open-closed">
+ <compilation-unit name="open-closed-15">
+ <output-dir compare="Text">open-closed-15</output-dir>
+ </compilation-unit>
+ </test-case>
+ -->
+ <!--
+ <test-case FilePath="open-closed">
+ <compilation-unit name="open-closed-16">
+ <output-dir compare="Text">open-closed-16</output-dir>
+ </compilation-unit>
+ </test-case>
+ -->
+ <!--
+ <test-case FilePath="open-closed">
+ <compilation-unit name="open-closed-17">
+ <output-dir compare="Text">open-closed-17</output-dir>
+ </compilation-unit>
+ </test-case>
+ -->
+ <!--
+ <test-case FilePath="open-closed">
+ <compilation-unit name="open-closed-19">
+ <output-dir compare="Text">open-closed-19</output-dir>
+ </compilation-unit>
+ </test-case>
+ -->
+ <!--
+ <test-case FilePath="open-closed">
+ <compilation-unit name="open-closed-20">
+ <output-dir compare="Text">open-closed-20</output-dir>
+ </compilation-unit>
+ </test-case>
+ -->
+ <!--
+ <test-case FilePath="open-closed">
+ <compilation-unit name="open-closed-21">
+ <output-dir compare="Text">open-closed-21</output-dir>
+ </compilation-unit>
+ </test-case>
+ -->
+ <!--
+ <test-case FilePath="open-closed">
+ <compilation-unit name="open-closed-22">
+ <output-dir compare="Text">open-closed-22</output-dir>
+ </compilation-unit>
+ </test-case>
+ -->
+ <test-case FilePath="open-closed">
+ <compilation-unit name="open-closed-24">
+ <output-dir compare="Text">open-closed-24</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-closed">
+ <compilation-unit name="open-closed-25">
+ <output-dir compare="Text">open-closed-25</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-closed">
+ <compilation-unit name="open-closed-26">
+ <output-dir compare="Text">open-closed-26</output-dir>
+ </compilation-unit>
+ </test-case>
+ <!--
+ <test-case FilePath="open-closed">
+ <compilation-unit name="open-closed-28">
+ <output-dir compare="Text">open-closed-28</output-dir>
+ </compilation-unit>
+ </test-case>
+ -->
+ <test-case FilePath="open-closed">
+ <compilation-unit name="open-closed-29">
+ <output-dir compare="Text">open-closed-29</output-dir>
+ </compilation-unit>
+ </test-case>
+ <!--
+ <test-case FilePath="open-closed">
+ <compilation-unit name="open-closed-30">
+ <output-dir compare="Text">open-closed-30</output-dir>
+ </compilation-unit>
+ </test-case>
+ -->
+ <test-case FilePath="open-closed">
+ <compilation-unit name="open-closed-31">
+ <output-dir compare="Text">open-closed-31</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-closed">
+ <compilation-unit name="open-closed-32">
+ <output-dir compare="Text">open-closed-32</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-closed">
+ <compilation-unit name="open-closed-33">
+ <output-dir compare="Text">open-closed-33</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-closed">
+ <compilation-unit name="query-proposal02">
+ <output-dir compare="Text">query-proposal02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-closed">
+ <compilation-unit name="query-proposal">
+ <output-dir compare="Text">query-proposal</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-closed">
+ <compilation-unit name="query-issue350">
+ <output-dir compare="Text">query-issue350</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-closed">
+ <compilation-unit name="query-issue350-2">
+ <output-dir compare="Text">query-issue350-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-closed">
+ <compilation-unit name="query-issue343">
+ <output-dir compare="Text">query-issue343</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-closed">
+ <compilation-unit name="query-issue343-2">
+ <output-dir compare="Text">query-issue343-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-closed">
+ <compilation-unit name="query-issue196">
+ <output-dir compare="Text">query-issue196</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-closed">
+ <compilation-unit name="query-issue377">
+ <output-dir compare="Text">query-issue377</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-closed">
+ <compilation-unit name="query-issue410">
+ <output-dir compare="Text">query-issue410</output-dir>
+ <expected-error>Field type double cannot be promoted to type string</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-closed">
+ <compilation-unit name="query-issue453">
+ <output-dir compare="Text">query-issue453</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-closed">
+ <compilation-unit name="query-issue453-2">
+ <output-dir compare="Text">query-issue453-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-closed">
+ <compilation-unit name="query-issue456">
+ <output-dir compare="Text">query-issue456</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-closed">
+ <compilation-unit name="query-issue465">
+ <output-dir compare="Text">query-issue465</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-closed">
+ <compilation-unit name="query-issue258">
+ <output-dir compare="Text">query-issue258</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-closed">
+ <compilation-unit name="query-issue423">
+ <output-dir compare="Text">query-issue423</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-closed">
+ <compilation-unit name="query-issue423-2">
+ <output-dir compare="Text">query-issue423-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-closed">
+ <compilation-unit name="query-issue442"><!-- Exception is not thrown!! -->
+ <output-dir compare="Text">query-issue442</output-dir>
+ <!-- <expected-error>org.apache.asterix.common.exceptions.AsterixException</expected-error> -->
+ </compilation-unit>
+ </test-case>
+ <!--<test-case FilePath="open-closed">
+ <compilation-unit name="query-issue487">
+ <output-dir compare="Text">query-issue487</output-dir>
+ <expected-error>org.apache.asterix.common.exceptions.AsterixException</expected-error>
+ </compilation-unit>
+ </test-case> -->
+ <test-case FilePath="open-closed">
+ <compilation-unit name="query-issue592">
+ <output-dir compare="Text">query-issue592</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-closed">
+ <compilation-unit name="query-issue625">
+ <output-dir compare="Text">query-issue625</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-closed">
+ <compilation-unit name="query-issue208">
+ <output-dir compare="Text">query-issue208</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-closed">
+ <compilation-unit name="query-issue236">
+ <output-dir compare="Text">query-issue236</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="quantifiers">
+ <test-case FilePath="quantifiers">
+ <compilation-unit name="any_and_every_01">
+ <output-dir compare="Text">any_and_every_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="quantifiers">
+ <compilation-unit name="anysat_01">
+ <output-dir compare="Text">somesat_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="quantifiers">
+ <compilation-unit name="everysat_01">
+ <output-dir compare="Text">everysat_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="quantifiers">
+ <compilation-unit name="query-ASTERIXDB-1005">
+ <output-dir compare="Text">query-ASTERIXDB-1005</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="quantifiers">
+ <compilation-unit name="query-ASTERIXDB-1674">
+ <output-dir compare="Text">query-ASTERIXDB-1674</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="quantifiers">
+ <compilation-unit name="query-ASTERIXDB-2307">
+ <output-dir compare="Text">query-ASTERIXDB-2307</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="quantifiers">
+ <compilation-unit name="query-ASTERIXDB-2696">
+ <output-dir compare="Text">query-ASTERIXDB-2696</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="quantifiers">
+ <compilation-unit name="query-ASTERIXDB-2947">
+ <output-dir compare="Text">query-ASTERIXDB-2947</output-dir>
+ </compilation-unit>
+ </test-case>
+ <!--
+ <test-case FilePath="quantifiers">
+ <compilation-unit name="everysat_02">
+ <output-dir compare="Text">everysat_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ -->
+ <!--
+ <test-case FilePath="quantifiers">
+ <compilation-unit name="everysat_03">
+ <output-dir compare="Text">everysat_03</output-dir>
+ </compilation-unit>
+ </test-case>
+ -->
+ <test-case FilePath="quantifiers">
+ <compilation-unit name="everysat_04">
+ <output-dir compare="Text">everysat_04</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="quantifiers">
+ <compilation-unit name="somesat_01">
+ <output-dir compare="Text">somesat_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="quantifiers">
+ <compilation-unit name="somesat_02">
+ <output-dir compare="Text">somesat_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <!--
+ <test-case FilePath="quantifiers">
+ <compilation-unit name="somesat_03">
+ <output-dir compare="Text">somesat_03</output-dir>
+ </compilation-unit>
+ </test-case>
+ -->
+ <!--
+ <test-case FilePath="quantifiers">
+ <compilation-unit name="somesat_04">
+ <output-dir compare="Text">somesat_04</output-dir>
+ </compilation-unit>
+ </test-case>
+ -->
+ <!--
+ <test-case FilePath="quantifiers">
+ <compilation-unit name="somesat_05">
+ <output-dir compare="Text">somesat_05</output-dir>
+ </compilation-unit>
+ </test-case>
+ -->
+ <test-case FilePath="quantifiers">
+ <compilation-unit name="somesat_06">
+ <output-dir compare="Text">somesat_06</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="query_index">
+ <test-case FilePath="query_index">
+ <compilation-unit name="q01">
+ <output-dir compare="Text">q01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="query_index">
+ <compilation-unit name="negative">
+ <output-dir compare="Text">negative</output-dir>
+ <expected-error>ASX1105: Operation not supported on primary index ds1</expected-error>
+ <expected-error>ASX1026: The given function expression query-index("test", "ds1", "pk_idx") cannot utilize index</expected-error>
+ <expected-error>ASX1026: The given function expression query-index("test", "ds1", "sample_idx_1_ds1") cannot utilize index</expected-error>
+ <expected-error>ASX1026: The given function expression query-index("test", "ds1", "ds1_array_idx") cannot utilize index</expected-error>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="range-hints">
+ <test-case FilePath="range-hints">
+ <compilation-unit name="order-by">
+ <output-dir compare="Text">order-by</output-dir>
+ </compilation-unit>
+ </test-case>
+ <!-- Fail sporadically <test-case FilePath="range-hints">
+ <compilation-unit name="order-by-exception_01">
+ <output-dir compare="Text">order-by</output-dir>
+ <expected-error>org.json.JSONException: JSONObject['summary'] not found</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="range-hints">
+ <compilation-unit name="order-by-exception_02">
+ <output-dir compare="Text">order-by</output-dir>
+ <expected-error>org.json.JSONException: JSONObject['summary'] not found</expected-error>
+ </compilation-unit>
+ </test-case> -->
+ </test-group>
+ <test-group name="resolution">
+ <test-case FilePath="resolution">
+ <compilation-unit name="conflict-field-dataset">
+ <output-dir compare="Text">conflict-field-dataset</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="resolution">
+ <compilation-unit name="conflict-field-dataset-from">
+ <output-dir compare="Text">conflict-field-dataset-from</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="resolution">
+ <compilation-unit name="conflict-field-dataset-fromterm">
+ <output-dir compare="Text">conflict-field-dataset-from</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="resolution">
+ <compilation-unit name="conflict-field-dataset-join">
+ <output-dir compare="Text">conflict-field-dataset-from</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="resolution">
+ <compilation-unit name="conflict-field-dataset-quantifier">
+ <output-dir compare="Text">conflict-field-dataset-from</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="resolution">
+ <compilation-unit name="conflict-field-dataset-unnest">
+ <output-dir compare="Text">conflict-field-dataset-from</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="resolution">
+ <compilation-unit name="conflict-fields-dataset">
+ <output-dir compare="Text">conflict-field-dataset</output-dir>
+ <expected-error>Cannot resolve ambiguous alias reference for identifier samptable</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="resolution">
+ <compilation-unit name="fullyqualified">
+ <output-dir compare="Text">fullyqualified</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="resolution">
+ <compilation-unit name="fullyqualified2">
+ <output-dir compare="Text">fullyqualified2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="resolution">
+ <compilation-unit name="order_1">
+ <output-dir compare="Text">order_1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="resolution">
+ <compilation-unit name="groupby_rename_with_sugar">
+ <output-dir compare="Text">groupby_rename_with_sugar</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="resolution">
+ <compilation-unit name="function_dataverse">
+ <output-dir compare="Text">function_dataverse</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="resolution">
+ <compilation-unit name="field_accessor_1">
+ <output-dir compare="Text">field_accessor_1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="resolution">
+ <compilation-unit name="field_accessor_2_negative">
+ <output-dir compare="Text">field_accessor_1</output-dir>
+ <expected-error>ASX1074: Cannot resolve ambiguous alias reference for identifier c2 (in line 25, at column 7)</expected-error>
+ <expected-error>ASX1074: Cannot resolve ambiguous alias reference for identifier c2 (in line 25, at column 7)</expected-error>
+ <expected-error>ASX1074: Cannot resolve ambiguous alias reference for identifier c2 (in line 25, at column 7)</expected-error>
+ <expected-error>ASX1074: Cannot resolve ambiguous alias reference for identifier c10 (in line 25, at column 51)</expected-error>
+ <expected-error>ASX1073: Cannot resolve alias reference for undefined identifier c20 (in line 26, at column 8)</expected-error>
+ <expected-error>ASX1073: Cannot resolve alias reference for undefined identifier c1 (in line 25, at column 19)</expected-error>
+ <expected-error>ASX1073: Cannot resolve alias reference for undefined identifier samptable (in line 25, at column 14)</expected-error>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="scan">
+ <test-case FilePath="scan">
+ <compilation-unit name="10">
+ <output-dir compare="Text">10</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="scan">
+ <compilation-unit name="20">
+ <output-dir compare="Text">20</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="scan">
+ <compilation-unit name="issue238_query_1">
+ <output-dir compare="Text">issue238_query_1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="scan">
+ <compilation-unit name="issue238_query_2">
+ <output-dir compare="Text">issue238_query_2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <!-- The syntax this test case tets isn't really invalid
+ <test-case FilePath="scan">
+ <compilation-unit name="invalid-scan-syntax">
+ <output-dir compare="Text">invalid-scan-syntax</output-dir>
+ <expected-error>java.lang.IllegalStateException: no result file</expected-error>
+ </compilation-unit>
+ </test-case>-->
+ <test-case FilePath="scan">
+ <compilation-unit name="30">
+ <output-dir compare="Text">30</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="scan">
+ <compilation-unit name="alltypes_01">
+ <output-dir compare="Text">alltypes_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="scan">
+ <compilation-unit name="alltypes_01">
+ <output-dir compare="Clean-JSON">alltypes_01-cleanjson</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="scan">
+ <compilation-unit name="alltypes_01">
+ <output-dir compare="Lossless-JSON">alltypes_01-losslessjson</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="scan">
+ <compilation-unit name="alltypes_01">
+ <output-dir compare="Lossless-ADM-JSON">alltypes_01-losslessadmjson</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="scan">
+ <compilation-unit name="alltypes_02">
+ <output-dir compare="Text">alltypes_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="scan">
+ <compilation-unit name="numeric_types_01">
+ <output-dir compare="Text">numeric_types_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="scan">
+ <compilation-unit name="spatial_types_01">
+ <output-dir compare="Text">spatial_types_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <!--
+ <test-case FilePath="scan">
+ <compilation-unit name="spatial_types_02">
+ <output-dir compare="Text">spatial_types_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ -->
+ <test-case FilePath="scan">
+ <compilation-unit name="temp_types_01">
+ <output-dir compare="Text">temp_types_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <!--
+ <test-case FilePath="scan">
+ <compilation-unit name="temp_types_02">
+ <output-dir compare="Text">temp_types_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ -->
+ </test-group>
+ <test-group name="select-star">
+ <test-case FilePath="select-star">
+ <compilation-unit name="group_by">
+ <output-dir compare="Text">group_by</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="select-star">
+ <compilation-unit name="from">
+ <output-dir compare="Text">from</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="select-star">
+ <compilation-unit name="join">
+ <output-dir compare="Text">join</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="select-star">
+ <compilation-unit name="mixed">
+ <output-dir compare="Text">mixed</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="select-star">
+ <compilation-unit name="no_star">
+ <output-dir compare="Text">no_star</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="select-star">
+ <compilation-unit name="var_star">
+ <output-dir compare="Text">var_star</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="select-star">
+ <compilation-unit name="var_star_2">
+ <output-dir compare="Text">var_star_2</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="semistructured">
+ <test-case FilePath="semistructured">
+ <compilation-unit name="count-nullable">
+ <output-dir compare="Text">count-nullable</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="semistructured">
+ <compilation-unit name="cust-filter">
+ <output-dir compare="Text">cust-filter</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="semistructured">
+ <compilation-unit name="has-param1">
+ <output-dir compare="Text">has-param1</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="similarity">
+ <test-case FilePath="similarity">
+ <compilation-unit name="edit-distance-check_ints">
+ <output-dir compare="Text">edit-distance-check_ints</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="similarity">
+ <compilation-unit name="edit-distance-check_strings">
+ <output-dir compare="Text">edit-distance-check_strings</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="similarity">
+ <compilation-unit name="edit-distance-check_unicode">
+ <output-dir compare="Text">edit-distance-check_unicode</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="similarity">
+ <compilation-unit name="edit-distance-list-is-filterable">
+ <output-dir compare="Text">edit-distance-list-is-filterable</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="similarity">
+ <compilation-unit name="edit-distance-string-is-filterable">
+ <output-dir compare="Text">edit-distance-string-is-filterable</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="similarity">
+ <compilation-unit name="edit-distance_ints">
+ <output-dir compare="Text">edit-distance_ints</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="similarity">
+ <compilation-unit name="edit-distance_strings">
+ <output-dir compare="Text">edit-distance_strings</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="similarity">
+ <compilation-unit name="fuzzyeq-edit-distance">
+ <output-dir compare="Text">fuzzyeq-edit-distance</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="similarity">
+ <compilation-unit name="fuzzyeq-similarity-jaccard">
+ <output-dir compare="Text">fuzzyeq-similarity-jaccard</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="similarity">
+ <compilation-unit name="prefix-len-jaccard">
+ <output-dir compare="Text">prefix-len-jaccard</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="similarity">
+ <compilation-unit name="similarity-jaccard-check_ints">
+ <output-dir compare="Text">similarity-jaccard-check_ints</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="similarity">
+ <compilation-unit name="similarity-jaccard-check_query">
+ <output-dir compare="Text">similarity-jaccard-check_query</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="similarity">
+ <compilation-unit name="similarity-jaccard-check_strings">
+ <output-dir compare="Text">similarity-jaccard-check_strings</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="similarity">
+ <compilation-unit name="similarity-jaccard-prefix-check">
+ <output-dir compare="Text">similarity-jaccard-prefix-check</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="similarity">
+ <compilation-unit name="similarity-jaccard-prefix">
+ <output-dir compare="Text">similarity-jaccard-prefix</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="similarity">
+ <compilation-unit name="similarity-jaccard-sorted-check_ints">
+ <output-dir compare="Text">similarity-jaccard-sorted-check_ints</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="similarity">
+ <compilation-unit name="similarity-jaccard-sorted-check_query">
+ <output-dir compare="Text">similarity-jaccard-sorted-check_query</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="similarity">
+ <compilation-unit name="similarity-jaccard-sorted-check_strings">
+ <output-dir compare="Text">similarity-jaccard-sorted-check_strings</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="similarity">
+ <compilation-unit name="similarity-jaccard-sorted_ints">
+ <output-dir compare="Text">similarity-jaccard-sorted_ints</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="similarity">
+ <compilation-unit name="similarity-jaccard-sorted_query">
+ <output-dir compare="Text">similarity-jaccard-sorted_query</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="similarity">
+ <compilation-unit name="similarity-jaccard-sorted_strings">
+ <output-dir compare="Text">similarity-jaccard-sorted_strings</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="similarity">
+ <compilation-unit name="similarity-jaccard_ints">
+ <output-dir compare="Text">similarity-jaccard_ints</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="similarity">
+ <compilation-unit name="similarity-jaccard_query">
+ <output-dir compare="Text">similarity-jaccard_query</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="similarity">
+ <compilation-unit name="similarity-jaccard_strings">
+ <output-dir compare="Text">similarity-jaccard_strings</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="similarity">
+ <compilation-unit name="similarity-jaccard-check_strings_issue628">
+ <output-dir compare="Text">similarity-jaccard-check_strings_issue628</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="spatial">
+ <test-case FilePath="spatial">
+ <compilation-unit name="cell-aggregation-with-filtering">
+ <output-dir compare="Text">cell-aggregation-with-filtering</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="spatial">
+ <compilation-unit name="cell-aggregation">
+ <output-dir compare="Text">cell-aggregation</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="spatial">
+ <compilation-unit name="circle_accessor">
+ <output-dir compare="Text">circle_accessor</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="spatial">
+ <compilation-unit name="circle-intersect-circle">
+ <output-dir compare="Text">circle-intersect-circle</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="spatial">
+ <compilation-unit name="create-rtree-index">
+ <output-dir compare="Text">create-rtree-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="spatial">
+ <compilation-unit name="distance-between-points">
+ <output-dir compare="Text">distance-between-points</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="spatial">
+ <compilation-unit name="hints_spatial_partitioning">
+ <output-dir compare="Text">hints_spatial_partitioning</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="spatial">
+ <compilation-unit name="line_accessor">
+ <output-dir compare="Text">line_accessor</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="spatial">
+ <compilation-unit name="line-intersect-circle">
+ <output-dir compare="Text">line-intersect-circle</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="spatial">
+ <compilation-unit name="line-intersect-line">
+ <output-dir compare="Text">line-intersect-line</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="spatial">
+ <compilation-unit name="line-intersect-polygon">
+ <output-dir compare="Text">line-intersect-polygon</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="spatial">
+ <compilation-unit name="line-intersect-rectangle">
+ <output-dir compare="Text">line-intersect-rectangle</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="spatial">
+ <compilation-unit name="point_accessor">
+ <output-dir compare="Text">point_accessor</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="spatial">
+ <compilation-unit name="point-equals-point">
+ <output-dir compare="Text">point-equals-point</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="spatial">
+ <compilation-unit name="point-in-circle">
+ <output-dir compare="Text">point-in-circle</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="spatial">
+ <compilation-unit name="point-in-polygon">
+ <output-dir compare="Text">point-in-polygon</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="spatial">
+ <compilation-unit name="point-in-rectangle">
+ <output-dir compare="Text">point-in-rectangle</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="spatial">
+ <compilation-unit name="point-on-line">
+ <output-dir compare="Text">point-on-line</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="spatial">
+ <compilation-unit name="polygon_accessor">
+ <output-dir compare="Text">polygon_accessor</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="spatial">
+ <compilation-unit name="polygon-intersect-circle">
+ <output-dir compare="Text">polygon-intersect-circle</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="spatial">
+ <compilation-unit name="polygon-intersect-polygon">
+ <output-dir compare="Text">polygon-intersect-polygon</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="spatial">
+ <compilation-unit name="polygon-intersect-rectangle">
+ <output-dir compare="Text">polygon-intersect-rectangle</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="spatial">
+ <compilation-unit name="rectangle_accessor">
+ <output-dir compare="Text">rectangle_accessor</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="spatial">
+ <compilation-unit name="rectangle-intersect-circle">
+ <output-dir compare="Text">rectangle-intersect-circle</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="spatial">
+ <compilation-unit name="rectangle-intersect-rectangle">
+ <output-dir compare="Text">rectangle-intersect-rectangle</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="spatial">
+ <compilation-unit name="spatial-area">
+ <output-dir compare="Text">spatial-area</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="spatial">
+ <compilation-unit name="spatial-large-data">
+ <output-dir compare="Text">spatial-large-data</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="spatial">
+ <compilation-unit name="spatial_join_dynamic_partitioning">
+ <output-dir compare="Text">spatial_join_dynamic_partitioning</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="spatial">
+ <compilation-unit name="spatial_join_static_partitioning">
+ <output-dir compare="Text">spatial_join_static_partitioning</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="spatial">
+ <compilation-unit name="spatial_left_outer_join_st_intersects">
+ <output-dir compare="Text">spatial_left_outer_join_st_intersects</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="spatial">
+ <compilation-unit name="spatial_join_projection_check">
+ <output-dir compare="Text">spatial_join_projection_check</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="sql-compat">
+ <test-case FilePath="sql-compat">
+ <compilation-unit name="in_non_list_01">
+ <output-dir compare="Text">in_non_list_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="sql-compat">
+ <compilation-unit name="outer_join_01">
+ <output-dir compare="Text">outer_join_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="sql-compat">
+ <compilation-unit name="outer_unnest_01">
+ <output-dir compare="Text">outer_unnest_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="sql-compat">
+ <compilation-unit name="select_star_01">
+ <output-dir compare="Text">select_star_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="sql-compat">
+ <compilation-unit name="select_star_02_negative">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>ASX1168: Ambiguous projection in SELECT clause (in line 32, at column 8)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="sql-compat">
+ <compilation-unit name="subquery_coercion_01_scalar">
+ <output-dir compare="Text">subquery_coercion_01_scalar</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="sql-compat">
+ <compilation-unit name="subquery_coercion_02_scalar_negative">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>ASX1169: Unable to do subquery coercion. Subquery returns more than one field (in line 32, at column 29)</expected-error>
+ <expected-error>ASX1169: Unable to do subquery coercion. Unsupported projection kind (in line 32, at column 20)</expected-error>
+ <expected-error>ASX1169: Unable to do subquery coercion. Both SELECT and SELECT VALUE are present (in line 35, at column 4)</expected-error>
+ <expected-error>ASX1169: Unable to do subquery coercion. (in line 34, at column 3)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="sql-compat">
+ <compilation-unit name="subquery_coercion_03_cmp">
+ <output-dir compare="Text">subquery_coercion_03_cmp</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="sql-compat">
+ <compilation-unit name="subquery_coercion_04_cmp_negative">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>ASX1169: Unable to do subquery coercion. Subquery returns more than one field (in line 35, at column 26)</expected-error>
+ <expected-error>ASX1169: Unable to do subquery coercion. Subquery returns more than one field (in line 35, at column 21)</expected-error>
+ <expected-error>ASX1169: Unable to do subquery coercion. Both SELECT and SELECT VALUE are present (in line 37, at column 3)</expected-error>
+ <expected-error>ASX1169: Unable to do subquery coercion. Both SELECT and SELECT VALUE are present (in line 38, at column 3)</expected-error>
+ <expected-error>ASX1169: Unable to do subquery coercion. (in line 36, at column 3)</expected-error>
+ <expected-error>ASX1169: Unable to do subquery coercion. Unsupported projection kind (in line 35, at column 20)</expected-error>
+ <expected-error>ASX1169: Unable to do subquery coercion. Unsupported projection kind (in line 36, at column 11)</expected-error>
+ <expected-error>ASX1169: Unable to do subquery coercion. Unsupported projection kind (in line 37, at column 11)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="sql-compat">
+ <compilation-unit name="subquery_coercion_05_in">
+ <output-dir compare="Text">subquery_coercion_05_in</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="sql-compat">
+ <compilation-unit name="subquery_coercion_06_in_negative">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>ASX1169: Unable to do subquery coercion. Subquery returns more than one field (in line 36, at column 27)</expected-error>
+ <expected-error>ASX1169: Unable to do subquery coercion. Subquery returns more than one field (in line 37, at column 17)</expected-error>
+ <expected-error>ASX1169: Unable to do subquery coercion. Both SELECT and SELECT VALUE are present (in line 38, at column 3)</expected-error>
+ <expected-error>ASX1169: Unable to do subquery coercion. Both SELECT and SELECT VALUE are present (in line 39, at column 3)</expected-error>
+ <expected-error>ASX1169: Unable to do subquery coercion. (in line 37, at column 3)</expected-error>
+ <expected-error>ASX1169: Unable to do subquery coercion. Unsupported projection kind (in line 36, at column 21)</expected-error>
+ <expected-error>ASX1169: Unable to do subquery coercion. Unsupported projection kind (in line 37, at column 25)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="sql-compat">
+ <compilation-unit name="subquery_coercion_07_from">
+ <output-dir compare="Text">subquery_coercion_07_from</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="sql-compat">
+ <compilation-unit name="subquery_coercion_08_misc">
+ <output-dir compare="Text">subquery_coercion_08_misc</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="sql-compat">
+ <compilation-unit name="union_all_01">
+ <output-dir compare="Text">union_all_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="sql-compat">
+ <compilation-unit name="union_all_02_negative">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>ASX1171: Unable to process UNION clause. Both SELECT and SELECT VALUE are present (in line 29, at column 1)</expected-error>
+ <expected-error>ASX1171: Unable to process UNION clause. Unequal number of input fields (in line 29, at column 1)</expected-error>
+ <expected-error>ASX1171: Unable to process UNION clause. Unsupported projection kind (in line 29, at column 8)</expected-error>
+ <expected-error>ASX1171: Unable to process UNION clause. Unsupported projection kind (in line 30, at column 8)</expected-error>
+ <expected-error>ASX1171: Unable to process UNION clause. Unsupported projection kind (in line 31, at column 8)</expected-error>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="statement-params">
+ <test-case FilePath="statement-params">
+ <compilation-unit name="index_01">
+ <output-dir compare="Text">index_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="statement-params">
+ <compilation-unit name="mixed_01">
+ <output-dir compare="Text">mixed_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="statement-params">
+ <compilation-unit name="named_01">
+ <output-dir compare="Text">named_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="statement-params">
+ <compilation-unit name="named_02">
+ <output-dir compare="Text">named_01</output-dir>
+ <expected-error>ASX1086: No value for parameter: $p2</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="statement-params">
+ <compilation-unit name="named_03">
+ <output-dir compare="Text">named_03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="statement-params">
+ <compilation-unit name="named_04">
+ <output-dir compare="Text">named_04</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="statement-params">
+ <compilation-unit name="positional_01">
+ <output-dir compare="Text">positional_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="statement-params">
+ <compilation-unit name="positional_02">
+ <output-dir compare="Text">positional_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="statement-params">
+ <compilation-unit name="positional_03">
+ <output-dir compare="Text">positional_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="statement-params">
+ <compilation-unit name="positional_04">
+ <output-dir compare="Text">positional_02</output-dir>
+ <expected-error>ASX1086: No value for parameter: $2</expected-error>
+ <expected-error>ASX1086: No value for parameter: $3</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="statement-params">
+ <compilation-unit name="positional_05">
+ <output-dir compare="Text">positional_05</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="statement-params">
+ <compilation-unit name="query-ASTERIXDB-2413">
+ <output-dir compare="Text">query-ASTERIXDB-2413</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="statement-params">
+ <compilation-unit name="query-ASTERIXDB-3116">
+ <output-dir compare="Text">query-ASTERIXDB-3116</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="string">
+ <test-case FilePath="string">
+ <compilation-unit name="codepoint-to-string1">
+ <output-dir compare="Text">codepoint-to-string1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="codepoint-to-string2">
+ <output-dir compare="Text">codepoint-to-string2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="concat/001">
+ <output-dir compare="Text">concat/001</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="concat/002">
+ <output-dir compare="Text">concat/002</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string" check-warnings="true">
+ <compilation-unit name="concat/003">
+ <output-dir compare="Text">concat/003</output-dir>
+ <expected-warn>Type mismatch: function string-concat expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 26, at column 1)</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="concat/004">
+ <output-dir compare="Text">concat/004</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="concat/005">
+ <output-dir compare="Text">concat/005</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="concat/006">
+ <output-dir compare="Text">concat/006</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="concat/007">
+ <output-dir compare="Text">concat/007</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="concat/008">
+ <output-dir compare="Text">concat/002</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="concat/concat_pipe">
+ <output-dir compare="Text">concat/002</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="concat/concat_pipe_multi">
+ <output-dir compare="Text">concat/concat_pipe_multi</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="constructor">
+ <output-dir compare="Text">constructor</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="contains_01">
+ <output-dir compare="Text">contains_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="cpttostr01">
+ <output-dir compare="Text">cpttostr01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="cpttostr02">
+ <output-dir compare="Text">cpttostr02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="cpttostr04">
+ <output-dir compare="Text">cpttostr04</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="ends-with1">
+ <output-dir compare="Text">ends-with1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="ends-with2">
+ <output-dir compare="Text">ends-with2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="ends-with3">
+ <output-dir compare="Text">ends-with3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="ends-with4">
+ <output-dir compare="Text">ends-with4</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="ends-with5">
+ <output-dir compare="Text">ends-with5</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="escapes01">
+ <output-dir compare="Text">escapes01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="escapes02">
+ <output-dir compare="Text">escapes02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="initcap">
+ <output-dir compare="Text">initcap</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="join/000">
+ <output-dir compare="Text">join/000</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string" check-warnings="true">
+ <compilation-unit name="join/001">
+ <output-dir compare="Text">join/001</output-dir>
+ <expected-warn>Type mismatch: function string-join expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 24, at column 15)</expected-warn>
+ <expected-warn>Type mismatch: function string-join expects its 1st input parameter to be of type array, but the actual input type is bigint (in line 24, at column 15)</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="join/002">
+ <output-dir compare="Text">join/002</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string" check-warnings="true">
+ <compilation-unit name="join/003">
+ <output-dir compare="Text">join/003</output-dir>
+ <expected-warn>Type mismatch: function string-join expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 23, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function string-join expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 30, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function string-join expects its 1st input parameter to be of type array, but the actual input type is bigint (in line 33, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function string-join expects its 1st input parameter to be of type array, but the actual input type is bigint (in line 26, at column 1)</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string" check-warnings="true">
+ <compilation-unit name="join/004">
+ <output-dir compare="Text">join/004</output-dir>
+ <expected-warn>Type mismatch: function string-join expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 26, at column 1)</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string" check-warnings="true">
+ <compilation-unit name="join/005">
+ <output-dir compare="Text">join/005</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="length_01">
+ <output-dir compare="Text">length_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="length_02">
+ <output-dir compare="Text">length_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="length_multi_code_point_01">
+ <output-dir compare="Text">length_multi_code_point_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="like_01">
+ <output-dir compare="Text">like_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="like_02">
+ <output-dir compare="Text">like_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="like_03_negative">
+ <output-dir compare="Text">like_03_negative</output-dir>
+ <expected-error>Invalid pattern '__\c' for LIKE (in line 21, at column 11)</expected-error>
+ <expected-error>Invalid pattern '%\' for LIKE (in line 21, at column 18)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="like_null">
+ <output-dir compare="Text">like_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="lowercase">
+ <output-dir compare="Text">lowercase</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="matches02">
+ <output-dir compare="Text">matches02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="matches03">
+ <output-dir compare="Text">matches03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="matches04">
+ <output-dir compare="Text">matches04</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="matches05">
+ <output-dir compare="Text">matches05</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="matches06">
+ <output-dir compare="Text">matches06</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="matches1">
+ <output-dir compare="Text">matches1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="matches11">
+ <output-dir compare="Text">matches11</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="matches2">
+ <output-dir compare="Text">matches2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="matches21">
+ <output-dir compare="Text">matches21</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="matches22">
+ <output-dir compare="Text">matches22</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="matches23">
+ <output-dir compare="Text">matches23</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="matches3">
+ <output-dir compare="Text">matches3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="matchesnull">
+ <output-dir compare="Text">matchesnull</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="position/offset0/position">
+ <output-dir compare="Text">position/offset0/position</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="position/offset0/pos">
+ <output-dir compare="Text">position/offset0/pos</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="position/offset0/position0">
+ <output-dir compare="Text">position/offset0/position0</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="position/offset0/pos0">
+ <output-dir compare="Text">position/offset0/pos0</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="position/offset0/pos0_multi_code_point">
+ <output-dir compare="Text">position/offset0/pos0_multi_code_point</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="position/offset1/position1">
+ <output-dir compare="Text">position/offset1/position1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="position/offset1/pos1">
+ <output-dir compare="Text">position/offset1/pos1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="position/offset1/pos1_multi_code_point">
+ <output-dir compare="Text">position/offset1/pos1_multi_code_point</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="regexp_contains/regexp_contains">
+ <output-dir compare="Text">regexp_contains/regexp_contains</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="regexp_contains/regex_contains">
+ <output-dir compare="Text">regexp_contains/regex_contains</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="regexp_contains/contains_regexp">
+ <output-dir compare="Text">regexp_contains/contains_regexp</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="regexp_contains/contains_regex">
+ <output-dir compare="Text">regexp_contains/contains_regex</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="regexp_contains_with_flag/regexp_contains_with_flag">
+ <output-dir compare="Text">regexp_contains_with_flag/regexp_contains_with_flag</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="regexp_contains_with_flag/regex_contains_with_flag">
+ <output-dir compare="Text">regexp_contains_with_flag/regex_contains_with_flag</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="regexp_contains_with_flag/contains_regexp_with_flag">
+ <output-dir compare="Text">regexp_contains_with_flag/contains_regexp_with_flag</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="regexp_contains_with_flag/contains_regex_with_flag">
+ <output-dir compare="Text">regexp_contains_with_flag/contains_regex_with_flag</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="regexp_like/regexp_like">
+ <output-dir compare="Text">regexp_like/regexp_like</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="regexp_like/regex_like">
+ <output-dir compare="Text">regexp_like/regex_like</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="regexp_like_with_flag/regexp_like_with_flag">
+ <output-dir compare="Text">regexp_like_with_flag/regexp_like_with_flag</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="regexp_like_with_flag/regex_like_with_flag">
+ <output-dir compare="Text">regexp_like_with_flag/regex_like_with_flag</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="regexp_matches/001">
+ <output-dir compare="Text">regexp_matches/001</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="regexp_matches/002">
+ <output-dir compare="Text">regexp_matches/002</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="regexp_matches/003">
+ <output-dir compare="Text">regexp_matches/003</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="regexp_matches/004">
+ <output-dir compare="Text">regexp_matches/004</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="regexp_position/offset0/regexp_position">
+ <output-dir compare="Text">regexp_position/offset0/regexp_position</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="regexp_position/offset0/regexp_pos">
+ <output-dir compare="Text">regexp_position/offset0/regexp_pos</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="regexp_position/offset0/regexp_position0">
+ <output-dir compare="Text">regexp_position/offset0/regexp_position0</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="regexp_position/offset0/regexp_pos0">
+ <output-dir compare="Text">regexp_position/offset0/regexp_pos0</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="regexp_position/offset0/regex_position">
+ <output-dir compare="Text">regexp_position/offset0/regex_position</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="regexp_position/offset0/regex_pos">
+ <output-dir compare="Text">regexp_position/offset0/regex_pos</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="regexp_position/offset0/regex_position0">
+ <output-dir compare="Text">regexp_position/offset0/regex_position0</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="regexp_position/offset0/regex_pos0">
+ <output-dir compare="Text">regexp_position/offset0/regex_pos0</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="regexp_position/offset1/regexp_position1">
+ <output-dir compare="Text">regexp_position/offset1/regexp_position1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="regexp_position/offset1/regexp_pos1">
+ <output-dir compare="Text">regexp_position/offset1/regexp_pos1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="regexp_position/offset1/regex_position1">
+ <output-dir compare="Text">regexp_position/offset1/regex_position1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="regexp_position/offset1/regex_pos1">
+ <output-dir compare="Text">regexp_position/offset1/regex_pos1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="regexp_position_with_flag/offset0/regexp_position_with_flag">
+ <output-dir compare="Text">regexp_position_with_flag/offset0/regexp_position_with_flag</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="regexp_position_with_flag/offset0/regexp_pos_with_flag">
+ <output-dir compare="Text">regexp_position_with_flag/offset0/regexp_pos_with_flag</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="regexp_position_with_flag/offset0/regexp_position0_with_flag">
+ <output-dir compare="Text">regexp_position_with_flag/offset0/regexp_position0_with_flag</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="regexp_position_with_flag/offset0/regexp_pos0_with_flag">
+ <output-dir compare="Text">regexp_position_with_flag/offset0/regexp_pos0_with_flag</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="regexp_position_with_flag/offset0/regex_position_with_flag">
+ <output-dir compare="Text">regexp_position_with_flag/offset0/regex_position_with_flag</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="regexp_position_with_flag/offset0/regex_pos_with_flag">
+ <output-dir compare="Text">regexp_position_with_flag/offset0/regex_pos_with_flag</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="substr-ASTERIXDB-2949">
+ <output-dir compare="Text">substr-ASTERIXDB-2949</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="regexp_position_with_flag/offset0/regex_position0_with_flag">
+ <output-dir compare="Text">regexp_position_with_flag/offset0/regex_position0_with_flag</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="regexp_position_with_flag/offset0/regex_pos0_with_flag">
+ <output-dir compare="Text">regexp_position_with_flag/offset0/regex_pos0_with_flag</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="regexp_position_with_flag/offset1/regexp_position1_with_flag">
+ <output-dir compare="Text">regexp_position_with_flag/offset1/regexp_position1_with_flag</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="regexp_position_with_flag/offset1/regexp_pos1_with_flag">
+ <output-dir compare="Text">regexp_position_with_flag/offset1/regexp_pos1_with_flag</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="regexp_position_with_flag/offset1/regex_position1_with_flag">
+ <output-dir compare="Text">regexp_position_with_flag/offset1/regex_position1_with_flag</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="regexp_position_with_flag/offset1/regex_pos1_with_flag">
+ <output-dir compare="Text">regexp_position_with_flag/offset1/regex_pos1_with_flag</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="regexp_split/001">
+ <output-dir compare="Text">regexp_split/001</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="regexp_split/002">
+ <output-dir compare="Text">regexp_split/002</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="regexp_split/003">
+ <output-dir compare="Text">regexp_split/003</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="regexp_split/004">
+ <output-dir compare="Text">regexp_split/004</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="repeat">
+ <output-dir compare="Text">repeat</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="replace">
+ <output-dir compare="Text">replace</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="replace_with_limit">
+ <output-dir compare="Text">replace_with_limit</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="regexp_replace/regexp_replace">
+ <output-dir compare="Text">regexp_replace/regexp_replace</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="regexp_replace/regex_replace">
+ <output-dir compare="Text">regexp_replace/regex_replace</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="regexp_replace_with_flag/regexp_replace_with_flag">
+ <output-dir compare="Text">regexp_replace_with_flag/regexp_replace_with_flag</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="regexp_replace_with_flag/regex_replace_with_flag">
+ <output-dir compare="Text">regexp_replace_with_flag/regex_replace_with_flag</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="reverse">
+ <output-dir compare="Text">reverse</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="split">
+ <output-dir compare="Text">split</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="starts-with1">
+ <output-dir compare="Text">starts-with1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="starts-with2">
+ <output-dir compare="Text">starts-with2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="starts-with3">
+ <output-dir compare="Text">starts-with3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="starts-with4">
+ <output-dir compare="Text">starts-with4</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="starts-with5">
+ <output-dir compare="Text">starts-with5</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="string-equal1">
+ <output-dir compare="Text">string-equal1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="string-equal2">
+ <output-dir compare="Text">string-equal2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="string-equal3">
+ <output-dir compare="Text">string-equal3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="string-equal4">
+ <output-dir compare="Text">string-equal4</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="string-literal1">
+ <output-dir compare="Text">string-literal1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="string-to-codepoint">
+ <output-dir compare="Text">string-to-codepoint</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="string-to-codepoint1">
+ <output-dir compare="Text">string-to-codepoint1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="string-to-codepoint2">
+ <output-dir compare="Text">string-to-codepoint2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="string_to_codepoint_multi_codepoints_01">
+ <output-dir compare="Text">string_to_codepoint_multi_codepoints_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="strlen02">
+ <output-dir compare="Text">strlen02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="strlen03">
+ <output-dir compare="Text">strlen03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="strtocpt01">
+ <output-dir compare="Text">strtocpt01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="strtocpt02">
+ <output-dir compare="Text">strtocpt02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="strtocpt03">
+ <output-dir compare="Text">strtocpt03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="substr01/offset0/substring">
+ <output-dir compare="Text">substr01/offset0/substring</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="substr01/offset0/substr">
+ <output-dir compare="Text">substr01/offset0/substr</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="substr01/offset0/substring0">
+ <output-dir compare="Text">substr01/offset0/substring0</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="substr01/offset0/substr0">
+ <output-dir compare="Text">substr01/offset0/substr0</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="substr01/offset1/substring1">
+ <output-dir compare="Text">substr01/offset1/substring1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="substr01/offset1/substr1">
+ <output-dir compare="Text">substr01/offset1/substr1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="substr04">
+ <output-dir compare="Text">substr04</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="substr05">
+ <output-dir compare="Text">substr05</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="substr06">
+ <output-dir compare="Text">substr06</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="substring_multi_codepoint_01">
+ <output-dir compare="Text">substring_multi_codepoint_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="substring-after-1">
+ <output-dir compare="Text">substring-after-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="substring-after-2">
+ <output-dir compare="Text">substring-after-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="substring-after-3">
+ <output-dir compare="Text">substring-after-3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="substring-after-4">
+ <output-dir compare="Text">substring-after-4</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="substring-before-1">
+ <output-dir compare="Text">substring-before-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="substring-before-2">
+ <output-dir compare="Text">substring-before-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="substring-before-3">
+ <output-dir compare="Text">substring-before-3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="substring2-1/offset0/substring">
+ <output-dir compare="Text">substring2-1/offset0/substring</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="substring2-1/offset0/substr">
+ <output-dir compare="Text">substring2-1/offset0/substr</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="substring2-1/offset0/substring0">
+ <output-dir compare="Text">substring2-1/offset0/substring0</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="substring2-1/offset0/substr0">
+ <output-dir compare="Text">substring2-1/offset0/substr0</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="substring2-1/offset1/substring1">
+ <output-dir compare="Text">substring2-1/offset1/substring1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="substring2-1/offset1/substr1">
+ <output-dir compare="Text">substring2-1/offset1/substr1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="substring2-2">
+ <output-dir compare="Text">substring2-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="substring2-3">
+ <output-dir compare="Text">substring2-3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="substring2-4">
+ <output-dir compare="Text">substring2-4</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="substring_01">
+ <output-dir compare="Text">substring_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="title">
+ <output-dir compare="Text">initcap</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="toLowerCase02">
+ <output-dir compare="Text">toLowerCase02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="toLowerCase03">
+ <output-dir compare="Text">toLowerCase03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="toLowerCase04">
+ <output-dir compare="Text">toLowerCase04</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="trim">
+ <output-dir compare="Text">trim</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="ltrim">
+ <output-dir compare="Text">ltrim</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="rtrim">
+ <output-dir compare="Text">rtrim</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="uppercase">
+ <output-dir compare="Text">uppercase</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="varlen-encoding">
+ <output-dir compare="Text">varlen-encoding</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="utf8">
+ <output-dir compare="Text">utf8</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="query-ASTERIXDB-1190">
+ <output-dir compare="Text">query-ASTERIXDB-1190</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="string-equal-public">
+ <output-dir compare="Text">string-equal-public</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="substring-after-5">
+ <output-dir compare="Text">substring-after-5</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
+ <compilation-unit name="substring-after-6">
+ <output-dir compare="Text">substring-after-5</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string" check-warnings="true">
+ <compilation-unit name="invalid-unicode">
+ <output-dir compare="Text">invalid-unicode</output-dir>
+ <expected-warn>Function 'string-length' failed to evaluate because: Decoding error - got a low surrogate without a leading high surrogate</expected-warn>
+ <expected-warn>Function 'string-to-codepoint' failed to evaluate because: Decoding error - got a low surrogate without a leading high surrogate</expected-warn>
+ <expected-warn>Function 'trim' failed to evaluate because: Decoding error - got a low surrogate without a leading high surrogate</expected-warn>
+ <expected-warn>Function 'trim' failed to evaluate because: Decoding error - got a low surrogate without a leading high surrogate</expected-warn>
+ <expected-warn>Function 'rtrim' failed to evaluate because: Decoding error - got a low surrogate without a leading high surrogate</expected-warn>
+ <expected-warn>Function 'rtrim' failed to evaluate because: Decoding error - got a low surrogate without a leading high surrogate</expected-warn>
+ <expected-warn>Function 'ltrim' failed to evaluate because: Decoding error - got a low surrogate without a leading high surrogate</expected-warn>
+ <expected-warn>Function 'ltrim' failed to evaluate because: Decoding error - got a low surrogate without a leading high surrogate</expected-warn>
+ <expected-warn>Function 'reverse' failed to evaluate because: Decoding error - got a low surrogate without a leading high surrogate</expected-warn>
+ <expected-warn>Function 'position' failed to evaluate because: Decoding error - got a low surrogate without a leading high surrogate</expected-warn>
+ <expected-warn>Function 'position1' failed to evaluate because: Decoding error - got a low surrogate without a leading high surrogate</expected-warn>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="subquery">
+ <test-case FilePath="subquery">
+ <compilation-unit name="select_element">
+ <output-dir compare="Text">select_element</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="subquery">
+ <compilation-unit name="aggregate_join">
+ <output-dir compare="Text">aggregate_join</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="subquery">
+ <compilation-unit name="aggregate_join_external">
+ <output-dir compare="Text">aggregate_join_external</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="subquery">
+ <compilation-unit name="gby_inline">
+ <output-dir compare="Text">gby_inline</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="subquery">
+ <compilation-unit name="query-ASTERIXDB-1168">
+ <output-dir compare="Text">query-ASTERIXDB-1168</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="subquery">
+ <compilation-unit name="decorrelate_with_unique_id">
+ <output-dir compare="Text">decorrelate_with_unique_id</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="subquery">
+ <compilation-unit name="exists">
+ <output-dir compare="Text">exists</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="subquery">
+ <compilation-unit name="in">
+ <output-dir compare="Text">in</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="subquery">
+ <compilation-unit name="in_as_or">
+ <output-dir compare="Text">in_as_or</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="subquery">
+ <compilation-unit name="in_correlated">
+ <output-dir compare="Text">in</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="subquery">
+ <compilation-unit name="in_let">
+ <output-dir compare="Text">in_let</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="subquery">
+ <compilation-unit name="not_exists">
+ <output-dir compare="Text">not_exists</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="subquery">
+ <compilation-unit name="not_in">
+ <output-dir compare="Text">not_in</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="subquery">
+ <compilation-unit name="alias_negative">
+ <output-dir compare="Text">alias_negative</output-dir>
+ <expected-error>Need an alias for the enclosed expression</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="subquery">
+ <compilation-unit name="relational_division">
+ <output-dir compare="Text">relational_division</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="subquery">
+ <compilation-unit name="relational_division2">
+ <output-dir compare="Text">relational_division</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="subquery">
+ <compilation-unit name="relational_division3">
+ <output-dir compare="Text">relational_division</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="subquery">
+ <compilation-unit name="non_unary_subplan_01">
+ <output-dir compare="Text">non_unary_subplan_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="subquery">
+ <compilation-unit name="query-ASTERIXDB-1571">
+ <output-dir compare="Text">query-ASTERIXDB-1571</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="subquery">
+ <compilation-unit name="query-ASTERIXDB-1571-2">
+ <output-dir compare="Text">query-ASTERIXDB-1571</output-dir>
+ <expected-error>Need an alias for the enclosed expression</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="subquery">
+ <compilation-unit name="query-ASTERIXDB-1571-3">
+ <output-dir compare="Text">query-ASTERIXDB-1571-3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="subquery">
+ <compilation-unit name="query-ASTERIXDB-1571-4">
+ <output-dir compare="Text">query-ASTERIXDB-1571-4</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="subquery">
+ <compilation-unit name="query-ASTERIXDB-1572">
+ <output-dir compare="Text">query-ASTERIXDB-1572</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="subquery">
+ <compilation-unit name="query-ASTERIXDB-1574">
+ <output-dir compare="Text">query-ASTERIXDB-1574</output-dir>
+ <expected-error>ASX1091: Type mismatch: expected value of type multiset or array, but got the value of type object (in line 25, at column 41)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="subquery">
+ <compilation-unit name="query-ASTERIXDB-1574-2">
+ <output-dir compare="Text">query-ASTERIXDB-1574</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="subquery">
+ <compilation-unit name="query-ASTERIXDB-1574-3">
+ <output-dir compare="Text">query-ASTERIXDB-1574</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="subquery">
+ <compilation-unit name="query-ASTERIXDB-1597">
+ <output-dir compare="Text">query-ASTERIXDB-1597</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="subquery">
+ <compilation-unit name="query-ASTERIXDB-1674">
+ <output-dir compare="Text">query-ASTERIXDB-1674</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="subquery">
+ <compilation-unit name="query-ASTERIXDB-2815">
+ <output-dir compare="Text">query-ASTERIXDB-2815</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="subquery">
+ <compilation-unit name="query-ASTERIXDB-2845">
+ <output-dir compare="Text">query-ASTERIXDB-2845</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="subquery">
+ <compilation-unit name="query-ASTERIXDB-3006">
+ <output-dir compare="Text">query-ASTERIXDB-3006</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="subset-collection">
+ <test-case FilePath="subset-collection">
+ <compilation-unit name="01">
+ <output-dir compare="Text">01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="subset-collection">
+ <compilation-unit name="02">
+ <output-dir compare="Text">02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="subset-collection">
+ <compilation-unit name="03">
+ <output-dir compare="Text">03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="subset-collection">
+ <compilation-unit name="05">
+ <output-dir compare="Text">05</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="subset-collection">
+ <compilation-unit name="06">
+ <output-dir compare="Text">06</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="subset-collection">
+ <compilation-unit name="07">
+ <output-dir compare="Text">07</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="synonym">
+ <test-case FilePath="synonym">
+ <compilation-unit name="synonym-01">
+ <output-dir compare="Text">synonym-01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="synonym">
+ <compilation-unit name="synonym-02-negative">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>ASX1063: Cannot find dataverse with name UNKNOWN_DATAVERSE</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="synonym">
+ <compilation-unit name="synonym-03">
+ <output-dir compare="Text">synonym-03</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="tokenizers">
+ <test-case FilePath="tokenizers">
+ <compilation-unit name="counthashed-gram-tokens_01">
+ <output-dir compare="Text">counthashed-gram-tokens_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tokenizers">
+ <compilation-unit name="counthashed-gram-tokens_02">
+ <output-dir compare="Text">counthashed-gram-tokens_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tokenizers">
+ <compilation-unit name="counthashed-word-tokens_01">
+ <output-dir compare="Text">counthashed-word-tokens_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tokenizers">
+ <compilation-unit name="gram-tokens_01">
+ <output-dir compare="Text">gram-tokens_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tokenizers">
+ <compilation-unit name="gram-tokens_02">
+ <output-dir compare="Text">gram-tokens_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tokenizers">
+ <compilation-unit name="hashed-gram-tokens_01">
+ <output-dir compare="Text">hashed-gram-tokens_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tokenizers">
+ <compilation-unit name="hashed-gram-tokens_02">
+ <output-dir compare="Text">hashed-gram-tokens_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tokenizers">
+ <compilation-unit name="hashed-word-tokens_01">
+ <output-dir compare="Text">hashed-word-tokens_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tokenizers">
+ <compilation-unit name="word-tokens_01">
+ <output-dir compare="Text">word-tokens_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tokenizers">
+ <compilation-unit name="word-tokens_02">
+ <output-dir compare="Text">word-tokens_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="tpcds">
+ <test-case FilePath="tpcds">
+ <compilation-unit name="query-ASTERIXDB-1580">
+ <output-dir compare="Text">query-ASTERIXDB-1580</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="query-ASTERIXDB-1581">
+ <output-dir compare="Text">query-ASTERIXDB-1581</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="query-ASTERIXDB-1581-2">
+ <output-dir compare="Text">query-ASTERIXDB-1581-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="query-ASTERIXDB-1581-3">
+ <output-dir compare="Text">query-ASTERIXDB-1581-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="query-ASTERIXDB-1581-4">
+ <output-dir compare="Text">query-ASTERIXDB-1581-4</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="query-ASTERIXDB-1581-5">
+ <output-dir compare="Text">query-ASTERIXDB-1581-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="query-ASTERIXDB-1581-6">
+ <output-dir compare="Text">query-ASTERIXDB-1581-6</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="query-ASTERIXDB-1581-correlated">
+ <output-dir compare="Text">query-ASTERIXDB-1581-correlated</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="query-ASTERIXDB-1581-correlated-2">
+ <output-dir compare="Text">query-ASTERIXDB-1581-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="query-ASTERIXDB-1591">
+ <output-dir compare="Text">query-ASTERIXDB-1591</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="query-ASTERIXDB-1596">
+ <output-dir compare="Text">query-ASTERIXDB-1596</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="query-ASTERIXDB-1602">
+ <output-dir compare="Text">query-ASTERIXDB-1602</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q01">
+ <output-dir compare="Text">q01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q03">
+ <output-dir compare="Text">q03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q05">
+ <output-dir compare="Text">q05</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q07">
+ <output-dir compare="Text">q07</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q09">
+ <output-dir compare="Text">q09</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q12">
+ <output-dir compare="Text">q12</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q15">
+ <output-dir compare="Text">q15</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q18">
+ <output-dir compare="Text">q18</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q19">
+ <output-dir compare="Text">q19</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q20">
+ <output-dir compare="Text">q20</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q21">
+ <output-dir compare="Text">q21</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q22">
+ <output-dir compare="Text">q22</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q24a">
+ <output-dir compare="Text">q24a</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q24b">
+ <output-dir compare="Text">q24b</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q25">
+ <output-dir compare="Text">q25</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q26">
+ <output-dir compare="Text">q26</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q27">
+ <output-dir compare="Text">q27</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q29">
+ <output-dir compare="Text">q29</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q30">
+ <output-dir compare="Text">q30</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q31">
+ <output-dir compare="Text">q31</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q34">
+ <output-dir compare="Text">q34</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q36">
+ <output-dir compare="Text">q36</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q37">
+ <output-dir compare="Text">q37</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q41">
+ <output-dir compare="Text">q41</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q42">
+ <output-dir compare="Text">q42</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q43">
+ <output-dir compare="Text">q43</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q45">
+ <output-dir compare="Text">q45</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q46">
+ <output-dir compare="Text">q46</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q49">
+ <output-dir compare="Text">q49</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q50">
+ <output-dir compare="Text">q50</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q52">
+ <output-dir compare="Text">q52</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q55">
+ <output-dir compare="Text">q55</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q57">
+ <output-dir compare="Text">q57</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q59">
+ <output-dir compare="Text">q59</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q62">
+ <output-dir compare="Text">q62</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q63">
+ <output-dir compare="Text">q63</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q67">
+ <output-dir compare="Text">q67</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q68">
+ <output-dir compare="Text">q68</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q70">
+ <output-dir compare="Text">q70</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q73">
+ <output-dir compare="Text">q73</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q77">
+ <output-dir compare="Text">q77</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q79">
+ <output-dir compare="Text">q79</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q80">
+ <output-dir compare="Text">q80</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q81">
+ <output-dir compare="Text">q81</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q82">
+ <output-dir compare="Text">q82</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q85">
+ <output-dir compare="Text">q85</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q86">
+ <output-dir compare="Text">q86</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q88">
+ <output-dir compare="Text">q88</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q89">
+ <output-dir compare="Text">q89</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q90">
+ <output-dir compare="Text">q90</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q91">
+ <output-dir compare="Text">q91</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q92">
+ <output-dir compare="Text">q92</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q94">
+ <output-dir compare="Text">q94</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q95">
+ <output-dir compare="Text">q95</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q96">
+ <output-dir compare="Text">q96</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q98">
+ <output-dir compare="Text">q98</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="tpch">
+ <test-case FilePath="tpch">
+ <compilation-unit name="distinct_by">
+ <output-dir compare="Text">distinct_by</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch">
+ <compilation-unit name="group_no_agg">
+ <output-dir compare="Text">group_no_agg</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch">
+ <compilation-unit name="nest_aggregate">
+ <output-dir compare="Text">nest_aggregate</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch">
+ <compilation-unit name="nest_aggregate2">
+ <output-dir compare="Text">nest_aggregate2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch">
+ <compilation-unit name="query-issue638">
+ <output-dir compare="Text">query-issue638</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch">
+ <compilation-unit name="query-issue785">
+ <output-dir compare="Text">query-issue785</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch">
+ <compilation-unit name="query-issue785-2">
+ <output-dir compare="Text">query-issue785-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch">
+ <compilation-unit name="query-issue786">
+ <output-dir compare="Text">query-issue786</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch">
+ <compilation-unit name="query-issue601">
+ <output-dir compare="Text">query-issue601</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch">
+ <compilation-unit name="q10_returned_item">
+ <output-dir compare="Text">q10_returned_item</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch">
+ <compilation-unit name="q10_returned_item_int64">
+ <output-dir compare="Text">q10_returned_item_int64</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch">
+ <compilation-unit name="q11_important_stock">
+ <output-dir compare="Text">q11_important_stock</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch">
+ <compilation-unit name="q12_shipping">
+ <output-dir compare="Text">q12_shipping</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch">
+ <compilation-unit name="q13_customer_distribution">
+ <output-dir compare="Text">q13_customer_distribution</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch">
+ <compilation-unit name="q14_promotion_effect">
+ <output-dir compare="Text">q14_promotion_effect</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch">
+ <compilation-unit name="q15_top_supplier">
+ <output-dir compare="Text">q15_top_supplier</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch">
+ <compilation-unit name="q16_parts_supplier_relationship">
+ <output-dir compare="Text">q16_parts_supplier_relationship</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch">
+ <compilation-unit name="q17_small_quantity_order_revenue">
+ <output-dir compare="Text">q17_small_quantity_order_revenue</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch">
+ <compilation-unit name="q17_large_gby_variant">
+ <output-dir compare="Text">q17_large_gby_variant</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch">
+ <compilation-unit name="q18_large_volume_customer">
+ <output-dir compare="Text">q18_large_volume_customer</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch">
+ <compilation-unit name="q19_discounted_revenue">
+ <output-dir compare="Text">q19_discounted_revenue</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch">
+ <compilation-unit name="q01_pricing_summary_report_nt">
+ <output-dir compare="Text">q01_pricing_summary_report_nt</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch">
+ <compilation-unit name="q20_potential_part_promotion">
+ <output-dir compare="Text">q20_potential_part_promotion</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch">
+ <compilation-unit name="q21_suppliers_who_kept_orders_waiting">
+ <output-dir compare="Text">q21_suppliers_who_kept_orders_waiting</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch">
+ <compilation-unit name="q22_global_sales_opportunity">
+ <output-dir compare="Text">q22_global_sales_opportunity</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch">
+ <compilation-unit name="q02_minimum_cost_supplier">
+ <output-dir compare="Text">q02_minimum_cost_supplier</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch">
+ <compilation-unit name="q03_shipping_priority_nt">
+ <output-dir compare="Text">q03_shipping_priority_nt</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch">
+ <compilation-unit name="q04_order_priority">
+ <output-dir compare="Text">q04_order_priority</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch">
+ <compilation-unit name="q05_local_supplier_volume">
+ <output-dir compare="Text">q05_local_supplier_volume</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch">
+ <compilation-unit name="q06_forecast_revenue_change">
+ <output-dir compare="Text">q06_forecast_revenue_change</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch">
+ <compilation-unit name="q07_volume_shipping">
+ <output-dir compare="Text">q07_volume_shipping</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch">
+ <compilation-unit name="q08_national_market_share">
+ <output-dir compare="Text">q08_national_market_share</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch">
+ <compilation-unit name="q09_product_type_profit_nt">
+ <output-dir compare="Text">q09_product_type_profit_nt</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch">
+ <compilation-unit name="query-issue562">
+ <output-dir compare="Text">query-issue562</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch">
+ <compilation-unit name="query-issue810">
+ <output-dir compare="Text">query-issue810</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch">
+ <compilation-unit name="query-issue810-2">
+ <output-dir compare="Text">query-issue810-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch">
+ <compilation-unit name="query-issue810-3">
+ <output-dir compare="Text">query-issue810-3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch">
+ <compilation-unit name="query-issue827">
+ <output-dir compare="Text">query-issue827</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch">
+ <compilation-unit name="query-issue827-2">
+ <output-dir compare="Text">query-issue827-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch">
+ <compilation-unit name="q01-ASTERIXDB-830">
+ <output-dir compare="Text">q01_pricing_summary_report_nt</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="tpch-sql">
+ <test-case FilePath="tpch-sql">
+ <compilation-unit name="distinct_by">
+ <output-dir compare="Text">distinct_by</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql">
+ <compilation-unit name="group_no_agg">
+ <output-dir compare="Text">group_no_agg</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql">
+ <compilation-unit name="nest_aggregate">
+ <output-dir compare="Text">nest_aggregate</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql">
+ <compilation-unit name="nest_aggregate2">
+ <output-dir compare="Text">nest_aggregate2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql">
+ <compilation-unit name="query-issue638">
+ <output-dir compare="Text">query-issue638</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql">
+ <compilation-unit name="query-issue785">
+ <output-dir compare="Text">query-issue785</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql">
+ <compilation-unit name="query-issue785-2">
+ <output-dir compare="Text">query-issue785-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql">
+ <compilation-unit name="query-issue786">
+ <output-dir compare="Text">query-issue786</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql">
+ <compilation-unit name="query-issue601">
+ <output-dir compare="Text">query-issue601</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql">
+ <compilation-unit name="q10_returned_item">
+ <output-dir compare="Text">q10_returned_item</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql">
+ <compilation-unit name="q10_returned_item_int64">
+ <output-dir compare="Text">q10_returned_item_int64</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql">
+ <compilation-unit name="q11_important_stock">
+ <output-dir compare="Text">q11_important_stock</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql">
+ <compilation-unit name="q12_shipping">
+ <output-dir compare="Text">q12_shipping</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql">
+ <compilation-unit name="q13_customer_distribution">
+ <output-dir compare="Text">q13_customer_distribution</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql">
+ <compilation-unit name="q14_promotion_effect">
+ <output-dir compare="Text">q14_promotion_effect</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql">
+ <compilation-unit name="q15_top_supplier">
+ <output-dir compare="Text">q15_top_supplier</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql">
+ <compilation-unit name="q16_parts_supplier_relationship">
+ <output-dir compare="Text">q16_parts_supplier_relationship</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql">
+ <compilation-unit name="q17_small_quantity_order_revenue">
+ <output-dir compare="Text">q17_small_quantity_order_revenue</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql">
+ <compilation-unit name="q17_large_gby_variant">
+ <output-dir compare="Text">q17_large_gby_variant</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql">
+ <compilation-unit name="q18_large_volume_customer">
+ <output-dir compare="Text">q18_large_volume_customer</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql">
+ <compilation-unit name="q19_discounted_revenue">
+ <output-dir compare="Text">q19_discounted_revenue</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql">
+ <compilation-unit name="q01_pricing_summary_report_nt">
+ <output-dir compare="Text">q01_pricing_summary_report_nt</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql">
+ <compilation-unit name="q20_potential_part_promotion">
+ <output-dir compare="Text">q20_potential_part_promotion</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql">
+ <compilation-unit name="q21_suppliers_who_kept_orders_waiting">
+ <output-dir compare="Text">q21_suppliers_who_kept_orders_waiting</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql">
+ <compilation-unit name="q22_global_sales_opportunity">
+ <output-dir compare="Text">q22_global_sales_opportunity</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql">
+ <compilation-unit name="q02_minimum_cost_supplier">
+ <output-dir compare="Text">q02_minimum_cost_supplier</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql">
+ <compilation-unit name="q03_shipping_priority_nt">
+ <output-dir compare="Text">q03_shipping_priority_nt</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql">
+ <compilation-unit name="q04_order_priority">
+ <output-dir compare="Text">q04_order_priority</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql">
+ <compilation-unit name="q05_local_supplier_volume">
+ <output-dir compare="Text">q05_local_supplier_volume</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql">
+ <compilation-unit name="q06_forecast_revenue_change">
+ <output-dir compare="Text">q06_forecast_revenue_change</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql">
+ <compilation-unit name="q07_volume_shipping">
+ <output-dir compare="Text">q07_volume_shipping</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql">
+ <compilation-unit name="q08_national_market_share">
+ <output-dir compare="Text">q08_national_market_share</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql">
+ <compilation-unit name="q09_product_type_profit_nt">
+ <output-dir compare="Text">q09_product_type_profit_nt</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql">
+ <compilation-unit name="query-issue562">
+ <output-dir compare="Text">query-issue562</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql">
+ <compilation-unit name="query-issue810">
+ <output-dir compare="Text">query-issue810</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql">
+ <compilation-unit name="query-issue810-2">
+ <output-dir compare="Text">query-issue810-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql">
+ <compilation-unit name="query-issue810-3">
+ <output-dir compare="Text">query-issue810-3</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="tpch-sql-sugar">
+ <test-case FilePath="tpch-sql-sugar">
+ <compilation-unit name="distinct_by">
+ <output-dir compare="Text">distinct_by</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-sugar">
+ <compilation-unit name="group_no_agg">
+ <output-dir compare="Text">group_no_agg</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-sugar">
+ <compilation-unit name="nest_aggregate">
+ <output-dir compare="Text">nest_aggregate</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-sugar">
+ <compilation-unit name="nest_aggregate2">
+ <output-dir compare="Text">nest_aggregate2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-sugar">
+ <compilation-unit name="query-issue638">
+ <output-dir compare="Text">query-issue638</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-sugar">
+ <compilation-unit name="query-issue785">
+ <output-dir compare="Text">query-issue785</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-sugar">
+ <compilation-unit name="query-issue785-2">
+ <output-dir compare="Text">query-issue785-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-sugar">
+ <compilation-unit name="query-issue786">
+ <output-dir compare="Text">query-issue786</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-sugar">
+ <compilation-unit name="query-issue601">
+ <output-dir compare="Text">query-issue601</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-sugar">
+ <compilation-unit name="q10_returned_item">
+ <output-dir compare="Text">q10_returned_item</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-sugar">
+ <compilation-unit name="q10_returned_item_int64">
+ <output-dir compare="Text">q10_returned_item_int64</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-sugar">
+ <compilation-unit name="q11_important_stock">
+ <output-dir compare="Text">q11_important_stock</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-sugar">
+ <compilation-unit name="q12_shipping">
+ <output-dir compare="Text">q12_shipping</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-sugar">
+ <compilation-unit name="q12_shipping_broadcast">
+ <output-dir compare="Text">q12_shipping</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-sugar">
+ <compilation-unit name="q13_customer_distribution">
+ <output-dir compare="Text">q13_customer_distribution</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-sugar">
+ <compilation-unit name="q14_promotion_effect">
+ <output-dir compare="Text">q14_promotion_effect</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-sugar">
+ <compilation-unit name="q15_top_supplier">
+ <output-dir compare="Text">q15_top_supplier</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-sugar">
+ <compilation-unit name="q16_parts_supplier_relationship">
+ <output-dir compare="Text">q16_parts_supplier_relationship</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-sugar">
+ <compilation-unit name="q17_small_quantity_order_revenue">
+ <output-dir compare="Text">q17_small_quantity_order_revenue</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-sugar">
+ <compilation-unit name="q17_large_gby_variant">
+ <output-dir compare="Text">q17_large_gby_variant</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-sugar">
+ <compilation-unit name="q17_large_gby_variant_parameter">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>Invalid query parameter compiler.groupmemory -- value has to be greater than or equal to</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-sugar">
+ <compilation-unit name="q18_large_volume_customer">
+ <output-dir compare="Text">q18_large_volume_customer</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-sugar">
+ <compilation-unit name="q19_discounted_revenue">
+ <output-dir compare="Text">q19_discounted_revenue</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-sugar">
+ <compilation-unit name="q01_pricing_summary_report_nt">
+ <output-dir compare="Text">q01_pricing_summary_report_nt</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-sugar">
+ <compilation-unit name="q01_pricing_summary_report_2">
+ <output-dir compare="Text">q01_pricing_summary_report_nt</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-sugar">
+ <compilation-unit name="q01_pricing_summary_report_parallelism">
+ <output-dir compare="Text">q01_pricing_summary_report_nt</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-sugar">
+ <compilation-unit name="q01_pricing_summary_report_parameter">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>Invalid query parameter compiler.sortmemory -- value has to be greater than or equal to</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-sugar">
+ <compilation-unit name="q20_potential_part_promotion">
+ <output-dir compare="Text">q20_potential_part_promotion</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-sugar">
+ <compilation-unit name="q21_suppliers_who_kept_orders_waiting">
+ <output-dir compare="Text">q21_suppliers_who_kept_orders_waiting</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-sugar">
+ <compilation-unit name="q22_global_sales_opportunity">
+ <output-dir compare="Text">q22_global_sales_opportunity</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-sugar">
+ <compilation-unit name="q02_minimum_cost_supplier">
+ <output-dir compare="Text">q02_minimum_cost_supplier</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-sugar">
+ <compilation-unit name="q03_shipping_priority_nt">
+ <output-dir compare="Text">q03_shipping_priority_nt</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-sugar">
+ <compilation-unit name="q04_order_priority">
+ <output-dir compare="Text">q04_order_priority</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-sugar">
+ <compilation-unit name="q05_local_supplier_volume">
+ <output-dir compare="Text">q05_local_supplier_volume</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-sugar">
+ <compilation-unit name="q06_forecast_revenue_change">
+ <output-dir compare="Text">q06_forecast_revenue_change</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-sugar">
+ <compilation-unit name="q07_volume_shipping">
+ <output-dir compare="Text">q07_volume_shipping</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-sugar">
+ <compilation-unit name="q08_national_market_share">
+ <output-dir compare="Text">q08_national_market_share</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-sugar">
+ <compilation-unit name="q09_product_type_profit_nt">
+ <output-dir compare="Text">q09_product_type_profit_nt</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-sugar">
+ <compilation-unit name="q09_product_type_profit_parameter">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>Invalid query parameter compiler.joinmemory -- value has to be greater than or equal to</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-sugar">
+ <compilation-unit name="query-issue562">
+ <output-dir compare="Text">query-issue562</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-sugar">
+ <compilation-unit name="query-issue810">
+ <output-dir compare="Text">query-issue810</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-sugar">
+ <compilation-unit name="query-issue810-2">
+ <output-dir compare="Text">query-issue810-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-sugar">
+ <compilation-unit name="query-issue810-3">
+ <output-dir compare="Text">query-issue810-3</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="tpch-with-index">
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="distinct_by">
+ <output-dir compare="Text">distinct_by</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="nest_aggregate">
+ <output-dir compare="Text">nest_aggregate</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="nest_aggregate2">
+ <output-dir compare="Text">nest_aggregate2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="query-issue785">
+ <output-dir compare="Text">query-issue785</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="query-issue785-2">
+ <output-dir compare="Text">query-issue785-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="query-issue786">
+ <output-dir compare="Text">query-issue786</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="query-issue601">
+ <output-dir compare="Text">query-issue601</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="q10_returned_item">
+ <output-dir compare="Text">q10_returned_item</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="q10_returned_item_int64">
+ <output-dir compare="Text">q10_returned_item_int64</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="q11_important_stock">
+ <output-dir compare="Text">q11_important_stock</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="q12_shipping">
+ <output-dir compare="Text">q12_shipping</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="q13_customer_distribution">
+ <output-dir compare="Text">q13_customer_distribution</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="q14_promotion_effect">
+ <output-dir compare="Text">q14_promotion_effect</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="q15_top_supplier">
+ <output-dir compare="Text">q15_top_supplier</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="q16_parts_supplier_relationship">
+ <output-dir compare="Text">q16_parts_supplier_relationship</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="q17_small_quantity_order_revenue">
+ <output-dir compare="Text">q17_small_quantity_order_revenue</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="q17_large_gby_variant">
+ <output-dir compare="Text">q17_large_gby_variant</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="q18_large_volume_customer">
+ <output-dir compare="Text">q18_large_volume_customer</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="q19_discounted_revenue">
+ <output-dir compare="Text">q19_discounted_revenue</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="q01_pricing_summary_report_nt">
+ <output-dir compare="Text">q01_pricing_summary_report_nt</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="q20_potential_part_promotion">
+ <output-dir compare="Text">q20_potential_part_promotion</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="q21_suppliers_who_kept_orders_waiting">
+ <output-dir compare="Text">q21_suppliers_who_kept_orders_waiting</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="q22_global_sales_opportunity">
+ <output-dir compare="Text">q22_global_sales_opportunity</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="q02_minimum_cost_supplier">
+ <output-dir compare="Text">q02_minimum_cost_supplier</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="q03_shipping_priority_nt">
+ <output-dir compare="Text">q03_shipping_priority_nt</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="q04_order_priority">
+ <output-dir compare="Text">q04_order_priority</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="q04_order_priority_with_nodegroup">
+ <output-dir compare="Text">q04_order_priority</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="q05_local_supplier_volume">
+ <output-dir compare="Text">q05_local_supplier_volume</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="q06_forecast_revenue_change">
+ <output-dir compare="Text">q06_forecast_revenue_change</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="q07_volume_shipping">
+ <output-dir compare="Text">q07_volume_shipping</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="q08_national_market_share">
+ <output-dir compare="Text">q08_national_market_share</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="q09_product_type_profit_nt">
+ <output-dir compare="Text">q09_product_type_profit_nt</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="query-issue562">
+ <output-dir compare="Text">query-issue562</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="query-issue810">
+ <output-dir compare="Text">query-issue810</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="query-issue810-2">
+ <output-dir compare="Text">query-issue810-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="query-issue810-3">
+ <output-dir compare="Text">query-issue810-3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="query-issue827">
+ <output-dir compare="Text">query-issue827</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="query-issue827-2">
+ <output-dir compare="Text">query-issue827-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="tpch-sql-like">
+ <test-case FilePath="tpch-sql-like">
+ <compilation-unit name="query-issue638">
+ <output-dir compare="Text">query-issue638</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-like">
+ <compilation-unit name="query-issue785">
+ <output-dir compare="Text">query-issue785</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-like">
+ <compilation-unit name="query-issue785-2">
+ <output-dir compare="Text">query-issue785-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-like">
+ <compilation-unit name="query-issue786">
+ <output-dir compare="Text">query-issue786</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-like">
+ <compilation-unit name="query-issue601">
+ <output-dir compare="Text">query-issue601</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-like">
+ <compilation-unit name="q10_returned_item">
+ <output-dir compare="Text">q10_returned_item</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-like">
+ <compilation-unit name="q10_returned_item_int64">
+ <output-dir compare="Text">q10_returned_item_int64</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-like">
+ <compilation-unit name="q11_important_stock">
+ <output-dir compare="Text">q11_important_stock</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-like">
+ <compilation-unit name="q12_shipping">
+ <output-dir compare="Text">q12_shipping</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-like">
+ <compilation-unit name="q13_customer_distribution">
+ <output-dir compare="Text">q13_customer_distribution</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-like">
+ <compilation-unit name="q14_promotion_effect">
+ <output-dir compare="Text">q14_promotion_effect</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-like">
+ <compilation-unit name="q15_top_supplier">
+ <output-dir compare="Text">q15_top_supplier</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-like">
+ <compilation-unit name="q16_parts_supplier_relationship">
+ <output-dir compare="Text">q16_parts_supplier_relationship</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-like">
+ <compilation-unit name="q17_small_quantity_order_revenue">
+ <output-dir compare="Text">q17_small_quantity_order_revenue</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-like">
+ <compilation-unit name="q17_large_gby_variant">
+ <output-dir compare="Text">q17_large_gby_variant</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-like">
+ <compilation-unit name="q18_large_volume_customer">
+ <output-dir compare="Text">q18_large_volume_customer</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-like">
+ <compilation-unit name="q19_discounted_revenue">
+ <output-dir compare="Text">q19_discounted_revenue</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-like">
+ <compilation-unit name="q01_pricing_summary_report_nt">
+ <output-dir compare="Text">q01_pricing_summary_report_nt</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-like">
+ <compilation-unit name="q20_potential_part_promotion">
+ <output-dir compare="Text">q20_potential_part_promotion</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-like">
+ <compilation-unit name="q21_suppliers_who_kept_orders_waiting">
+ <output-dir compare="Text">q21_suppliers_who_kept_orders_waiting</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-like">
+ <compilation-unit name="q22_global_sales_opportunity">
+ <output-dir compare="Text">q22_global_sales_opportunity</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-like">
+ <compilation-unit name="q02_minimum_cost_supplier">
+ <output-dir compare="Text">q02_minimum_cost_supplier</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-like">
+ <compilation-unit name="q03_shipping_priority_nt">
+ <output-dir compare="Text">q03_shipping_priority_nt</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-like">
+ <compilation-unit name="q04_order_priority">
+ <output-dir compare="Text">q04_order_priority</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-like">
+ <compilation-unit name="q05_local_supplier_volume">
+ <output-dir compare="Text">q05_local_supplier_volume</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-like">
+ <compilation-unit name="q06_forecast_revenue_change">
+ <output-dir compare="Text">q06_forecast_revenue_change</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-like">
+ <compilation-unit name="q07_volume_shipping">
+ <output-dir compare="Text">q07_volume_shipping</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-like">
+ <compilation-unit name="q08_national_market_share">
+ <output-dir compare="Text">q08_national_market_share</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-sql-like">
+ <compilation-unit name="q09_product_type_profit_nt">
+ <output-dir compare="Text">q09_product_type_profit_nt</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="writers">
+ <test-case FilePath="writers">
+ <compilation-unit name="print_01">
+ <output-dir compare="Text">print_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <!-- TODO(madhusudancs): Enable this test when REST API supports serialized output support.
+ <test-case FilePath="writers">
+ <compilation-unit name="serialized_01">
+ <output-dir compare="Text">serialized_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ -->
+ </test-group>
+ <test-group name="cross-dataverse">
+ <test-case FilePath="cross-dataverse">
+ <compilation-unit name="cross-dv01">
+ <output-dir compare="Text">cross-dv01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="cross-dataverse">
+ <compilation-unit name="cross-dv02">
+ <output-dir compare="Text">cross-dv02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="cross-dataverse">
+ <compilation-unit name="cross-dv03">
+ <output-dir compare="Text">cross-dv03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="cross-dataverse">
+ <compilation-unit name="cross-dv04">
+ <output-dir compare="Text">cross-dv04</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="cross-dataverse">
+ <compilation-unit name="cross-dv07">
+ <output-dir compare="Text">cross-dv07</output-dir>
+ </compilation-unit>
+ </test-case>
+ <!--NotImplementedException: No binary comparator factory implemented for type OBJECT.
+ <test-case FilePath="cross-dataverse">
+ <compilation-unit name="cross-dv08">
+ <output-dir compare="Text">cross-dv08</output-dir>
+ </compilation-unit>
+ </test-case>
+ -->
+ <test-case FilePath="cross-dataverse">
+ <compilation-unit name="cross-dv09">
+ <output-dir compare="Text">cross-dv09</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="cross-dataverse">
+ <compilation-unit name="cross-dv11">
+ <output-dir compare="Text">cross-dv11</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="cross-dataverse">
+ <compilation-unit name="cross-dv12">
+ <output-dir compare="Text">cross-dv12</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="cross-dataverse">
+ <compilation-unit name="cross-dv14">
+ <output-dir compare="Text">cross-dv14</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="cross-dataverse">
+ <compilation-unit name="cross-dv15">
+ <output-dir compare="Text">cross-dv15</output-dir>
+ </compilation-unit>
+ </test-case>
+ <!--NotImplementedException: No binary comparator factory implemented for type OBJECT.
+ <test-case FilePath="cross-dataverse">
+ <compilation-unit name="cross-dv17">
+ <output-dir compare="Text">cross-dv17</output-dir>
+ </compilation-unit>
+ </test-case>
+ -->
+ <!--NotImplementedException: No binary comparator factory implemented for type OBJECT.
+ <test-case FilePath="cross-dataverse">
+ <compilation-unit name="cross-dv18">
+ <output-dir compare="Text">cross-dv18</output-dir>
+ </compilation-unit>
+ </test-case>
+ -->
+ <test-case FilePath="cross-dataverse">
+ <compilation-unit name="cross-dv19">
+ <output-dir compare="Text">cross-dv19</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="cross-dataverse">
+ <compilation-unit name="cross-dv20">
+ <output-dir compare="Text">cross-dv20</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="cross-dataverse">
+ <compilation-unit name="insert_across_dataverses">
+ <output-dir compare="Text">insert_across_dataverses</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="cross-dataverse">
+ <compilation-unit name="join_across_dataverses">
+ <output-dir compare="Text">join_across_dataverses</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="cross-dataverse">
+ <compilation-unit name="drop-dataverse">
+ <output-dir compare="Text">drop-dataverse</output-dir>
+ <expected-error>ASX1147: Cannot drop dataverse: type a.a being used by dataset b.b1</expected-error>
+ <expected-error>ASX1147: Cannot drop dataverse: synonym a.s1 being used by function b.f1()</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="cross-dataverse">
+ <compilation-unit name="drop-type-used-elsewhere">
+ <output-dir compare="Text">drop-type-used-elsewhere</output-dir>
+ <expected-error>Cannot drop type a.a being used by dataset b.b1</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="cross-dataverse">
+ <compilation-unit name="drop-type-used-here-dataset">
+ <output-dir compare="Text">drop-type-used-here-dataset</output-dir>
+ <expected-error>Cannot drop type c.a being used by dataset c.a1</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="cross-dataverse">
+ <compilation-unit name="drop-type-used-here-type">
+ <output-dir compare="Text">drop-type-used-here-type</output-dir>
+ <expected-error>Cannot drop type c.a being used by type c.b</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="cross-dataverse">
+ <compilation-unit name="query-dataset-with-foreign-type">
+ <output-dir compare="Text">query-dataset-with-foreign-type</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="user-defined-functions">
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="inline-in-expr">
+ <output-dir compare="Text">inline-in-expr</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="bad-function-ddl-1">
+ <output-dir compare="Text">bad-function-ddl-1</output-dir>
+ <expected-error>Cannot find dataset TweetMessages in dataverse experiments nor an alias with name TweetMessages</expected-error>
+ <expected-error>Cannot find dataset TweetMessages2 in dataverse experiments2 nor an alias with name TweetMessages2</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="bad-function-ddl-2">
+ <output-dir compare="Text">bad-function-ddl-2</output-dir>
+ <expected-error>Cannot find dataset TweetMessages in dataverse experiments2 nor an alias with name TweetMessages</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="bad-function-ddl-3">
+ <output-dir compare="Text">bad-function-ddl-3</output-dir>
+ <expected-error>Cannot find dataset TweetMessages in dataverse experiments nor an alias with name TweetMessages</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="bad-function-ddl-4">
+ <output-dir compare="Text">bad-function-ddl-4</output-dir>
+ <expected-error>Cannot find dataset TweetMessages in dataverse experients nor an alias with name TweetMessages</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="bad-function-ddl-5">
+ <output-dir compare="Text">bad-function-ddl-5</output-dir>
+ <expected-error>ASX1081: Cannot find function with signature experiments.function_that_does_not_exist()</expected-error>
+ <expected-error>ASX1081: Cannot find function with signature experiments2.function_that_does_not_exist()</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="bad-function-ddl-6">
+ <output-dir compare="Text">bad-function-ddl-6</output-dir>
+ <expected-error>ASX1081: Cannot find function with signature experiments.function_that_does_not_exist(2)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="bad-function-ddl-7">
+ <output-dir compare="Text">bad-function-ddl-7</output-dir>
+ <expected-error>Cannot find dataset TweetMessaes in dataverse experiments nor an alias with name TweetMessaes</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="bad-function-ddl-8">
+ <output-dir compare="Text">bad-function-ddl-8</output-dir>
+ <expected-error>Cannot find dataset TweetMessaes in dataverse experiments nor an alias with name TweetMessaes</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="bad-function-ddl-9">
+ <output-dir compare="Text">bad-function-ddl-9</output-dir>
+ <expected-error>ASX1081: Cannot find function with signature experiments.function_that_does_not_exist()</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="bad-function-ddl-10">
+ <output-dir compare="Text">bad-function-ddl-10</output-dir>
+ <expected-error>ASX1081: Cannot find function with signature experiments.f0(2)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="bad-function-ddl-11">
+ <output-dir compare="Text">bad-function-ddl-11</output-dir>
+ <expected-error>ASX1001: Syntax error: Unexpected type declaration for parameter a in function myfn001</expected-error>
+ <expected-error>ASX1001: Syntax error: Unexpected return type declaration for function myfn002</expected-error>
+ <expected-error>ASX1001: Syntax error: Unexpected return type declaration for function myfn003</expected-error>
+ <expected-error>ASX1001: Syntax error: Unexpected IF NOT EXISTS (in line 29, at column 45)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="check-dependencies-1">
+ <output-dir compare="Text">check-dependencies-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="create-or-replace-function-1">
+ <output-dir compare="Text">create-or-replace-function-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="drop-dependency-1">
+ <output-dir compare="Text">drop-dependency-1</output-dir>
+ <expected-error>ASX1147: Cannot drop dataverse: function C.f1(2) being used by function B.f0(2)</expected-error>
+ <expected-error>ASX1147: Cannot drop dataverse: function C.f2(...) being used by function B.f3(2)</expected-error>
+ <expected-error>ASX1147: Cannot drop dataverse: function C.f4(2) being used by function B.f5(...)</expected-error>
+ <expected-error>ASX1147: Cannot drop dataverse: function C.f6(...) being used by function B.f7(...)</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="drop-dependency-2">
+ <output-dir compare="Text">drop-dependency-2</output-dir>
+ <expected-error>ASX1147: Cannot drop dataverse: dataset (or view) C.TweetMessages being used by function B.f2(2)</expected-error>
+ <expected-error>ASX1147: Cannot drop dataverse: dataset (or view) C.TweetMessages being used by function B.f3(...)</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="drop-dependency-3">
+ <output-dir compare="Text">drop-dependency-3</output-dir>
+ <expected-error>Cannot drop function C.f1(2) being used by function B.f0(2)</expected-error>
+ <expected-error>Cannot drop function C.f3(...) being used by function B.f2(2)</expected-error>
+ <expected-error>Cannot drop function C.f5(2) being used by function B.f4(...)</expected-error>
+ <expected-error>Cannot drop function C.f7(...) being used by function B.f6(...)</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="drop-dependency-4">
+ <output-dir compare="Text">drop-dependency-4</output-dir>
+ <expected-error>ASX1148: Cannot drop dataset C.TweetMessages being used by function B.f2(2)</expected-error>
+ <expected-error>ASX1148: Cannot drop dataset C.TweetMessages being used by function B.f2(...)</expected-error>
+ <expected-error>ASX1148: Cannot drop synonym C.TweetMessagesSyn being used by function B.f2(2)</expected-error>
+ <expected-error>ASX1148: Cannot drop synonym C.TweetMessagesSyn being used by function B.f2(...)</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="drop-dependency-5">
+ <output-dir compare="Text">drop-dependency-5</output-dir>
+ <expected-error>Cannot drop function C.f1(2) being used by function C.f0(2)</expected-error>
+ <expected-error>Cannot drop function C.f1(2) being used by function C.f0(...)</expected-error>
+ <expected-error>Cannot drop function C.f1(...) being used by function C.f0(2)</expected-error>
+ <expected-error>Cannot drop function C.f1(...) being used by function C.f0(...)</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="drop-dependency-6">
+ <output-dir compare="Text">drop-dependency-6</output-dir>
+ <expected-error>ASX1148: Cannot drop dataset C.TweetMessages being used by function C.f2(2)</expected-error>
+ <expected-error>ASX1148: Cannot drop dataset C.TweetMessages being used by function C.f2(...)</expected-error>
+ <expected-error>ASX1148: Cannot drop synonym C.TweetMessagesSyn being used by function C.f2(2)</expected-error>
+ <expected-error>ASX1148: Cannot drop synonym C.TweetMessagesSyn being used by function C.f2(...)</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="drop-function-1">
+ <output-dir compare="Text">drop-function-1</output-dir>
+ <expected-error>ASX1081: Cannot find function with signature experiments.my_sum(2)</expected-error>
+ <expected-error>ASX1081: Cannot find function with signature experiments.my_sum_va(2)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="single-line-definition">
+ <output-dir compare="Text">single-line-definition</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="query-ASTERIXDB-1298">
+ <output-dir compare="Text">query-ASTERIXDB-1298</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="query-ASTERIXDB-1652">
+ <output-dir compare="Text">query-ASTERIXDB-1652-2</output-dir>
+ <expected-error>ASX1063: Cannot find dataverse with name test</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="query-ASTERIXDB-1652-2">
+ <output-dir compare="Text">query-ASTERIXDB-1652-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="query-issue218-2">
+ <output-dir compare="Text">query-issue218-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="query-issue218">
+ <output-dir compare="Text">query-issue218</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="query-issue201">
+ <output-dir compare="Text">query-issue201</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="query-issue172">
+ <output-dir compare="Text">query-issue172</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="query-issue455">
+ <output-dir compare="Text">query-issue455</output-dir>
+ <expected-error>ASX1081: Cannot find function with signature test.printName()</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="query-issue489">
+ <output-dir compare="Text">query-issue489</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="udf01">
+ <output-dir compare="Text">udf01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="udf02">
+ <output-dir compare="Text">udf02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="udf03">
+ <output-dir compare="Text">udf03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="udf04">
+ <output-dir compare="Text">udf04</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="udf05">
+ <output-dir compare="Text">udf05</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="udf06">
+ <output-dir compare="Text">udf06</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="udf07">
+ <output-dir compare="Text">udf07</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="udf08">
+ <output-dir compare="Text">udf08</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="udf09">
+ <output-dir compare="Text">udf09</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="udf10">
+ <output-dir compare="Text">udf10</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="udf11">
+ <output-dir compare="Text">udf11</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="udf12">
+ <output-dir compare="Text">udf12</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="udf13">
+ <output-dir compare="Text">udf13</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="udf14">
+ <output-dir compare="Text">udf14</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="udf15">
+ <output-dir compare="Text">udf15</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="udf16">
+ <output-dir compare="Text">udf16</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="udf17">
+ <output-dir compare="Text">udf17</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="udf18">
+ <output-dir compare="Text">udf18</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="udf19">
+ <output-dir compare="Text">udf19</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="udf20">
+ <output-dir compare="Text">udf20</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="udf21">
+ <output-dir compare="Text">udf21</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="udf22">
+ <output-dir compare="Text">udf22</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="udf23">
+ <output-dir compare="Text">udf23</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="udf24">
+ <output-dir compare="Text">udf24</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="udf25">
+ <output-dir compare="Text">udf25</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="udf27">
+ <output-dir compare="Text">udf27</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="udf28">
+ <output-dir compare="Text">udf28</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="udf29">
+ <output-dir compare="Text">udf29</output-dir>
+ </compilation-unit>
+ </test-case>
+ <!-- Need to verify the expected exception -->
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="udf30">
+ <output-dir compare="Text">udf30</output-dir>
+ <expected-error>ASX1073: Cannot resolve alias reference for undefined identifier y (in line 30, at column 8)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="udf31">
+ <output-dir compare="Text">udf31</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="udf32_metadata">
+ <output-dir compare="Text">udf32_metadata</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="udf33_overloading">
+ <output-dir compare="Text">udf33_overloading</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="udf34">
+ <output-dir compare="Text">udf34</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="udf35_varargs_misc">
+ <output-dir compare="Text">udf35_varargs_misc</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="udf36_in_with">
+ <output-dir compare="Text">udf36_in_with</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="udf37_recursion">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>ASX1149: Illegal function or view recursion (in line 24, at column 1)</expected-error>
+ <expected-error>ASX1149: Illegal function or view recursion (in line 25, at column 1)</expected-error>
+ <expected-error>ASX1149: Illegal function or view recursion (in line 26, at column 1)</expected-error>
+ <expected-error>ASX1149: Illegal function or view recursion (in line 27, at column 1)</expected-error>
+ <expected-error>ASX1149: Illegal function or view recursion (in line 28, at column 1)</expected-error>
+ <expected-error>ASX1149: Illegal function or view recursion (in line 29, at column 1)</expected-error>
+ <expected-error>ASX1149: Illegal function or view recursion (in line 30, at column 1)</expected-error>
+ <expected-error>ASX1149: Illegal function or view recursion (in line 31, at column 1)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="udf38_no_recursion">
+ <output-dir compare="Text">udf38_no_recursion</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="udf39_illegal_call">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>ASX1150: Illegal use of function test.f1a(1) (in line 32, at column 26)</expected-error>
+ <expected-error>ASX1150: Illegal use of function test.f2a(...) (in line 29, at column 28)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="f01">
+ <output-dir compare="Text">f01</output-dir>
+ <expected-error>ASX1081: Cannot find function with signature test.tinyint()</expected-error>
+ </compilation-unit>
+ </test-case>
+ <!-- This test case is not valid anymore since we do not required "IMPORT_PRIVATE_FUNCTIONS" flag anymore -->
+ <!-- <test-case FilePath="user-defined-functions">
+ <compilation-unit name="invoke-private-function">
+ <output-dir compare="Text">invoke-private-function</output-dir>
+ </compilation-unit>
+ </test-case>-->
+ <!--
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="query-ASTERIXDB-1308-1">
+ <output-dir compare="Text">query-ASTERIXDB-1308-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ -->
+ <!-- <test-case FilePath="user-defined-functions">
+ <compilation-unit name="query-ASTERIXDB-1308-2">
+ <output-dir compare="Text">query-ASTERIXDB-1308-2</output-dir>
+ </compilation-unit>
+ </test-case> -->
+ <!-- This test case is not valid anymore since we do not required "IMPORT_PRIVATE_FUNCTIONS" flag anymore -->
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="query-ASTERIXDB-1317">
+ <output-dir compare="Text">query-ASTERIXDB-1317</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="view">
+ <test-case FilePath="view">
+ <compilation-unit name="create-view-1">
+ <output-dir compare="Text">create-view-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="view">
+ <compilation-unit name="create-view-2-negative">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>ASX1063: Cannot find dataverse with name test (in line 24, at column 1)</expected-error>
+ <expected-error><![CDATA[ASX1001: Syntax error: In line 25 >>create view test.v1 as select * from range(1,2) r order by;<< Encountered ";" at column 59]]></expected-error>
+ <expected-error>ASX1081: Cannot find function with signature test.undefined_range(2) (in line 25, at column 38)</expected-error>
+ <expected-error>ASX1160: A view with this name test.v1 already exists (in line 26, at column 1)</expected-error>
+ <expected-error>ASX1072: A dataset with name ds1 already exists in dataverse test (in line 30, at column 1)</expected-error>
+ <expected-error>ASX1072: A dataset with name ds2 already exists in dataverse test (in line 30, at column 1)</expected-error>
+ <expected-error>ASX1160: A view with this name test.ds1 already exists (in line 27, at column 1)</expected-error>
+ <expected-error><![CDATA[ASX1001: Syntax error: Unexpected IF NOT EXISTS (in line 25, at column 1)]]></expected-error>
+ <expected-error>ASX1149: Illegal function or view recursion (in line 31, at column 1)</expected-error>
+ <expected-error>ASX1149: Illegal function or view recursion (in line 32, at column 1)</expected-error>
+ <expected-error>ASX1149: Illegal function or view recursion (in line 33, at column 1)</expected-error>
+ <expected-error><![CDATA[ASX1001: Syntax error: In line 25 >>create view test.v1 primary key (r) not enforced as<< Encountered "primary" at column 21]]></expected-error>
+ <expected-error><![CDATA[ASX1001: Syntax error: In line 28 >>create view test.v2 foreign key (r) references v1 not enforced as<< Encountered <IDENTIFIER> "foreign" at column 21]]></expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="view">
+ <compilation-unit name="create-view-3-typed">
+ <output-dir compare="Text">create-view-3-typed</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="view" check-warnings="true">
+ <compilation-unit name="create-view-4-typed-warn">
+ <output-dir compare="Text">create-view-4-typed-warn</output-dir>
+ <expected-warn>ASX0006: Invalid format for tinyint in a (in line 30, at column 6)</expected-warn>
+ <expected-warn>ASX0006: Invalid format for smallint in b (in line 30, at column 6)</expected-warn>
+ <expected-warn>ASX0006: Invalid format for integer in c (in line 30, at column 6)</expected-warn>
+ <expected-warn>ASX0006: Invalid format for bigint in d (in line 30, at column 6)</expected-warn>
+ <expected-warn>ASX0006: Invalid format for float in e (in line 30, at column 6)</expected-warn>
+ <expected-warn>ASX0006: Invalid format for double in f (in line 30, at column 6)</expected-warn>
+ <expected-warn>ASX0006: Invalid format for datetime in g (in line 30, at column 6)</expected-warn>
+ <expected-warn>ASX0006: Invalid format for date in h (in line 30, at column 6)</expected-warn>
+ <expected-warn>ASX0006: Invalid format for time in j (in line 30, at column 6)</expected-warn>
+ <expected-warn>ASX0006: Invalid format for duration in k (in line 30, at column 6)</expected-warn>
+ <expected-warn>ASX0006: Invalid format for yearmonthduration in m (in line 30, at column 6)</expected-warn>
+ <expected-warn>ASX0006: Invalid format for daytimeduration in n (in line 30, at column 6)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: boolean-default-null() cannot process input type duration (in line 30, at column 6)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: int8-default-null() cannot process input type duration (in line 30, at column 6)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: int16-default-null() cannot process input type duration (in line 30, at column 6)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: int32-default-null() cannot process input type duration (in line 30, at column 6)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: int64-default-null() cannot process input type duration (in line 30, at column 6)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: float-default-null() cannot process input type duration (in line 30, at column 6)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: double-default-null() cannot process input type duration (in line 30, at column 6)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: datetime-default-null() cannot process input type duration (in line 30, at column 6)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: date-default-null() cannot process input type duration (in line 30, at column 6)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: time-default-null() cannot process input type duration (in line 30, at column 6)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: duration-default-null() cannot process input type date (in line 30, at column 6)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: year-month-duration-default-null() cannot process input type date (in line 30, at column 6)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: day-time-duration-default-null() cannot process input type date (in line 30, at column 6)</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="view" check-warnings="true">
+ <compilation-unit name="create-view-5-typed-warn">
+ <output-dir compare="Text">create-view-5-typed-warn</output-dir>
+ <expected-warn>ASX0006: Invalid format for datetime in a (in line 27, at column 6)</expected-warn>
+ <expected-warn>ASX0006: Invalid format for date in b (in line 27, at column 6)</expected-warn>
+ <expected-warn>ASX0006: Invalid format for time in c (in line 27, at column 6)</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="view">
+ <compilation-unit name="create-view-6-typed-negative">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>ASX1082: Cannot find datatype with name unknown_dv.t1</expected-error>
+ <expected-error>ASX1082: Cannot find datatype with name test.t1_unknown</expected-error>
+ <expected-error>ASX1079: Compilation error: view type cannot have open fields (in line 29, at column 1)</expected-error>
+ <expected-error>ASX1004: Unsupported type: view cannot process input type t1_a (in line 30, at column 1)</expected-error>
+ <expected-error><![CDATA[ASX1001: Syntax error: In line 25 >>create view test.v1(r bigint, a [bigint]) default null as<< Encountered "[" at column 33]]></expected-error>
+ <expected-error>ASX1092: Parameter date_illegal_property_name cannot be set (in line 25, at column 1)</expected-error>
+ <expected-error><![CDATA[ASX1001: Syntax error: In line 25 >>create view test.v1(r bigint) as<< Encountered "as" at column 31]]></expected-error>
+ <expected-error><![CDATA[ASX1014: Field 'unknown_field' is not found (in line 25, at column 1)]]></expected-error>
+ <expected-error><![CDATA[ASX1014: Field 'unknown_field_2' is not found (in line 25, at column 1)]]></expected-error>
+ <expected-error><![CDATA[ASX1001: Syntax error: In line 28 >> as select r from range(1,2) r;<< Encountered "as" at column 3]]></expected-error>
+ <expected-error><![CDATA[ASX0013: Duplicate field name 'r' (in line 25, at column 20)]]></expected-error>
+ <expected-error><![CDATA[ASX1167: Cannot change primary key of view test1.employee_v1 (in line 38, at column 1)]]></expected-error>
+ <expected-error><![CDATA[ASX1162: Invalid primary key definition (in line 25, at column 1)]]></expected-error>
+ <expected-error><![CDATA[ASX1162: Invalid primary key definition (in line 26, at column 1)]]></expected-error>
+ <expected-error><![CDATA[ASX1001: Syntax error: In line 36 >> as employee;<< Encountered "as" at column 3]]></expected-error>
+ <expected-error><![CDATA[ASX1165: Invalid foreign key definition: view test1.employee_v1 does not have a primary key (in line 32, at column 1)]]></expected-error>
+ <expected-error><![CDATA[ASX1166: Invalid foreign key definition: foreign key does not match primary key of view test1.employee_v1 (in line 34, at column 1)]]></expected-error>
+ <expected-error><![CDATA[ASX1166: Invalid foreign key definition: foreign key does not match primary key of view test1.employee_v2 (in line 34, at column 1)]]></expected-error>
+ <expected-error><![CDATA[ASX1164: Invalid foreign key definition (in line 34, at column 1)]]></expected-error>
+ <expected-error><![CDATA[ASX1063: Cannot find dataverse with name test3 (in line 42, at column 1)]]></expected-error>
+ <expected-error><![CDATA[ASX1159: Cannot find view with name test1.employee_v3 (in line 42, at column 1)]]></expected-error>
+ <expected-error><![CDATA[ASX1159: Cannot find view with name test1.employee (in line 43, at column 1)]]></expected-error>
+ <expected-error><![CDATA[ASX1164: Invalid foreign key definition (in line 43, at column 1)]]></expected-error>
+ <expected-error><![CDATA[ASX1164: Invalid foreign key definition (in line 39, at column 1)]]></expected-error>
+ <expected-error><![CDATA[ASX1166: Invalid foreign key definition: foreign key does not match primary key of view test1.employee_v1 (in line 39, at column 1)]]></expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="view">
+ <compilation-unit name="create-view-7-foreign-key">
+ <output-dir compare="Text">create-view-7-foreign-key</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="view">
+ <compilation-unit name="drop-dataverse-1">
+ <output-dir compare="Text">drop-dataverse-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="view">
+ <compilation-unit name="drop-dataverse-2-negative">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>ASX1147: Cannot drop dataverse: dataset (or view) test2.v2 being used by view test1.v1</expected-error>
+ <expected-error>ASX1147: Cannot drop dataverse: dataset (or view) test2.v2 being used by function test1.f1()</expected-error>
+ <expected-error>ASX1147: Cannot drop dataverse: dataset (or view) test2.ds2 being used by view test1.v1</expected-error>
+ <expected-error>ASX1147: Cannot drop dataverse: function test2.f2() being used by view test1.v1</expected-error>
+ <expected-error>ASX1147: Cannot drop dataverse: synonym test2.s3 being used by view test1.v1</expected-error>
+ <expected-error>ASX1147: Cannot drop dataverse: type test2.t1 being used by dataset test1.v1</expected-error>
+ <expected-error>ASX1147: Cannot drop dataverse: dataset (or view) test2.employee_v2 being used by view test1.employee_v1</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="view">
+ <compilation-unit name="drop-view-1">
+ <output-dir compare="Text">drop-view-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="view">
+ <compilation-unit name="drop-view-2-negative">
+ <output-dir compare="Text">drop-view-2-negative</output-dir>
+ <expected-error>ASX1063: Cannot find dataverse with name test (in line 24, at column 1)</expected-error>
+ <expected-error>ASX1159: Cannot find view with name test.v1 (in line 25, at column 1)</expected-error>
+ <expected-error>ASX1050: Cannot find dataset with name v1 in dataverse test (in line 27, at column 1)</expected-error>
+ <expected-error>ASX1159: Cannot find view with name test.ds1 (in line 30, at column 1)</expected-error>
+ <expected-error>ASX1148: Cannot drop dataset test2.ds2 being used by view test1.v1</expected-error>
+ <expected-error>ASX1148: Cannot drop function test2.f2() being used by view test1.v1</expected-error>
+ <expected-error>ASX1148: Cannot drop synonym test2.s2 being used by view test1.v1</expected-error>
+ <expected-error>ASX1148: Cannot drop view test2.v2 being used by view test1.v1</expected-error>
+ <expected-error>ASX1148: Cannot drop view test2.v2 being used by function test1.f1()</expected-error>
+ <expected-error>ASX1148: Cannot drop type test2.t1 being used by dataset test1.v1</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="view">
+ <compilation-unit name="view-1">
+ <output-dir compare="Text">view-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="view">
+ <compilation-unit name="view-2-negative">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>ASX1050: Cannot find dataset with name v1 in dataverse test1 (in line 24, at column 17)</expected-error>
+ <expected-error>ASX1050: Cannot find dataset with name v2 in dataverse test1 (in line 24, at column 17)</expected-error>
+ <expected-error>ASX1050: Cannot find dataset with name v3 in dataverse test1 (in line 24, at column 1)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="view">
+ <compilation-unit name="view-pushdown">
+ <output-dir compare="Text">view-pushdown</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="load">
+ <test-case FilePath="load">
+ <compilation-unit name="load_non-empty_index">
+ <output-dir compare="Text">load_non-empty_index</output-dir>
+ <expected-error>HYR0034: Cannot load an index that is not empty</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="load">
+ <compilation-unit name="csv_01">
+ <output-dir compare="Text">csv_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="load">
+ <compilation-unit name="csv_02">
+ <output-dir compare="Text">csv_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="load">
+ <compilation-unit name="csv_03">
+ <output-dir compare="Text">csv_03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="load">
+ <compilation-unit name="csv_04">
+ <output-dir compare="Text">csv_04</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="load">
+ <compilation-unit name="csv_05">
+ <output-dir compare="Text">csv_05</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="load">
+ <compilation-unit name="csv_06">
+ <output-dir compare="Text">csv_06</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="load">
+ <compilation-unit name="csv_07">
+ <output-dir compare="Text">csv_07</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="load">
+ <compilation-unit name="csv_08_header_cr">
+ <output-dir compare="Text">csv_08</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="load">
+ <compilation-unit name="csv_08_header_lf">
+ <output-dir compare="Text">csv_08</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="load">
+ <compilation-unit name="csv_08_header_crlf">
+ <output-dir compare="Text">csv_08</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="load">
+ <compilation-unit name="issue14_query">
+ <output-dir compare="Text">issue14_query</output-dir>
+ <expected-error>Unspecified parameter: format</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="load">
+ <compilation-unit name="issue315_query">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>Invalid path</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="load">
+ <compilation-unit name="issue289_query">
+ <output-dir compare="Text">issue289_query</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="load">
+ <compilation-unit name="issue650_query">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>Cannot find dataset with name Users in dataverse fuzzyjoin</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="load">
+ <compilation-unit name="type_promotion_0">
+ <output-dir compare="Text">type_promotion_0</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="load">
+ <compilation-unit name="escapes01">
+ <output-dir compare="Text">escapes01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="load">
+ <compilation-unit name="escapes02">
+ <output-dir compare="Text">escapes02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="load">
+ <compilation-unit name="escapes-err-1"><!-- Exception is never thrown!!!. needs to be investigated -->
+ <output-dir compare="Text">none</output-dir>
+ <!-- <expected-error>org.apache.hyracks.api.exceptions.HyracksException</expected-error> -->
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="user-defined-functions">
+ <compilation-unit name="query-issue244">
+ <output-dir compare="Text">query-issue244</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="load">
+ <compilation-unit name="duplicate-key-error">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>Loading duplicate keys into the primary storage</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="load">
+ <compilation-unit name="issue610_adm_token_end_collection">
+ <output-dir compare="Text">issue610_adm_token_end_collection</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="load">
+ <compilation-unit name="adm_binary">
+ <output-dir compare="Text">adm_binary</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="load">
+ <compilation-unit name="dataset-with-meta">
+ <output-dir compare="Text">dataset-with-meta</output-dir>
+ <expected-error>ASX1079: Compilation error: DatasetWithMeta: load dataset is not supported on datasets with meta records (in line 27, at column 1)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="load">
+ <compilation-unit name="escapes-err-1">
+ <output-dir compare="Text">escapes-err-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="load">
+ <compilation-unit name="escapes01">
+ <output-dir compare="Text">escapes01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="load">
+ <compilation-unit name="escapes02">
+ <output-dir compare="Text">escapes02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="load">
+ <compilation-unit name="file-not-found">
+ <output-dir compare="Text">file-not-found</output-dir>
+ <expected-error>ASX3077: bla: path not found</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="load">
+ <compilation-unit name="type_promotion_0">
+ <output-dir compare="Text">type_promotion_0</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="load">
+ <compilation-unit name="utf8">
+ <output-dir compare="Text">utf8</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="hints">
+ <test-case FilePath="hints">
+ <compilation-unit name="issue_251_dataset_hint_5">
+ <output-dir compare="Text">issue_251_dataset_hint_5</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="hints">
+ <compilation-unit name="issue_251_dataset_hint_7">
+ <output-dir compare="Text">issue_251_dataset_hint_7</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="function">
+ <test-case FilePath="function">
+ <compilation-unit name="issue-2394">
+ <output-dir compare="Text">issue-2394</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="function">
+ <compilation-unit name="drop_if_exists">
+ <output-dir compare="Text">drop_if_exists</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="feeds">
+ <test-case FilePath="feeds">
+ <compilation-unit name="feeds_07">
+ <output-dir compare="Text">feeds_07</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="feeds_08">
+ <output-dir compare="Text">feeds_08</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="feeds_09">
+ <output-dir compare="Text">feeds_09</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="create-policy-from-file">
+ <output-dir compare="Text">create-policy-from-file</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="feeds_01">
+ <output-dir compare="Text">feeds_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="feeds_02">
+ <output-dir compare="Text">feeds_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="feeds_03">
+ <output-dir compare="Text">feeds_03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="feeds_10">
+ <output-dir compare="Text">feeds_10</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="feeds_11">
+ <output-dir compare="Text">feeds_11</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="feeds_12">
+ <output-dir compare="Text">feeds_12</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="feeds_13">
+ <output-dir compare="Text">feeds_13</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="issue_230_feeds">
+ <output-dir compare="Text">issue_230_feeds</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="insert-feed">
+ <output-dir compare="Text">insert-feed</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="insert-feed-with-pk-index">
+ <output-dir compare="Text">insert-feed</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="connect-feed-with-function">
+ <output-dir compare="Text">connect-feed-with-function</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="change-feed-filter-on-meta-dataset">
+ <output-dir compare="Text">change-feed-filter-on-meta-dataset</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="change-feed-with-meta-pk-index">
+ <output-dir compare="Text">change-feed-with-meta-pk-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="change-feed-with-meta-with-mixed-index">
+ <output-dir compare="Text">change-feed-with-meta-with-mixed-index</output-dir>
+ <expected-error>Compilation error: Cannot create index on meta fields (in line 61, at column 35)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="change-feed-with-meta-pk-in-meta-index-with-missing-after-ingest">
+ <output-dir compare="Text">change-feed-with-meta-pk-in-meta-index-with-missing-after-ingest</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="change-feed-with-meta-pk-in-meta-open-index-with-missing">
+ <output-dir compare="Text">change-feed-with-meta-pk-in-meta-open-index-with-missing</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="change-feed-with-meta-open-index-in-meta">
+ <output-dir compare="Text">change-feed-with-meta-open-index-in-meta</output-dir>
+ <expected-error>Compilation error: Cannot create index on meta fields (in line 60, at column 34)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="change-feed-with-meta-pk-in-meta-open-index-in-value">
+ <output-dir compare="Text">change-feed-with-meta-pk-in-meta-open-index-in-value</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="change-feed-with-meta-pk-in-meta-index-after-ingest">
+ <output-dir compare="Text">change-feed-with-meta-pk-in-meta-index-after-ingest</output-dir>
+ <expected-error>Compilation error: Cannot create index on meta fields (in line 27, at column 37)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="change-feed-with-meta-pk-in-meta-index-in-meta">
+ <output-dir compare="Text">change-feed-with-meta-pk-in-meta-index-in-meta</output-dir>
+ <expected-error>Compilation error: Cannot create index on meta fields (in line 60, at column 37)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="change-feed-with-meta-csv">
+ <output-dir compare="Text">change-feed-with-meta-csv</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="change-feed-with-meta-pk-in-meta">
+ <output-dir compare="Text">change-feed-with-meta-pk-in-meta</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="change-feed">
+ <output-dir compare="Text">change-feed</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="drop-nonexistent-feed">
+ <output-dir compare="Text">drop-nonexistent-feed</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="twitter-feed">
+ <output-dir compare="Text">twitter-feed</output-dir>
+ <expected-error>Twitter4J library not found!</expected-error>
+ <expected-error>Unknown source feed: TwitterFeed</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="revised-tweet-parser">
+ <output-dir compare="Text">revised-tweet-parser</output-dir>
+ <expected-error>Twitter4J library not found!</expected-error>
+ <expected-error>Unknown source feed: TwitterFeed</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="connect-feed">
+ <output-dir compare="Text">connect-feed</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="change-feed-with-meta-pk-in-meta">
+ <output-dir compare="Text">change-feed-with-meta-pk-in-meta</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="feed-with-filtered-dataset">
+ <output-dir compare="Text">feed-with-filtered-dataset</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="change-feed">
+ <output-dir compare="Text">change-feed</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="drop-dataverse-with-disconnected-feed">
+ <output-dir compare="Text">drop-dataverse-with-disconnected-feed</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="feed-push-socket">
+ <output-dir compare="Text">feed-push-socket</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="insert-feed">
+ <output-dir compare="Text">insert-feed</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="start-feed">
+ <output-dir compare="Text">start-feed</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="start-started-feed">
+ <output-dir compare="Text">start-started-feed</output-dir>
+ <expected-error>experiments.TweetFeed(Feed) is already started</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="stop-stopped-feed">
+ <output-dir compare="Text">stop-stopped-feed</output-dir>
+ <expected-error>new_experiments.TweetFeed(Feed) cannot be stopped because its state is STOPPED</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="push-socket-with-auuid">
+ <output-dir compare="Text">push-socket-with-auuid</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="disconnect-live-feed">
+ <output-dir compare="Text">disconnect-live-feed</output-dir>
+ <expected-error>This operation cannot be done when Feed</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="connect-live-feed">
+ <output-dir compare="Text">connect-live-feed</output-dir>
+ <expected-error>This operation cannot be done when Feed</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="record-reader-with-malformed-input-stream">
+ <output-dir compare="Text">record-reader-with-malformed-input-stream</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="feed-with-undefined-function">
+ <output-dir compare="Text">feed-with-undefined-function</output-dir>
+ <expected-error>Cannot find function</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="drop-function-used-by-feed">
+ <output-dir compare="Text">drop-function-used-by-feed</output-dir>
+ <expected-error>ASX1148: Cannot drop function experiments.test_func0(1) being used by feed connection experiments.UserFeed</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="drop-function-no-longer-used-by-feed">
+ <output-dir compare="Text">drop-function-used-by-feed</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="drop-dataverse-with-function-used-by-feed">
+ <output-dir compare="Text">drop-dataverse-with-function-used-by-feed</output-dir>
+ <expected-error>ASX1147: Cannot drop dataverse: function fundv.test_func0(1) being used by feed connection feeddv.UserFeed</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="http_feed">
+ <output-dir compare="Text">http_feed</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="http_feed_json">
+ <output-dir compare="Text">http_feed_json</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="change-feed-with-where-on-meta">
+ <output-dir compare="Text">change-feed-with-where-on-meta</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="meta">
+ <test-case FilePath="meta">
+ <compilation-unit name="meta_in_with_clause">
+ <output-dir compare="Text">meta_in_with_clause</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="meta">
+ <compilation-unit name="resolving_pk_with_meta">
+ <output-dir compare="Text">resolving_pk_with_meta</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="meta">
+ <compilation-unit name="meta_after_gby">
+ <output-dir compare="Text">meta_after_gby</output-dir>
+ <expected-error>Compilation error: Inappropriate use of function 'meta'. For example, after GROUP BY (in line 29, at column 21)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="meta">
+ <compilation-unit name="query_dataset_with_meta-1">
+ <output-dir compare="Text">query_dataset_with_meta-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="meta">
+ <compilation-unit name="query_dataset_with_meta-2">
+ <output-dir compare="Text">query_dataset_with_meta-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="meta">
+ <compilation-unit name="query_dataset_with_meta_failure">
+ <output-dir compare="Text">query_dataset_with_meta_failure</output-dir>
+ <expected-error>ASX1079: Compilation error: Cannot resolve ambiguous meta function call. There are more than one dataset choice (in line 24, at column 7)</expected-error>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="big-object">
+ <test-case FilePath="big-object">
+ <compilation-unit name="big_object_sort">
+ <output-dir compare="Text">big_object_sort</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="big-object">
+ <compilation-unit name="big_object_groupby">
+ <output-dir compare="Text">big_object_groupby</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="big-object">
+ <compilation-unit name="big_object_groupby-2">
+ <output-dir compare="Text">big_object_groupby-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="big-object">
+ <compilation-unit name="big_object_join">
+ <output-dir compare="Text">big_object_join</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="big-object">
+ <compilation-unit name="big_object_join_low_memory_err">
+ <output-dir compare="Text">big_object_join</output-dir>
+ <expected-error>HYR0123: Insufficient memory is provided for the join operators, please increase the join memory budget.</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="big-object">
+ <compilation-unit name="big_object_load_20M">
+ <output-dir compare="Text">big_object_load_20M</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="big-object">
+ <compilation-unit name="big_object_bulkload">
+ <output-dir compare="Text">big_object_bulkload</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="big-object">
+ <compilation-unit name="big_object_feed_20M">
+ <output-dir compare="Text">big_object_feed_20M</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="big-object">
+ <compilation-unit name="big_object_insert">
+ <output-dir compare="Text">big_object_insert</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="big-object">
+ <compilation-unit name="big_object_load_only_20M">
+ <output-dir compare="Text">big_object_load_only_20M</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="leftouterjoin">
+ <test-case FilePath="leftouterjoin">
+ <compilation-unit name="loj-01-core">
+ <output-dir compare="Text">loj-01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="leftouterjoin">
+ <compilation-unit name="loj-01-sugar">
+ <output-dir compare="Text">loj-01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="leftouterjoin">
+ <compilation-unit name="loj-02-push-select">
+ <output-dir compare="Text">loj-02-push-select</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="leftouterjoin">
+ <compilation-unit name="loj-03-no-listify">
+ <output-dir compare="Text">loj-03-no-listify</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="leftouterjoin">
+ <compilation-unit name="query_issue658">
+ <output-dir compare="Text">query_issue658</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="leftouterjoin">
+ <compilation-unit name="query_issue285">
+ <output-dir compare="Text">query_issue285</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="leftouterjoin">
+ <compilation-unit name="query_issue285-2">
+ <output-dir compare="Text">query_issue285-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="leftouterjoin">
+ <compilation-unit name="query_issue849">
+ <output-dir compare="Text">query_issue849</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="leftouterjoin">
+ <compilation-unit name="query_issue849-2">
+ <output-dir compare="Text">query_issue849-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="leftouterjoin">
+ <compilation-unit name="empty-dataset">
+ <output-dir compare="Text">empty-dataset</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="leftouterjoin">
+ <compilation-unit name="query-ASTERIXDB-769">
+ <output-dir compare="Text">query-ASTERIXDB-769</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="leftouterjoin">
+ <compilation-unit name="query-ASTERIXDB-2857">
+ <output-dir compare="Text">query-ASTERIXDB-2857</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="leftouterjoin">
+ <compilation-unit name="right_branch_opt_1">
+ <output-dir compare="Text">right_branch_opt_1</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="index-leftouterjoin">
+ <test-case FilePath="index-leftouterjoin">
+ <compilation-unit name="probe-pidx-with-join-btree-sidx1">
+ <output-dir compare="Text">probe-pidx-with-join-btree-sidx1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-leftouterjoin">
+ <compilation-unit name="probe-pidx-with-join-btree-sidx2">
+ <output-dir compare="Text">probe-pidx-with-join-btree-sidx2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-leftouterjoin">
+ <compilation-unit name="probe-pidx-join-btree-sidx3-idxonly">
+ <output-dir compare="Text">probe-pidx-join-btree-sidx3-idxonly</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-leftouterjoin">
+ <compilation-unit name="probe-pidx-with-join-rtree-sidx1">
+ <output-dir compare="Text">probe-pidx-with-join-rtree-sidx1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-leftouterjoin">
+ <compilation-unit name="probe-pidx-with-join-rtree-sidx2">
+ <output-dir compare="Text">probe-pidx-with-join-rtree-sidx2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-leftouterjoin">
+ <compilation-unit name="probe-pidx-with-join-invidx-sidx1">
+ <output-dir compare="Text">probe-pidx-with-join-invidx-sidx1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-leftouterjoin">
+ <compilation-unit name="probe-pidx-with-join-invidx-sidx2">
+ <output-dir compare="Text">probe-pidx-with-join-invidx-sidx2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-leftouterjoin">
+ <compilation-unit name="probe-sidx-btree-idxonly-join-btree-pidx1">
+ <output-dir compare="Text">probe-sidx-btree-idxonly-join-btree-pidx1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-leftouterjoin">
+ <compilation-unit name="probe-sidx-btree-idxonly-join-btree-sidx1-idxonly">
+ <output-dir compare="Text">probe-sidx-btree-idxonly-join-btree-sidx1-idxonly</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-leftouterjoin">
+ <compilation-unit name="probe-sidx-btree-non-idxonly-join-btree-pidx1">
+ <output-dir compare="Text">probe-sidx-btree-non-idxonly-join-btree-pidx1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-leftouterjoin">
+ <compilation-unit name="probe-sidx-btree-non-idxonly-join-btree-sidx1-idxonly">
+ <output-dir compare="Text">probe-sidx-btree-non-idxonly-join-btree-sidx1-idxonly</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-leftouterjoin">
+ <compilation-unit name="probe-pidx-with-join-btree-pidx1">
+ <output-dir compare="Text">probe-pidx-with-join-btree-pidx1</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="distinct">
+ <test-case FilePath="distinct">
+ <compilation-unit name="array">
+ <output-dir compare="Text">array</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="distinct">
+ <compilation-unit name="record">
+ <output-dir compare="Text">record</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="distinct">
+ <compilation-unit name="query-issue443">
+ <output-dir compare="Text">query-issue443</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="distinct">
+ <compilation-unit name="query-issue443-2">
+ <output-dir compare="Text">query-issue443-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="rightouterjoin">
+ <test-case FilePath="rightouterjoin">
+ <compilation-unit name="roj-01-core">
+ <output-dir compare="Text">roj-01-core</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="rightouterjoin">
+ <compilation-unit name="roj-02-core">
+ <output-dir compare="Text">roj-02-core</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="rightouterjoin">
+ <compilation-unit name="roj-03-negative">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>ASX1130: Illegal use of RIGHT OUTER JOIN</expected-error>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="tinysocial">
+ <test-case FilePath="tinysocial">
+ <compilation-unit name="tinysocial-suite">
+ <output-dir compare="Text">tinysocial-suite</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tinysocial">
+ <compilation-unit name="tinysocial-suite-open">
+ <output-dir compare="Text">tinysocial-suite-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="types">
+ <test-case FilePath="types">
+ <compilation-unit name="any-object">
+ <output-dir compare="Text">any-object</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="gettype">
+ <output-dir compare="Text">gettype</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="isarray">
+ <output-dir compare="Text">isarray</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="isatomic">
+ <output-dir compare="Text">isatomic</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="isboolean">
+ <output-dir compare="Text">isboolean</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="isnumber">
+ <output-dir compare="Text">isnumber</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="isobject">
+ <output-dir compare="Text">isobject</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="isstring">
+ <output-dir compare="Text">isstring</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="isbinary">
+ <output-dir compare="Text">isbinary</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="iscircle">
+ <output-dir compare="Text">iscircle</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="isdate">
+ <output-dir compare="Text">isdate</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="isdatetime">
+ <output-dir compare="Text">isdatetime</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="isduration">
+ <output-dir compare="Text">isduration</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="isinterval">
+ <output-dir compare="Text">isinterval</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="isline">
+ <output-dir compare="Text">isline</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="ismultiset">
+ <output-dir compare="Text">ismultiset</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="ispoint">
+ <output-dir compare="Text">ispoint</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="ispolygon">
+ <output-dir compare="Text">ispolygon</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="isrectangle">
+ <output-dir compare="Text">isrectangle</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="isspatial">
+ <output-dir compare="Text">isspatial</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="istemporal">
+ <output-dir compare="Text">istemporal</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="istime">
+ <output-dir compare="Text">istime</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="isuuid">
+ <output-dir compare="Text">isuuid</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="is_all_types">
+ <output-dir compare="Text">is_all_types</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="record01">
+ <output-dir compare="Text">record01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="type_promotion_0">
+ <output-dir compare="Text">type_promotion_0</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="type_promotion_1">
+ <output-dir compare="Text">type_promotion_1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="opentype_orderby_01">
+ <output-dir compare="Text">opentype_orderby_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="promotion_closedtype_field_vs_closedtype_field_01">
+ <output-dir compare="Text">promotion_closedtype_field_vs_closedtype_field_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="promotion_closedtype_field_vs_closedtype_field_02">
+ <output-dir compare="Text">promotion_closedtype_field_vs_closedtype_field_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="promotion_closedtype_field_vs_closedtype_field_03">
+ <output-dir compare="Text">promotion_closedtype_field_vs_closedtype_field_03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="promotion_closedtype_field_vs_closedtype_field_04">
+ <output-dir compare="Text">promotion_closedtype_field_vs_closedtype_field_04</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="promotion_closedtype_field_vs_closedtype_field_05">
+ <output-dir compare="Text">promotion_closedtype_field_vs_closedtype_field_05</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="promotion_closedtype_field_vs_constant_01">
+ <output-dir compare="Text">promotion_closedtype_field_vs_constant_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="promotion_closedtype_field_vs_constant_02">
+ <output-dir compare="Text">promotion_closedtype_field_vs_constant_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="promotion_closedtype_field_vs_constant_03">
+ <output-dir compare="Text">promotion_closedtype_field_vs_constant_03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="promotion_closedtype_field_vs_constant_04">
+ <output-dir compare="Text">promotion_closedtype_field_vs_constant_04</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="promotion_closedtype_field_vs_constant_05">
+ <output-dir compare="Text">promotion_closedtype_field_vs_constant_05</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="promotion_closedtype_field_vs_constant_06">
+ <output-dir compare="Text">promotion_closedtype_field_vs_constant_06</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="promotion_closedtype_field_vs_constant_07">
+ <output-dir compare="Text">promotion_closedtype_field_vs_constant_07</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="promotion_closedtype_field_vs_constant_08">
+ <output-dir compare="Text">promotion_closedtype_field_vs_constant_08</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="promotion_closedtype_field_vs_constant_09">
+ <output-dir compare="Text">promotion_closedtype_field_vs_constant_09</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="promotion_closedtype_field_vs_constant_10">
+ <output-dir compare="Text">promotion_closedtype_field_vs_constant_10</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="promotion_closedtype_field_vs_constant_11">
+ <output-dir compare="Text">promotion_closedtype_field_vs_constant_11</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="promotion_closedtype_field_vs_opentype_field_01">
+ <output-dir compare="Text">promotion_closedtype_field_vs_opentype_field_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="promotion_closedtype_field_vs_opentype_field_02">
+ <output-dir compare="Text">promotion_closedtype_field_vs_opentype_field_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="promotion_closedtype_field_vs_opentype_field_03">
+ <output-dir compare="Text">promotion_closedtype_field_vs_opentype_field_03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="promotion_closedtype_field_vs_opentype_field_04">
+ <output-dir compare="Text">promotion_closedtype_field_vs_opentype_field_04</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="promotion_opentype_field_vs_constant_01">
+ <output-dir compare="Text">promotion_opentype_field_vs_constant_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="promotion_opentype_field_vs_constant_02">
+ <output-dir compare="Text">promotion_opentype_field_vs_constant_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="promotion_opentype_field_vs_constant_03">
+ <output-dir compare="Text">promotion_opentype_field_vs_constant_03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="promotion_opentype_field_vs_constant_04">
+ <output-dir compare="Text">promotion_opentype_field_vs_constant_04</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="promotion_opentype_field_vs_constant_05">
+ <output-dir compare="Text">promotion_opentype_field_vs_constant_05</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="promotion_opentype_field_vs_constant_06">
+ <output-dir compare="Text">promotion_opentype_field_vs_constant_06</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="promotion_opentype_field_vs_constant_07">
+ <output-dir compare="Text">promotion_opentype_field_vs_constant_07</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="promotion_opentype_field_vs_constant_08">
+ <output-dir compare="Text">promotion_opentype_field_vs_constant_08</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="promotion_opentype_field_vs_opentype_field_01">
+ <output-dir compare="Text">promotion_opentype_field_vs_opentype_field_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="promotion_opentype_field_vs_opentype_field_02">
+ <output-dir compare="Text">promotion_opentype_field_vs_opentype_field_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="to_array">
+ <output-dir compare="Text">to_array</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="to_atomic">
+ <output-dir compare="Text">to_atomic</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="to_boolean_01">
+ <output-dir compare="Text">to_boolean_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types" check-warnings="true">
+ <compilation-unit name="to_boolean_02">
+ <output-dir compare="Text">to_boolean_02</output-dir>
+ <expected-warn>ASX0004: Unsupported type: to-boolean() cannot process input type date (in line 24, at column 8)</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="to_bigint_01">
+ <output-dir compare="Text">to_bigint_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types" check-warnings="true">
+ <compilation-unit name="to_bigint_02">
+ <output-dir compare="Text">to_bigint_02</output-dir>
+ <expected-warn>ASX0004: Unsupported type: to-bigint() cannot process input type date (in line 24, at column 7)</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="to_double_01">
+ <output-dir compare="Text">to_double_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types" check-warnings="true">
+ <compilation-unit name="to_double_02">
+ <output-dir compare="Text">to_double_02</output-dir>
+ <expected-warn>ASX0004: Unsupported type: to-double() cannot process input type date (in line 24, at column 7)</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="to_number_01">
+ <output-dir compare="Text">to_number_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="to_number_02">
+ <output-dir compare="Text">to_number_02</output-dir>
+ <expected-error>ASX0002: Type mismatch</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="to_object">
+ <output-dir compare="Text">to_object</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="to_string_01">
+ <output-dir compare="Text">to_string_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="to_string_02">
+ <output-dir compare="Text">to_string_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="domain_boundaries">
+ <output-dir compare="Text">domain_boundaries</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="domain_boundaries_error">
+ <output-dir compare="Text">domain_boundaries_error</output-dir>
+ <expected-error>ASX0001: Field type null cannot be promoted to type tinyint</expected-error>
+ <expected-error>ASX0001: Field type null cannot be promoted to type smallint</expected-error>
+ <expected-error>ASX0001: Field type null cannot be promoted to type integer</expected-error>
+ <expected-error>ASX0001: Field type null cannot be promoted to type bigint</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="query-ASTERIXDB-2950">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>ASX1002: Type mismatch: function get-item expects its 1st input parameter to be of type multiset or array, but the actual input type is bigint (in line 24, at column 21)</expected-error>
+ <expected-error>ASX1002: Type mismatch: function get-item expects its 1st input parameter to be of type multiset or array, but the actual input type is bigint (in line 25, at column 31)</expected-error>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="cleanjson">
+ <test-case FilePath="json">
+ <compilation-unit name="issue-ASTERIXDB-1165">
+ <output-dir compare="Clean-JSON">issue-ASTERIXDB-1165</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="json">
+ <compilation-unit name="int01">
+ <output-dir compare="Clean-JSON">int01-cleanjson</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="materialization">
+ <test-case FilePath="materialization">
+ <compilation-unit name="assign-reuse">
+ <output-dir compare="Text">assign-reuse</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="filters">
+ <test-case FilePath="filters">
+ <compilation-unit name="equality-predicate">
+ <output-dir compare="Text">equality-predicate</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="filters">
+ <compilation-unit name="filter-auto-key">
+ <output-dir compare="Text">filter-auto-key</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="filters">
+ <compilation-unit name="load">
+ <output-dir compare="Text">load</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="filters">
+ <compilation-unit name="load-with-secondary-btree">
+ <output-dir compare="Text">load-with-secondary-btree</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="filters">
+ <compilation-unit name="load-with-secondary-btree-index-only">
+ <output-dir compare="Text">load-with-secondary-btree-index-only</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="filters">
+ <compilation-unit name="load-with-secondary-inverted-ngram">
+ <output-dir compare="Text">load-with-secondary-inverted-ngram</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="filters">
+ <compilation-unit name="load-with-secondary-inverted-word">
+ <output-dir compare="Text">load-with-secondary-inverted-word</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="filters">
+ <compilation-unit name="load-with-secondary-rtree">
+ <output-dir compare="Text">load-with-secondary-rtree</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="filters">
+ <compilation-unit name="insert">
+ <output-dir compare="Text">insert</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="filters">
+ <compilation-unit name="insert-with-secondary-btree">
+ <output-dir compare="Text">insert-with-secondary-btree</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="filters">
+ <compilation-unit name="insert-with-correlated-secondary-btree">
+ <output-dir compare="Text">insert-with-secondary-btree</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="filters">
+ <compilation-unit name="insert-with-secondary-inverted-ngram">
+ <output-dir compare="Text">insert-with-secondary-inverted-ngram</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="filters">
+ <compilation-unit name="insert-with-correlated-secondary-inverted-ngram">
+ <output-dir compare="Text">insert-with-secondary-inverted-ngram</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="filters">
+ <compilation-unit name="insert-with-secondary-inverted-word">
+ <output-dir compare="Text">insert-with-secondary-inverted-word</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="filters">
+ <compilation-unit name="insert-with-correlated-secondary-inverted-word">
+ <output-dir compare="Text">insert-with-secondary-inverted-word</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="filters">
+ <compilation-unit name="insert-with-secondary-rtree">
+ <output-dir compare="Text">insert-with-secondary-rtree</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="filters">
+ <compilation-unit name="insert-with-correlated-secondary-rtree">
+ <output-dir compare="Text">insert-with-secondary-rtree</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="filters">
+ <compilation-unit name="nested-filter-equality-predicate">
+ <output-dir compare="Text">nested-filter-equality-predicate</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="filters">
+ <compilation-unit name="upsert">
+ <output-dir compare="Text">upsert</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="filters">
+ <compilation-unit name="delete">
+ <output-dir compare="Text">delete</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="json">
+ <test-case FilePath="json">
+ <compilation-unit name="int01">
+ <output-dir compare="Lossless-JSON">int01-losslessjson</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="csv">
+ <test-case FilePath="csv">
+ <compilation-unit name="basic-types">
+ <output-dir compare="CSV">basic-types</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="csv">
+ <compilation-unit name="basic-types">
+ <output-dir compare="CSV_Header">basic-types-header</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="csv-tsv-parser">
+ <test-case FilePath="csv-tsv-parser">
+ <compilation-unit name="csv-parser-001">
+ <output-dir compare="Text">csv-parser-001</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="csv-tsv-parser">
+ <compilation-unit name="tsv-parser-001">
+ <output-dir compare="Text">tsv-parser-001</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="binary">
+ <test-case FilePath="binary">
+ <compilation-unit name="parse">
+ <output-dir compare="Text">parse</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="binary">
+ <compilation-unit name="print">
+ <output-dir compare="Text">print</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="binary">
+ <compilation-unit name="concat">
+ <output-dir compare="Text">concat</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="binary">
+ <compilation-unit name="concat2">
+ <output-dir compare="Text">concat2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="binary">
+ <compilation-unit name="subbinary">
+ <output-dir compare="Text">subbinary</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="binary">
+ <compilation-unit name="find">
+ <output-dir compare="Text">find</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="binary">
+ <compilation-unit name="insert">
+ <output-dir compare="Text">insert</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="binary">
+ <compilation-unit name="equal_join">
+ <output-dir compare="Text">equal_join</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="binary">
+ <compilation-unit name="index_join">
+ <output-dir compare="Text">index_join</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="binary">
+ <compilation-unit name="length">
+ <output-dir compare="Text">length</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="unnest">
+ <test-case FilePath="unnest">
+ <compilation-unit name="ASTERIXDB-2750_unnest_join">
+ <output-dir compare="Text">ASTERIXDB-2750_unnest_join</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="unnest">
+ <compilation-unit name="ASTERIXDB-2844_unnest_syntax">
+ <output-dir compare="Text">ASTERIXDB-2844_unnest_syntax</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="unnest">
+ <compilation-unit name="left-outer-unnest">
+ <output-dir compare="Text">left-outer-unnest</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="unnest">
+ <compilation-unit name="left-outer-unnest-with-pos">
+ <output-dir compare="Text">left-outer-unnest-with-pos</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="union">
+ <test-case FilePath="union">
+ <compilation-unit name="union">
+ <output-dir compare="Text">union</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="union">
+ <compilation-unit name="union_heterogeneous_scalar">
+ <output-dir compare="Text">union_heterogeneous_scalar</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="union">
+ <compilation-unit name="union_negative">
+ <output-dir compare="Text">union</output-dir>
+ <expected-error>Cannot find dataset t in dataverse TinySocial nor an alias with name t</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="union">
+ <compilation-unit name="union_negative_3">
+ <output-dir compare="Text">union</output-dir>
+ <expected-error>Operation UNION with set semantics is not supported.</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="union">
+ <compilation-unit name="union_nested">
+ <output-dir compare="Text">union_nested</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="union">
+ <compilation-unit name="union_opt_1">
+ <output-dir compare="Text">union_opt_1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="union">
+ <compilation-unit name="union_orderby">
+ <output-dir compare="Text">union_orderby</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="union">
+ <compilation-unit name="union_orderby_2">
+ <output-dir compare="Text">union_orderby_2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="union">
+ <compilation-unit name="union_orderby_3">
+ <output-dir compare="Text">union_orderby_3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="union">
+ <compilation-unit name="union_orderby_4">
+ <output-dir compare="Text">union_orderby_3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="union">
+ <compilation-unit name="union_orderby_5">
+ <output-dir compare="Text">union_orderby_5</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="union">
+ <compilation-unit name="query-ASTERIXDB-1354-2">
+ <output-dir compare="Text">query-ASTERIXDB-1354-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="union">
+ <compilation-unit name="query-ASTERIXDB-1629">
+ <output-dir compare="Text">query-ASTERIXDB-1629</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="union">
+ <compilation-unit name="query-ASTERIXDB-1047">
+ <output-dir compare="Text">query-ASTERIXDB-1047</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="union">
+ <compilation-unit name="query-ASTERIXDB-1205-2">
+ <output-dir compare="Text">query-ASTERIXDB-1205-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="union">
+ <compilation-unit name="query-ASTERIXDB-1205-3">
+ <output-dir compare="Text">query-ASTERIXDB-1205-3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="union">
+ <compilation-unit name="query-ASTERIXDB-1205">
+ <output-dir compare="Text">query-ASTERIXDB-1205</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="union">
+ <compilation-unit name="query-ASTERIXDB-1354">
+ <output-dir compare="Text">query-ASTERIXDB-1354</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="upsert">
+ <test-case FilePath="upsert">
+ <compilation-unit name="filtered-dataset">
+ <output-dir compare="Text">filtered-dataset</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="upsert">
+ <compilation-unit name="issue1587-foreignDataType">
+ <output-dir compare="Text">issue1587-foreignDataType</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="upsert">
+ <compilation-unit name="nested-index">
+ <output-dir compare="Text">nested-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="upsert">
+ <compilation-unit name="primary-secondary-rtree">
+ <output-dir compare="Text">primary-secondary-rtree</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="upsert">
+ <compilation-unit name="primary-correlated-secondary-rtree">
+ <output-dir compare="Text">primary-secondary-rtree</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="upsert">
+ <compilation-unit name="upsert-with-self-read">
+ <output-dir compare="Text">upsert-with-self-read</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="upsert">
+ <compilation-unit name="nullable-index">
+ <output-dir compare="Text">nullable-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="upsert">
+ <compilation-unit name="open-index">
+ <output-dir compare="Text">open-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="upsert">
+ <compilation-unit name="primary-index">
+ <output-dir compare="Text">primary-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="upsert">
+ <compilation-unit name="primary-secondary-btree">
+ <output-dir compare="Text">primary-secondary-btree</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="upsert">
+ <compilation-unit name="primary-correlated-secondary-btree">
+ <output-dir compare="Text">primary-secondary-btree</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="upsert">
+ <compilation-unit name="primary-secondary-inverted">
+ <output-dir compare="Text">primary-secondary-inverted</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="upsert">
+ <compilation-unit name="primary-correlated-secondary-inverted">
+ <output-dir compare="Text">primary-secondary-inverted</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="upsert">
+ <compilation-unit name="multiple-secondaries">
+ <output-dir compare="Text">multiple-secondaries</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="upsert">
+ <compilation-unit name="multiple-correlated-secondaries">
+ <output-dir compare="Text">multiple-secondaries</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="upsert">
+ <compilation-unit name="upsert-case-returning">
+ <output-dir compare="Text">upsert-case-returning</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="json-parser">
+ <test-case FilePath="json-parser">
+ <compilation-unit name="numeric-tinyint">
+ <output-dir compare="Text">numeric-tinyint</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="json-parser">
+ <compilation-unit name="numeric-tinyint-overflow">
+ <output-dir compare="Text">numeric-tinyint</output-dir>
+ <expected-error>Numeric value (1000) out of range of Java byte</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="json-parser">
+ <compilation-unit name="numeric-float">
+ <output-dir compare="Text">numeric-float</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="json-parser">
+ <compilation-unit name="numeric-demote-double-bigint">
+ <output-dir compare="Text">numeric-demote-double-bigint</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="json-parser">
+ <compilation-unit name="numeric-promote-bigint-double">
+ <output-dir compare="Text">numeric-promote-bigint-double</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="json-parser">
+ <compilation-unit name="duplicate-fields">
+ <output-dir compare="Text">duplicate-fields</output-dir>
+ <expected-error>Duplicate field 'field'</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="json-parser">
+ <compilation-unit name="malformed-json">
+ <output-dir compare="Text">malformed-json</output-dir>
+ <expected-error>Unexpected character ('}' (code 125)): was expecting double-quote to start field name</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="json-parser">
+ <compilation-unit name="null-missing">
+ <output-dir compare="Text">null-missing</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="json-parser">
+ <compilation-unit name="nonoptional-missing">
+ <output-dir compare="Text">nonoptional-missing</output-dir>
+ <expected-error>ASX3075: Closed field missing_value has null value</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="json-parser">
+ <compilation-unit name="nonoptional-null">
+ <output-dir compare="Text">nonoptional-null</output-dir>
+ <expected-error>ASX3075: Closed field null_value has null value</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="json-parser">
+ <compilation-unit name="spatial">
+ <output-dir compare="Text">spatial</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="json-parser">
+ <compilation-unit name="spatial-line-3-points">
+ <output-dir compare="Text">spatial-line-3-points</output-dir>
+ <expected-error>Line must have 4 coordinates</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="json-parser">
+ <compilation-unit name="spatial-polygon-unclosed">
+ <output-dir compare="Text">spatial-polygon-unclosed</output-dir>
+ <expected-error>Unclosed polygon is not supported</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="json-parser">
+ <compilation-unit name="spatial-polygon-with-hole">
+ <output-dir compare="Text">spatial-polygon-with-hole</output-dir>
+ <expected-error>Only simple geometries are supported (Point, LineString and Polygon without holes)</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="json-parser">
+ <compilation-unit name="temporal">
+ <output-dir compare="Text">temporal</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="json-parser">
+ <compilation-unit name="type-mismatch">
+ <output-dir compare="Text">type-mismatch</output-dir>
+ <expected-error>ASX3054: Mismatch Type, expecting a value of type string</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="json-parser">
+ <compilation-unit name="unsupported-type-circle">
+ <output-dir compare="Text">unsupported-type-circle</output-dir>
+ <expected-error>ASX0004: Unsupported type: org.apache.asterix.external.parser.factory.JSONDataParserFactory cannot process input type circle</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="json-parser">
+ <compilation-unit name="unsupported-type-daytimeduration">
+ <output-dir compare="Text">unsupported-type-daytimeduration</output-dir>
+ <expected-error>ASX0004: Unsupported type: org.apache.asterix.external.parser.factory.JSONDataParserFactory cannot process input type daytimeduration</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="json-parser">
+ <compilation-unit name="unsupported-type-duration">
+ <output-dir compare="Text">unsupported-type-duration</output-dir>
+ <expected-error>ASX0004: Unsupported type: org.apache.asterix.external.parser.factory.JSONDataParserFactory cannot process input type duration</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="json-parser">
+ <compilation-unit name="unsupported-type-interval">
+ <output-dir compare="Text">unsupported-type-interval</output-dir>
+ <expected-error>ASX0004: Unsupported type: org.apache.asterix.external.parser.factory.JSONDataParserFactory cannot process input type interval</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="json-parser">
+ <compilation-unit name="unsupported-type-multiset">
+ <output-dir compare="Text">unsupported-type-multiset</output-dir>
+ <expected-error>ASX0004: Unsupported type: org.apache.asterix.external.parser.factory.JSONDataParserFactory cannot process input type multiset</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="json-parser">
+ <compilation-unit name="unsupported-type-point3d">
+ <output-dir compare="Text">unsupported-type-point3d</output-dir>
+ <expected-error>ASX0004: Unsupported type: org.apache.asterix.external.parser.factory.JSONDataParserFactory cannot process input type point3d</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="json-parser">
+ <compilation-unit name="unsupported-type-rectangle">
+ <output-dir compare="Text">unsupported-type-rectangle</output-dir>
+ <expected-error>ASX0004: Unsupported type: org.apache.asterix.external.parser.factory.JSONDataParserFactory cannot process input type rectangle</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="json-parser" check-warnings="true">
+ <compilation-unit name="parse-json-function">
+ <output-dir compare="Text">parse-json-function</output-dir>
+ <source-location>false</source-location>
+ <expected-warn>Type mismatch: function parse-json expects its 1st input parameter to be of type string, but the actual input type is bigint</expected-warn>
+ <expected-warn>Malformed input stream</expected-warn>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="composite-key">
+ <test-case FilePath="composite-key">
+ <compilation-unit name="query-ASTERIXDB-920">
+ <output-dir compare="Text">query-ASTERIXDB-920</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="composite-key">
+ <compilation-unit name="query-ASTERIXDB-2334">
+ <output-dir compare="Text">query-ASTERIXDB-2334</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="composite-key">
+ <compilation-unit name="composite-low-high">
+ <output-dir compare="Text">composite-low-high</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="composite-key">
+ <compilation-unit name="composite-prefix">
+ <output-dir compare="Text">composite-prefix</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="composite-key">
+ <compilation-unit name="composite-prefix-low-high">
+ <output-dir compare="Text">composite-prefix-low-high</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="limit">
+ <test-case FilePath="limit">
+ <compilation-unit name="limit_negative_value">
+ <output-dir compare="Text">limit_negative_value</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="limit">
+ <compilation-unit name="limit_type_01">
+ <output-dir compare="Text">limit_type_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="limit">
+ <compilation-unit name="limit_type_02">
+ <output-dir compare="Text">limit_type_01</output-dir>
+ <expected-error>ASX0039: Expected integer value, got 2.75 (in line 28, at column 10)</expected-error>
+ <expected-error>ASX0039: Expected integer value, got 1.75 (in line 28, at column 19)</expected-error>
+ <expected-error>ASX1091: Type mismatch: expected value of type integer, but got the value of type string (in line 28, at column 7)</expected-error>
+ <expected-error>ASX1091: Type mismatch: expected value of type integer, but got the value of type boolean (in line 28, at column 16)</expected-error>
+ <expected-error>ASX0021: Source value 9999999999 is out of range that integer can hold - integer.MAX_VALUE: 2147483647, integer.MIN_VALUE: -2147483648</expected-error>
+ <expected-error>ASX0021: Source value 8888888888 is out of range that integer can hold - integer.MAX_VALUE: 2147483647, integer.MIN_VALUE: -2147483648</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="limit">
+ <compilation-unit name="offset_without_limit">
+ <output-dir compare="Text">offset_without_limit</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="limit">
+ <compilation-unit name="push-limit-to-external-scan">
+ <output-dir compare="Text">push-limit-to-external-scan</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="limit">
+ <compilation-unit name="push-limit-to-external-scan-select">
+ <output-dir compare="Text">push-limit-to-external-scan-select</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="limit">
+ <compilation-unit name="push-limit-to-primary-scan">
+ <output-dir compare="Text">push-limit-to-primary-scan</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="limit">
+ <compilation-unit name="push-limit-to-primary-scan-select">
+ <output-dir compare="Text">push-limit-to-primary-scan-select</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="limit">
+ <compilation-unit name="push-limit-to-primary-lookup">
+ <output-dir compare="Text">push-limit-to-primary-lookup</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="limit">
+ <compilation-unit name="push-limit-to-primary-lookup-select">
+ <output-dir compare="Text">push-limit-to-primary-lookup-select</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="limit">
+ <compilation-unit name="query-ASTERIXDB-2420">
+ <output-dir compare="Text">query-ASTERIXDB-2420</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="compression">
+ <test-case FilePath="compression">
+ <compilation-unit name="incompressible-pages/large-page">
+ <output-dir compare="Text">incompressible-pages/large-page</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="compression">
+ <compilation-unit name="incompressible-pages/small-page">
+ <output-dir compare="Text">incompressible-pages/small-page</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="compression">
+ <compilation-unit name="invalid-compression-scheme">
+ <output-dir compare="Text">invalid-compression-scheme</output-dir>
+ <expected-error>ASX1096: Unknown compression scheme zip. Supported schemes are [snappy,none]</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="compression">
+ <compilation-unit name="scheme-none">
+ <output-dir compare="Text">scheme-none</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="compression">
+ <compilation-unit name="scheme-snappy">
+ <output-dir compare="Text">scheme-snappy</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="ddl-with-clause">
+ <test-case FilePath="ddl-with-clause">
+ <compilation-unit name="missing-non-optional">
+ <output-dir compare="Text">missing-non-optional</output-dir>
+ <expected-error>ASX1061: Field 'merge-policy.name' in the with clause cannot be null or missing</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="ddl-with-clause">
+ <compilation-unit name="type-mismatch">
+ <output-dir compare="Text">type-mismatch</output-dir>
+ <expected-error>ASX1060: Field 'merge-policy.parameters.max-mergable-component-size' in the with clause must be of type bigint, but found string</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="ddl-with-clause">
+ <compilation-unit name="unsupported-field">
+ <output-dir compare="Text">unsupported-field</output-dir>
+ <expected-error>ASX1059: Field(s) [unknown-field] unsupported in the with clause</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="ddl-with-clause">
+ <compilation-unit name="unsupported-subfield">
+ <output-dir compare="Text">unsupported-subfield</output-dir>
+ <expected-error>ASX1097: Subfield(s) [unknown-subfield] in 'merge-policy' unsupported in the with clause</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="metrics">
+ <test-case FilePath="metrics">
+ <compilation-unit name="full-scan">
+ <output-dir compare="Text">full-scan</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="warnings">
+ <test-case FilePath="warnings" check-warnings="true">
+ <compilation-unit name="broadcast-join-hint-warning">
+ <output-dir compare="Text">broadcast-join-hint-warning</output-dir>
+ <expected-warn><![CDATA[ASX1107: Unexpected hint: hash-bcas. "hash-bcast", "indexnl", "hashjoin", "skip-index", "use-index", "selectivity", "productivity" expected at this location (in line 36, at column 22)]]></expected-warn>
+ <expected-warn><![CDATA[ASX1132: Invalid specification for hint hash-bcast. ASX1001: Syntax error: In line 1 >>()<< Encountered ")" at column 2. (in line 36, at column 22)]]></expected-warn>
+ <expected-warn>HYR10006: Could not apply broadcast hash join hint: broadcast cn (in line 36, at column 43)</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="warnings" check-warnings="true">
+ <compilation-unit name="hashjoin-hint-warning">
+ <output-dir compare="Text">hashjoin-hint-warning</output-dir>
+ <expected-warn><![CDATA[ASX1132: Invalid specification for hint hashjoin. ASX1001: Syntax error: In line 1 >>build<< Encountered <EOF> at column 5. (in line 36, at column 22)]]></expected-warn>
+ <expected-warn><![CDATA[ASX1107: Unexpected hint: hashjon build. "hash-bcast", "indexnl", "hashjoin", "skip-index", "use-index", "selectivity", "productivity" expected at this location (in line 36, at column 22)]]></expected-warn>
+ <expected-warn><![CDATA[ASX1132: Invalid specification for hint hashjoin. ASX1001: Syntax error: In line 1 >>buil<< Encountered <EOF> at column 4. (in line 36, at column 22)]]></expected-warn>
+ <expected-warn><![CDATA[ASX1132: Invalid specification for hint hashjoin. ASX1001: Syntax error: In line 1 >>build ()<< Encountered ")" at column 8. (in line 36, at column 22)]]></expected-warn>
+ <expected-warn>HYR10006: Could not apply hash join hint: build with cn (in line 36, at column 47)</expected-warn>
+ <expected-warn><![CDATA[ASX1132: Invalid specification for hint hashjoin. ASX1001: Syntax error: In line 1 >>probe<< Encountered <EOF> at column 5. (in line 36, at column 22)]]></expected-warn>
+ <expected-warn><![CDATA[ASX1107: Unexpected hint: hashjon probe. "hash-bcast", "indexnl", "hashjoin", "skip-index", "use-index", "selectivity", "productivity" expected at this location (in line 36, at column 22)]]></expected-warn>
+ <expected-warn><![CDATA[ASX1132: Invalid specification for hint hashjoin. ASX1001: Syntax error: In line 1 >>prob<< Encountered <EOF> at column 4. (in line 36, at column 22)]]></expected-warn>
+ <expected-warn><![CDATA[ASX1132: Invalid specification for hint hashjoin. ASX1001: Syntax error: In line 1 >>probe ()<< Encountered ")" at column 8. (in line 36, at column 22)]]></expected-warn>
+ <expected-warn>HYR10006: Could not apply hash join hint: probe with cn (in line 36, at column 47)</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="warnings" check-warnings="true">
+ <compilation-unit name="cardinality-hint-warning">
+ <output-dir compare="Text">cardinality-hint-warning</output-dir>
+ <expected-warn><![CDATA[ASX1132: Invalid specification for hint selectivity. Expected selectivity value (in line 31, at column 52)]]></expected-warn>
+ <expected-warn><![CDATA[ASX1132: Invalid specification for hint selectivity. Selectivity has to be a decimal value greater than 0 and less than 1 (in line 31, at column 52)]]></expected-warn>
+ <expected-warn>HYR10006: Could not apply selectivity hint: Selectivity specified: 0.0, has to be a decimal value greater than 0 and less than 1 (in line 31, at column 73)</expected-warn>
+ <expected-warn><![CDATA[ASX1132: Invalid specification for hint selectivity. Selectivity has to be a decimal value greater than 0 and less than 1 (in line 31, at column 52)]]></expected-warn>
+ <expected-warn><![CDATA[ASX1132: Invalid specification for hint productivity. Expected productivity collection name and value (in line 31, at column 23)]]></expected-warn>
+ <expected-warn><![CDATA[ASX1132: Invalid specification for hint productivity. Invalid format for productivity values (in line 31, at column 23)]]></expected-warn>
+ <expected-warn><![CDATA[ASX1132: Invalid specification for hint productivity. Invalid format for productivity values (in line 31, at column 23)]]></expected-warn>
+ <expected-warn><![CDATA[ASX1132: Invalid specification for hint productivity. Invalid format for productivity values (in line 31, at column 23)]]></expected-warn>
+ <expected-warn><![CDATA[ASX1132: Invalid specification for hint productivity. Invalid format for productivity values (in line 31, at column 23)]]></expected-warn>
+ <expected-warn>HYR10006: Could not apply productivity hint: Productivity specified: 0.0, has to be a decimal value greater than 0 (in line 31, at column 47)</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="warnings" check-warnings="true">
+ <compilation-unit name="inapplicable-hint-warning">
+ <output-dir compare="Text">inapplicable-hint-warning</output-dir>
+ <expected-warn>HYR10006: Could not apply Group By hint: hash</expected-warn>
+ <expected-warn>ASX1107: Unexpected hint: indexnl. "hash" expected at this location</expected-warn>
+ <expected-warn>ASX1107: Unexpected hint: hash. "hash-bcast", "indexnl", "hashjoin", "skip-index", "use-index", "selectivity", "productivity" expected at this location</expected-warn>
+ <expected-warn>ASX1107: Unexpected hint: auto. "indexnl", "skip-index", "hashjoin", "use-index", "selectivity" expected at this location</expected-warn>
+ <expected-warn>ASX1107: Unexpected hint: hash. "indexnl", "range", "hashjoin", "skip-index", "spatial-partitioning", "use-index" expected at this location</expected-warn>
+ <expected-warn>ASX1107: Unexpected hint: hash. None expected at this location</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="warnings" check-warnings="true">
+ <compilation-unit name="min-max-incompatible-types">
+ <output-dir compare="Text">min-max-incompatible-types</output-dir>
+ <expected-warn>ASX0003: Type incompatibility: function min/max gets incompatible input values: bigint and string</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: min/max cannot process input type object</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="warnings" check-warnings="true">
+ <compilation-unit name="multiple-hints-warning">
+ <output-dir compare="Text">multiple-hints-warning</output-dir>
+ <expected-warn><![CDATA[ASX1132: Invalid specification for hint hashjoin. ASX1001: Syntax error: In line 1 >>build ()<< Encountered ")" at column 8. (in line 30, at column 20)]]></expected-warn>
+ <expected-warn><![CDATA[ASX1132: Invalid specification for hint productivity. Invalid format for productivity values (in line 30, at column 46)]]></expected-warn>
+ <expected-warn><![CDATA[ASX1132: Invalid specification for hint productivity. Invalid format for productivity values (in line 30, at column 45)]]></expected-warn>
+ <expected-warn><![CDATA[ASX1132: Invalid specification for hint hashjoin. ASX1001: Syntax error: In line 1 >>build ()<< Encountered ")" at column 8. (in line 30, at column 20)]]></expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="warnings" check-warnings="true">
+ <compilation-unit name="plan-warning">
+ <output-dir compare="Text">plan-warning</output-dir>
+ <expected-warn>HYR10007: Encountered a cross product join</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="warnings" check-warnings="true">
+ <compilation-unit name="unknown-hint-warning">
+ <output-dir compare="Text">unknown-hint-warning</output-dir>
+ <expected-warn>ASX1107: Unexpected hint: unknown_hint_groupby. "hash" expected at this location</expected-warn>
+ <expected-warn>ASX1107: Unexpected hint: unknown_hint_relexpr. "hash-bcast", "indexnl", "hashjoin", "skip-index", "use-index", "selectivity", "productivity" expected at this location</expected-warn>
+ <expected-warn>ASX1107: Unexpected hint: unknown_hint_between. "indexnl", "skip-index", "hashjoin", "use-index", "selectivity" expected at this location</expected-warn>
+ <expected-warn>ASX1107: Unexpected hint: unknown_hint_funcall. "indexnl", "range", "hashjoin", "skip-index", "spatial-partitioning", "use-index" expected at this location</expected-warn>
+ <expected-warn>ASX1107: Unexpected hint: unknown_hint_elsewhere. None expected at this location</expected-warn>
+ <expected-warn>ASX1107: Unexpected hint: unknown_hint_relexpr_6. "hash-bcast", "indexnl", "hashjoin", "skip-index", "use-index", "selectivity", "productivity" expected at this location</expected-warn>
+ <expected-warn>ASX1107: Unexpected hint: unknown_hint_relexpr_6. "hash-bcast", "indexnl", "hashjoin", "skip-index", "use-index", "selectivity", "productivity" expected at this location</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="warnings">
+ <compilation-unit name="warnings-limit">
+ <output-dir compare="Clean-JSON">warnings-limit</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="nonpure">
+ <test-case FilePath="nonpure">
+ <compilation-unit name="global-datetime-use-index">
+ <output-dir compare="Text">global-datetime-use-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nonpure">
+ <compilation-unit name="local-datetime-ignore-index">
+ <output-dir compare="Text">local-datetime-ignore-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="orderby_limit">
+ <test-case FilePath="orderby_limit">
+ <compilation-unit name="limit_on_variable_01">
+ <output-dir compare="Text">limit_on_variable_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="orderby_limit">
+ <compilation-unit name="orderby_limit_01">
+ <output-dir compare="Text">orderby_limit_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="orderby_limit">
+ <compilation-unit name="orderby_limit_02">
+ <output-dir compare="Text">orderby_limit_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="orderby_limit">
+ <compilation-unit name="orderby_limit_offset_01">
+ <output-dir compare="Text">orderby_limit_offset_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="orderby_limit">
+ <compilation-unit name="orderby_limit_primary_index_01">
+ <output-dir compare="Text">orderby_limit_primary_index_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="fun_return_null_missing/string_fun">
+ <test-case FilePath="fun_return_null_missing/string_fun" check-warnings="true">
+ <compilation-unit name="string_fun_001">
+ <output-dir compare="Text">string_fun_001</output-dir>
+ <expected-warn>Type mismatch: function trim expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 41, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function like expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 32, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function uppercase expects its 1st input parameter to be of type string, but the actual input type is smallint (in line 38, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function ltrim expects its 1st input parameter to be of type string, but the actual input type is smallint (in line 42, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function like expects its 1st input parameter to be of type string, but the actual input type is tinyint (in line 33, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function string-length expects its 1st input parameter to be of type string, but the actual input type is integer (in line 36, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function trim expects its 1st input parameter to be of type string, but the actual input type is integer (in line 40, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function lowercase expects its 1st input parameter to be of type string, but the actual input type is smallint (in line 37, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function rtrim expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 45, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function contains expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 34, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function ltrim expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 43, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function position expects its 1st input parameter to be of type string, but the actual input type is tinyint (in line 46, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function string-to-codepoint expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 35, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function initcap expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 39, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function rtrim expects its 1st input parameter to be of type string, but the actual input type is integer (in line 44, at column 1)</expected-warn>
+
+ <expected-warn>Type mismatch: function trim expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 41, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function like expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 32, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function uppercase expects its 1st input parameter to be of type string, but the actual input type is smallint (in line 38, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function ltrim expects its 1st input parameter to be of type string, but the actual input type is smallint (in line 42, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function like expects its 1st input parameter to be of type string, but the actual input type is tinyint (in line 33, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function string-length expects its 1st input parameter to be of type string, but the actual input type is integer (in line 36, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function trim expects its 1st input parameter to be of type string, but the actual input type is integer (in line 40, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function lowercase expects its 1st input parameter to be of type string, but the actual input type is smallint (in line 37, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function rtrim expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 45, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function contains expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 34, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function ltrim expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 43, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function position expects its 1st input parameter to be of type string, but the actual input type is tinyint (in line 46, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function string-to-codepoint expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 35, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function initcap expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 39, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function rtrim expects its 1st input parameter to be of type string, but the actual input type is integer (in line 44, at column 1)</expected-warn>
+
+ <expected-warn>Type mismatch: function trim expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 41, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function like expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 32, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function uppercase expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 38, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function ltrim expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 42, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function like expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 33, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function string-length expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 36, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function trim expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 40, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function lowercase expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 37, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function rtrim expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 45, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function contains expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 34, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function ltrim expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 43, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function position expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 46, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function string-to-codepoint expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 35, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function initcap expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 39, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function rtrim expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 44, at column 1)</expected-warn>
+
+ <expected-warn>Type mismatch: function regexp-position expects its 1st input parameter to be of type string, but the actual input type is integer (in line 38, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function replace expects its 1st input parameter to be of type string, but the actual input type is smallint (in line 42, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function starts-with expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 32, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function reverse expects its 1st input parameter to be of type string, but the actual input type is integer (in line 43, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function regexp-like expects its 3rd input parameter to be of type string, but the actual input type is tinyint (in line 37, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function string-equal expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 41, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function substring-after expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 44, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function matches expects its 2nd input parameter to be of type string, but the actual input type is integer (in line 34, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function regexp-like expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 36, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function regexp-position expects its 3rd input parameter to be of type string, but the actual input type is tinyint (in line 39, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function substring-before expects its 1st input parameter to be of type string, but the actual input type is tinyint (in line 45, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function regexp-replace expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 40, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function matches expects its 3rd input parameter to be of type string, but the actual input type is tinyint (in line 35, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function split expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 46, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function ends-with expects its 1st input parameter to be of type string, but the actual input type is smallint (in line 33, at column 1)</expected-warn>
+
+ <expected-warn>Type mismatch: function regexp-position expects its 1st input parameter to be of type string, but the actual input type is integer (in line 38, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function replace expects its 1st input parameter to be of type string, but the actual input type is smallint (in line 42, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function starts-with expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 32, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function reverse expects its 1st input parameter to be of type string, but the actual input type is integer (in line 43, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function regexp-like expects its 3rd input parameter to be of type string, but the actual input type is tinyint (in line 37, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function string-equal expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 41, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function substring-after expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 44, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function matches expects its 2nd input parameter to be of type string, but the actual input type is integer (in line 34, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function regexp-like expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 36, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function regexp-position expects its 3rd input parameter to be of type string, but the actual input type is tinyint (in line 39, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function substring-before expects its 1st input parameter to be of type string, but the actual input type is tinyint (in line 45, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function regexp-replace expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 40, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function matches expects its 3rd input parameter to be of type string, but the actual input type is tinyint (in line 35, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function split expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 46, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function ends-with expects its 1st input parameter to be of type string, but the actual input type is smallint (in line 33, at column 1)</expected-warn>
+
+ <expected-warn>Type mismatch: function regexp-position expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 38, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function replace expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 42, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function starts-with expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 32, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function reverse expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 43, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function regexp-like expects its 3rd input parameter to be of type string, but the actual input type is bigint (in line 37, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function string-equal expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 41, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function substring-after expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 44, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function matches expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 34, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function regexp-like expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 36, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function regexp-position expects its 3rd input parameter to be of type string, but the actual input type is bigint (in line 39, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function substring-before expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 45, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function regexp-replace expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 40, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function matches expects its 3rd input parameter to be of type string, but the actual input type is bigint (in line 35, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function split expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 46, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function ends-with expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 33, at column 1)</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="fun_return_null_missing/string_fun" check-warnings="true">
+ <compilation-unit name="string_fun_002">
+ <output-dir compare="Text">string_fun_002</output-dir>
+ <expected-warn>Type mismatch: function rtrim expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 42, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function regexp-replace expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 52, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function regexp-like expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 48, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function position expects its 1st input parameter to be of type string, but the actual input type is tinyint (in line 43, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function matches expects its 2nd input parameter to be of type string, but the actual input type is integer (in line 46, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function replace expects its 1st input parameter to be of type string, but the actual input type is smallint (in line 54, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function starts-with expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 44, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function regexp-position expects its 3rd input parameter to be of type string, but the actual input type is tinyint (in line 51, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function rtrim expects its 1st input parameter to be of type string, but the actual input type is integer (in line 41, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function substring-after expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 56, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function substring-before expects its 1st input parameter to be of type string, but the actual input type is tinyint (in line 57, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function lowercase expects its 1st input parameter to be of type string, but the actual input type is smallint (in line 34, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function ends-with expects its 1st input parameter to be of type string, but the actual input type is smallint (in line 45, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function trim expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 38, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function uppercase expects its 1st input parameter to be of type string, but the actual input type is smallint (in line 35, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function regexp-like expects its 3rd input parameter to be of type string, but the actual input type is tinyint (in line 49, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function matches expects its 3rd input parameter to be of type string, but the actual input type is tinyint (in line 47, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function string-equal expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 53, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function split expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 58, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function string-to-codepoint expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 32, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function trim expects its 1st input parameter to be of type string, but the actual input type is integer (in line 37, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function reverse expects its 1st input parameter to be of type string, but the actual input type is integer (in line 55, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function like expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 29, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function contains expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 31, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function ltrim expects its 1st input parameter to be of type string, but the actual input type is smallint (in line 39, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function initcap expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 36, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function ltrim expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 40, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function string-length expects its 1st input parameter to be of type string, but the actual input type is integer (in line 33, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function like expects its 1st input parameter to be of type string, but the actual input type is tinyint (in line 30, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function regexp-position expects its 1st input parameter to be of type string, but the actual input type is integer (in line 50, at column 1)</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="fun_return_null_missing/string_fun" check-warnings="true">
+ <compilation-unit name="string_fun_003">
+ <output-dir compare="Text">string_fun_003</output-dir>
+ <expected-warn>Invalid value: function repeat expects its 2nd input parameter to be an integer value, got 5.3 (in line 31, at column 1)</expected-warn>
+ <expected-warn>Invalid value: function regexp-replace expects its 4th input parameter to be an integer value, got 5.3 (in line 36, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function substring expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 37, at column 1)</expected-warn>
+ <expected-warn>Invalid value: function replace expects its 4th input parameter to be an integer value, got 5.3 (in line 34, at column 1)</expected-warn>
+ <expected-warn>Invalid value: function repeat expects its 2nd input parameter to be a non-negative value, got -2.0 (in line 30, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function string-join expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 32, at column 1)</expected-warn>
+
+ <expected-warn>Invalid value: function repeat expects its 2nd input parameter to be an integer value, got 5.3 (in line 31, at column 1)</expected-warn>
+ <expected-warn>Invalid value: function regexp-replace expects its 4th input parameter to be an integer value, got 5.3 (in line 36, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function substring expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 37, at column 1)</expected-warn>
+ <expected-warn>Invalid value: function replace expects its 4th input parameter to be an integer value, got 5.3 (in line 34, at column 1)</expected-warn>
+ <expected-warn>Invalid value: function repeat expects its 2nd input parameter to be a non-negative value, got -2.0 (in line 30, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function string-join expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 32, at column 1)</expected-warn>
+
+ <expected-warn>Invalid value: function repeat expects its 2nd input parameter to be an integer value, got Infinity (in line 29, at column 1)</expected-warn>
+ <expected-warn>Invalid value: function replace expects its 4th input parameter to be an integer value, got -Infinity (in line 30, at column 1)</expected-warn>
+ <expected-warn>Invalid value: function replace expects its 4th input parameter to be an integer value, got NaN (in line 31, at column 1)</expected-warn>
+ <expected-warn>Invalid value: function regexp-replace expects its 4th input parameter to be an integer value, got NaN (in line 32, at column 1)</expected-warn>
+ <expected-warn>Invalid value: function regexp-replace expects its 4th input parameter to be an integer value, got -Infinity (in line 33, at column 1)</expected-warn>
+ <expected-warn>Invalid value: function substring expects its 2nd input parameter to be an integer value, got Infinity (in line 34, at column 1)</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="fun_return_null_missing/string_fun" check-warnings="true">
+ <compilation-unit name="string_fun_004">
+ <output-dir compare="Text">string_fun_004</output-dir>
+ <expected-warn>Type mismatch: function string-concat expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 30, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function substring expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string (in line 32, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function substring expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 33, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function string-concat expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 31, at column 7)</expected-warn>
+ <expected-warn>Type mismatch: function codepoint-to-string expects its 1st input parameter to be of type array, but the actual input type is string (in line 34, at column 1)</expected-warn>
+ <expected-warn>Unsupported type: codepoint-to-string cannot process input type string (in line 35, at column 1)</expected-warn>
+
+ <expected-warn>Type mismatch: function string-concat expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 30, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function substring expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string (in line 32, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function substring expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 33, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function string-concat expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 31, at column 7)</expected-warn>
+ <expected-warn>Type mismatch: function codepoint-to-string expects its 1st input parameter to be of type array, but the actual input type is string (in line 34, at column 1)</expected-warn>
+ <expected-warn>Unsupported type: codepoint-to-string cannot process input type string (in line 35, at column 1)</expected-warn>
+
+ <expected-warn>Type mismatch: function string-concat expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 30, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function string-concat expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 31, at column 7)</expected-warn>
+ <expected-warn>Type mismatch: function substring expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string (in line 32, at column 1)</expected-warn>
+ <expected-warn>Type mismatch: function codepoint-to-string expects its 1st input parameter to be of type array, but the actual input type is string (in line 33, at column 1)</expected-warn>
+ <expected-warn>Unsupported type: codepoint-to-string cannot process input type string (in line 34, at column 1)</expected-warn>
+ <expected-warn>Unsupported type: codepoint-to-string cannot process input type double (in line 35, at column 1)</expected-warn>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="fun_return_null_missing/numeric_fun" >
+ <test-case FilePath="fun_return_null_missing/numeric_fun" check-warnings="true">
+ <compilation-unit name="numeric_fun_001">
+ <output-dir compare="Text">numeric_fun_001</output-dir>
+ <expected-warn>Type mismatch: function abs expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function acos expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string </expected-warn>
+ <expected-warn>Type mismatch: function asin expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function atan expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function atan2 expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function degrees expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function radians expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function cos expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function cosh expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function sin expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function sinh expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function tan expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function tanh expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function exp expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function ln expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function log expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function sqrt expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function sign expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function ceiling expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function floor expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function round-half-to-even expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function numeric-unary-minus expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function random expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+
+ <expected-warn>Type mismatch: function abs expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function acos expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string </expected-warn>
+ <expected-warn>Type mismatch: function asin expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function atan expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function atan2 expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function degrees expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function radians expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function cos expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function cosh expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function sin expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function sinh expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function tan expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function tanh expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function exp expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function ln expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function log expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function sqrt expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function sign expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function ceiling expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function floor expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function round-half-to-even expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function numeric-unary-minus expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function random expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+
+ <expected-warn>Type mismatch: function abs expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function acos expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string </expected-warn>
+ <expected-warn>Type mismatch: function asin expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function atan expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function atan2 expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function degrees expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function radians expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function cos expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function cosh expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function sin expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function sinh expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function tan expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function tanh expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function exp expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function ln expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function log expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function sqrt expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function sign expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function ceiling expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function floor expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function round-half-to-even expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function numeric-unary-minus expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function random expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="fun_return_null_missing/numeric_fun" check-warnings="true">
+ <compilation-unit name="numeric_fun_002">
+ <output-dir compare="Text">numeric_fun_002</output-dir>
+ <expected-warn>Type mismatch: function round expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function round-half-to-even expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function trunc expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function round-half-to-even expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function trunc expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function round expects its 2nd input parameter to be of type tinyint, smallint, integer or bigint, but the actual input type is string</expected-warn>
+
+ <expected-warn>Type mismatch: function round expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function round-half-to-even expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function trunc expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function round expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function round-half-to-even expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function trunc expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+
+ <expected-warn>Type mismatch: function round expects its 2nd input parameter to be of type tinyint, smallint, integer or bigint, but the actual input type is string (in line 33, at column 24)</expected-warn>
+ <expected-warn>Type mismatch: function round expects its 2nd input parameter to be of type tinyint, smallint, integer or bigint, but the actual input type is string (in line 33, at column 47)</expected-warn>
+ <expected-warn>Type mismatch: function trunc expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string (in line 34, at column 24)</expected-warn>
+ <expected-warn>Type mismatch: function trunc expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string (in line 34, at column 47)</expected-warn>
+
+ <expected-warn>Type mismatch: function round-half-to-even expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string (in line 36, at column 40)</expected-warn>
+ <expected-warn>Type mismatch: function round-half-to-even expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string (in line 36, at column 79)</expected-warn>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="fun_return_null_missing/numeric_fun" check-warnings="true">
+ <compilation-unit name="numeric_fun_003">
+ <output-dir compare="Text">numeric_fun_003</output-dir>
+ <expected-warn>Type mismatch: function numeric-add expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function numeric-divide expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
+ <expected-warn>Type incompatibility: function numeric-multiply gets incompatible input values: daytimeduration and string</expected-warn>
+ <expected-warn>Type mismatch: function numeric-add expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function numeric-subtract expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
+ <expected-warn>Type incompatibility: function numeric-divide gets incompatible input values: time and string</expected-warn>
+ <expected-warn>Type incompatibility: function numeric-add gets incompatible input values: yearmonthduration and string</expected-warn>
+ <expected-warn>Type mismatch: function numeric-subtract expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function numeric-multiply expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
+ <expected-warn>Type incompatibility: function numeric-add gets incompatible input values: daytimeduration and string</expected-warn>
+ <expected-warn>Type mismatch: function numeric-div expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function numeric-subtract expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function power expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function numeric-mod expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
+ <expected-warn>Type incompatibility: function numeric-add gets incompatible input values: time and string</expected-warn>
+ <expected-warn>Type incompatibility: function power gets incompatible input values: yearmonthduration and string</expected-warn>
+ <expected-warn>Type mismatch: function numeric-multiply expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function numeric-multiply expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function numeric-mod expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function numeric-divide expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function numeric-div expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
+ <expected-warn>Type incompatibility: function numeric-multiply gets incompatible input values: date and string</expected-warn>
+ <expected-warn>Type mismatch: function numeric-add expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
+ <expected-warn>Type incompatibility: function numeric-add gets incompatible input values: duration and string</expected-warn>
+ <expected-warn>Type incompatibility: function numeric-div gets incompatible input values: datetime and string</expected-warn>
+ <expected-warn>Type mismatch: function numeric-mod expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function power expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
+ <expected-warn>Type incompatibility: function numeric-add gets incompatible input values: date and string</expected-warn>
+ <expected-warn>Type incompatibility: function numeric-mod gets incompatible input values: duration and string</expected-warn>
+ <expected-warn>Type mismatch: function numeric-subtract expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function numeric-subtract expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function numeric-divide expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function numeric-subtract expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function numeric-subtract expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function numeric-div expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function numeric-subtract expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
+ <expected-warn>Type incompatibility: function numeric-add gets incompatible input values: datetime and string</expected-warn>
+ <expected-warn>Type mismatch: function power expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="fun_return_null_missing" check-warnings="true">
+ <compilation-unit name="field-access">
+ <output-dir compare="Text">field-access</output-dir>
+ <expected-warn>Type mismatch: function field-access-by-name expects its 1st input parameter to be of type object, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function field-access-by-name expects its 1st input parameter to be of type object, but the actual input type is string</expected-warn>
+ <expected-warn>Type mismatch: function field-access-by-name expects its 1st input parameter to be of type object, but the actual input type is string</expected-warn>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="window">
+ <test-case FilePath="window">
+ <compilation-unit name="cume_dist_01">
+ <output-dir compare="Text">cume_dist_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="window">
+ <compilation-unit name="dense_rank_01">
+ <output-dir compare="Text">dense_rank_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="window">
+ <compilation-unit name="first_value_01">
+ <output-dir compare="Text">first_value_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="window">
+ <compilation-unit name="lag_01">
+ <output-dir compare="Text">lag_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="window">
+ <compilation-unit name="last_value_01">
+ <output-dir compare="Text">last_value_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="window">
+ <compilation-unit name="lead_01">
+ <output-dir compare="Text">lead_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="window">
+ <compilation-unit name="misc_01">
+ <output-dir compare="Text">misc_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="window">
+ <compilation-unit name="nth_value_01">
+ <output-dir compare="Text">nth_value_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="window">
+ <compilation-unit name="ntile_01">
+ <output-dir compare="Text">ntile_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="window">
+ <compilation-unit name="percent_rank_01">
+ <output-dir compare="Text">percent_rank_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="window">
+ <compilation-unit name="pg_win">
+ <output-dir compare="Text">pg_win</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="window">
+ <compilation-unit name="rank_01">
+ <output-dir compare="Text">rank_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="window">
+ <compilation-unit name="ratio_to_report_01">
+ <output-dir compare="Text">ratio_to_report_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="window">
+ <compilation-unit name="row_number_01">
+ <output-dir compare="Text">row_number_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="window">
+ <compilation-unit name="win_negative">
+ <output-dir compare="Text">misc_01</output-dir>
+ <expected-error>ASX0002: Type mismatch</expected-error>
+ <expected-error>ASX1104: Invalid modifier FROM FIRST/LAST for function</expected-error>
+ <expected-error>ASX1037: Invalid query parameter compiler.windowmemory</expected-error>
+ <expected-error>ASX1102: Expected window or aggregate function, got: lowercase</expected-error>
+ <expected-error>ASX1079: Compilation error: count is a SQL-92 aggregate function</expected-error>
+ <expected-error>ASX1104: Invalid modifier RESPECT/IGNORE NULLS for function</expected-error>
+ <expected-error>ASX1104: Invalid modifier RESPECT/IGNORE NULLS for function</expected-error>
+ <expected-error>ASX1104: Invalid modifier FROM FIRST/LAST for function</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="window">
+ <compilation-unit name="win_null_missing">
+ <output-dir compare="Text">win_null_missing</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="window">
+ <compilation-unit name="win_opt_01">
+ <output-dir compare="Text">win_opt_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="window">
+ <compilation-unit name="win_opt_02">
+ <output-dir compare="Text">win_opt_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+</test-group>
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_it_python.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_it_python.xml
index 686ede2..7b92376 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_it_python.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_it_python.xml
@@ -52,15 +52,7 @@
<test-case FilePath="external-library" check-warnings="true">
<compilation-unit name="py_function_error">
<output-dir compare="Clean-JSON">py_function_error</output-dir>
- <expected-warn>ASX0201: External UDF returned exception. Returned exception was: Traceback (most recent call last):
- File "entrypoint.py", line 181, in handle_call
- result[0].append(self.next_tuple(*arg, key=self.mid))
- File "entrypoint.py", line 99, in next_tuple
- return self.wrapped_fns[key](*args)
- File "site-packages/roundtrip.py", line 32, in warning
- raise ArithmeticError("oof")
-ArithmeticError: oof
- (in line 28, at column 1)</expected-warn>
+ <expected-warn>ArithmeticError: oof</expected-warn>
</compilation-unit>
</test-case>
<test-case FilePath="external-library">
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 41c0424..288a310 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -17,6 +17,7 @@
! under the License.
!-->
<!DOCTYPE test-suite [
+ <!ENTITY SqlppQueries SYSTEM "sqlpp_queries.xml">
<!ENTITY ObjectsQueries SYSTEM "queries_sqlpp/objects/ObjectsQueries.xml">
<!ENTITY AsyncDeferredQueries SYSTEM "queries_sqlpp/async-deferred/AsyncDeferredQueries.xml">
<!ENTITY GeoQueries SYSTEM "queries_sqlpp/geojson/GeoJSONQueries.xml">
@@ -27,15876 +28,5 @@
&AsyncDeferredQueries;
&GeoQueries;
&TemporalQueries;
- <test-group name="api">
- <test-case FilePath="api">
- <compilation-unit name="compileonly">
- <output-dir compare="Text">compileonly</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="api">
- <compilation-unit name="multiple-param-values">
- <output-dir compare="Text">multiple-param-values</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="api">
- <compilation-unit name="readonly-request">
- <output-dir compare="Text">readonly-request</output-dir>
- <expected-error>ASX0044: CREATE_DATAVERSE statement is not supported in read-only mode</expected-error>
- <expected-error>ASX0044: DATAVERSE_DROP statement is not supported in read-only mode</expected-error>
- <expected-error>ASX0044: DATASET_DECL statement is not supported in read-only mode</expected-error>
- <expected-error>ASX0044: DATASET_DROP statement is not supported in read-only mode</expected-error>
- <expected-error>ASX0044: CREATE_INDEX statement is not supported in read-only mode</expected-error>
- <expected-error>ASX0044: INDEX_DROP statement is not supported in read-only mode</expected-error>
- <expected-error>ASX0044: INSERT statement is not supported in read-only mode</expected-error>
- <expected-error>ASX0044: UPSERT statement is not supported in read-only mode</expected-error>
- <expected-error>ASX0044: DELETE statement is not supported in read-only mode</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="api" check-warnings="true">
- <compilation-unit name="request-dataverse">
- <output-dir compare="Text">request-dataverse</output-dir>
- <expected-warn>ASX1063: Cannot find dataverse with name testUnknown</expected-warn>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="api">
- <compilation-unit name="request-param-validation">
- <output-dir compare="Text">request-param-validation</output-dir>
- <expected-error>Invalid value for parameter 'format': foo</expected-error>
- <expected-error>Invalid value for parameter 'pretty': bar</expected-error>
- <expected-error>Invalid value for parameter 'plan-format': blah</expected-error>
- <expected-error>Invalid value for parameter 'max-result-reads': foo</expected-error>
- <expected-error>Invalid value for parameter 'max-result-reads': 9999999999999999999999999999999999999999</expected-error>
- <expected-error>Invalid value for parameter 'max-warnings': baz</expected-error>
- <expected-error>Invalid value for parameter 'max-warnings': 1.5</expected-error>
- <expected-error>Invalid value for parameter 'mode': asyn</expected-error>
- <expected-error>Invalid value for parameter 'timeout': 12</expected-error>
- <expected-error>Invalid value for parameter 'args': 12</expected-error>
- <expected-error>Unable to process JSON content in request</expected-error>
- <expected-error>Unable to process JSON content in request</expected-error>
- <expected-error>Invalid value for parameter 'format': foo</expected-error>
- <expected-error>Invalid value for parameter 'pretty': bar</expected-error>
- <expected-error>Invalid value for parameter 'plan-format': blah</expected-error>
- <expected-error>Invalid value for parameter 'max-result-reads': foo</expected-error>
- <expected-error>Invalid value for parameter 'max-warnings': baz</expected-error>
- <expected-error>Invalid value for parameter 'mode': asyn</expected-error>
- <expected-error>Invalid value for parameter 'args': 12</expected-error>
- <expected-error>Unable to process JSON content in request</expected-error>
- <expected-error>Unable to process JSON content in request</expected-error>
- <expected-error>Invalid value for parameter 'profile': true</expected-error>
- <expected-error>Invalid value for parameter 'profile': true</expected-error>
- <expected-error>Invalid value for parameter 'profile': foo</expected-error>
- <expected-error>Invalid value for parameter 'profile': foo</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="api">
- <compilation-unit name="request-param-validation-400-BAD">
- <output-dir compare="Text">request-param-validation-400-BAD</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="api">
- <compilation-unit name="request-param">
- <output-dir compare="Text">request-param</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="api">
- <compilation-unit name="format-param-in-accept-01">
- <output-dir compare="Text">format-param-in-accept-01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="api">
- <compilation-unit name="format-param-in-accept-02">
- <output-dir compare="Clean-JSON">format-param-in-accept-02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="api">
- <compilation-unit name="format-param-in-accept-03">
- <output-dir compare="Lossless-JSON">format-param-in-accept-03</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="api">
- <compilation-unit name="format-param-in-accept-04">
- <output-dir compare="Text">format-param-in-accept-04</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="api">
- <compilation-unit name="format-param-in-accept-05">
- <output-dir compare="AST">format-param-in-accept-05</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="api">
- <compilation-unit name="get-non-query">
- <output-dir compare="Text">get-non-query</output-dir>
- <expected-error>CREATE_DATAVERSE statement is not supported in read-only mode</expected-error>
- <expected-error>TYPE_DECL statement is not supported in read-only mode</expected-error>
- <expected-error>CREATE_FUNCTION statement is not supported in read-only mode</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="api">
- <compilation-unit name="get-query">
- <output-dir compare="Text">get-query</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="api">
- <compilation-unit name="post-non-query">
- <output-dir compare="Text">post-non-query</output-dir>
- <expected-error>TYPE_DECL statement is not supported in read-only mode</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="api">
- <compilation-unit name="signature">
- <output-dir compare="Text">signature</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="api">
- <compilation-unit name="ignore-body-for-get">
- <output-dir compare="Text">ignore-body-for-get</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="flwor">
- <test-case FilePath="flwor">
- <compilation-unit name="at00">
- <output-dir compare="Text">at00</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="at01">
- <output-dir compare="Text">at01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="at02">
- <output-dir compare="Text">at02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="at03">
- <output-dir compare="Text">at03</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="at04">
- <output-dir compare="Text">at04</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="at05">
- <output-dir compare="Text">at05</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="at06">
- <output-dir compare="Text">at06</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="at07">
- <output-dir compare="Text">at07</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="order-by-13">
- <output-dir compare="Text">order-by-13</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="query-issue550">
- <output-dir compare="Text">query-issue550</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="query-ASTERIXDB-883">
- <output-dir compare="Text">query-ASTERIXDB-883</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="query-ASTERIXDB-1576">
- <output-dir compare="Text">query-ASTERIXDB-883</output-dir>
- <expected-error>Duplicate alias definitions: samptable1</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="query-ASTERIXDB-1576-2">
- <output-dir compare="Text">query-ASTERIXDB-883</output-dir>
- <expected-error>Duplicate alias definitions: s2</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="query-ASTERIXDB-1576-3">
- <output-dir compare="Text">query-ASTERIXDB-883</output-dir>
- <expected-error>Duplicate alias definitions: s1</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor" check-warnings="true">
- <compilation-unit name="query-ASTERIXDB-2446">
- <output-dir compare="Text">query-ASTERIXDB-2446</output-dir>
- <expected-error>ASX0013: Duplicate field name 'a'</expected-error>
- <expected-warn>Duplicate field name 'c' (in line 28, at column 84)</expected-warn>
- <expected-warn>Duplicate field name 'e' (in line 28, at column 116)</expected-warn>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="query-ASTERIXDB-2446-2">
- <output-dir compare="Text">query-ASTERIXDB-883</output-dir>
- <expected-error>ASX0013: Duplicate field name 'a' (in line 27, at column 20)</expected-error>
- <expected-error>ASX0013: Duplicate field name 'b' (in line 27, at column 20)</expected-error>
- <expected-error>ASX0013: Duplicate field name 'c' (in line 27, at column 11)</expected-error>
- <expected-error>ASX0013: Duplicate field name 'd' (in line 27, at column 11)</expected-error>
- <expected-error>ASX0013: Duplicate field name 'e' (in line 27, at column 14)</expected-error>
- <expected-error>ASX0013: Duplicate field name 'f' (in line 27, at column 11)</expected-error>
- <expected-error>ASX0013: Duplicate field name 'g' (in line 27, at column 11)</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="select-let">
- <output-dir compare="Text">select-let</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="let33">
- <output-dir compare="Text">let33</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="at00">
- <output-dir compare="Text">at00</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="query-ASTERIXDB-1463">
- <output-dir compare="Text">query-ASTERIXDB-1485</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="query-ASTERIXDB-1485">
- <output-dir compare="Text">query-ASTERIXDB-1485</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="sorting">
- <test-case FilePath="sorting">
- <compilation-unit name="arrays">
- <output-dir compare="Text">arrays</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="sorting">
- <compilation-unit name="range_hint">
- <output-dir compare="Text">range_hint</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="sorting">
- <compilation-unit name="records">
- <output-dir compare="Text">records</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="explain">
- <test-case FilePath="explain">
- <compilation-unit name="explain_simple">
- <output-dir compare="Text">explain_simple</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="explain">
- <compilation-unit name="explain_caps">
- <output-dir compare="Text">explain_simple</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="explain">
- <compilation-unit name="explain_pretty">
- <parameter name="pretty" value="true" />
- <output-dir compare="Text">explain_simple</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="explain">
- <compilation-unit name="explain_object_constructor-01">
- <output-dir compare="Text">explain_object_constructor-01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="explain">
- <compilation-unit name="explain_object_constructor-02">
- <output-dir compare="Text">explain_object_constructor-02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="explain">
- <compilation-unit name="explain_object_constructor-03">
- <output-dir compare="Text">explain_object_constructor-03</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="explain">
- <compilation-unit name="explain_field_access">
- <output-dir compare="Text">explain_field_access</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="explain">
- <compilation-unit name="explain_field_access_closed">
- <output-dir compare="Text">explain_field_access_closed</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="explain">
- <compilation-unit name="explain_negative">
- <output-dir compare="Text">explain_simple</output-dir>
- <expected-error>ASX1001: Syntax error: EXPLAIN is not supported for this kind of statement</expected-error>
- <expected-error>ASX1001: Syntax error: EXPLAIN is not supported for this kind of statement</expected-error>
- <expected-error>ASX1001: Syntax error: EXPLAIN is not supported for this kind of statement</expected-error>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="aggregate">
- <test-case FilePath="aggregate">
- <compilation-unit name="issue531_string_min_max">
- <output-dir compare="Text">issue531_string_min_max</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="agg_null">
- <output-dir compare="Text">agg_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="agg_null_rec">
- <output-dir compare="Text">agg_null_rec</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="agg_null_rec_1">
- <output-dir compare="Text">agg_null_rec_1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="agg_number_rec">
- <output-dir compare="Text">agg_number_rec</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate" check-warnings="true">
- <compilation-unit name="avg_mixed">
- <output-dir compare="Text">avg_mixed</output-dir>
- <expected-warn>Unsupported type: agg-avg cannot process input type string (in line 26, at column 16)</expected-warn>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate" check-warnings="true">
- <compilation-unit name="serial_avg_mixed">
- <output-dir compare="Text">serial_avg_mixed</output-dir>
- <expected-warn>Unsupported type: agg-avg cannot process input type string (in line 29, at column 39)</expected-warn>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="group_only">
- <output-dir compare="Text">group_only</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="min_mixed">
- <output-dir compare="Text">min_mixed</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="stddev_mixed">
- <output-dir compare="Text">stddev_mixed</output-dir>
- <expected-error>Type incompatibility: function agg-stddev_samp gets incompatible input values: string and float</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_stddev_mixed">
- <output-dir compare="Text">serial_stddev_mixed</output-dir>
- <expected-error>Type incompatibility: function agg-stddev_samp gets incompatible input values: string and float</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="stddev_pop_mixed">
- <output-dir compare="Text">stddev_pop_mixed</output-dir>
- <expected-error>Type incompatibility: function agg-stddev_pop gets incompatible input values: string and float</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_stddev_pop_mixed">
- <output-dir compare="Text">serial_stddev_pop_mixed</output-dir>
- <expected-error>Type incompatibility: function agg-stddev_pop gets incompatible input values: string and float</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate" check-warnings="true">
- <compilation-unit name="sum/sum_mixed">
- <output-dir compare="Text">sum/sum_mixed</output-dir>
- <expected-warn>Unsupported type: agg-sum cannot process input type string (in line 27, at column 16)</expected-warn>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate" check-warnings="true">
- <compilation-unit name="sum/serial_sum_mixed">
- <output-dir compare="Text">sum/serial_sum_mixed</output-dir>
- <expected-warn>Unsupported type: agg-sum cannot process input type string (in line 29, at column 39)</expected-warn>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="var_mixed">
- <output-dir compare="Text">var_mixed</output-dir>
- <expected-error>Type incompatibility: function agg-var_samp gets incompatible input values: string and float</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_var_mixed">
- <output-dir compare="Text">serial_var_mixed</output-dir>
- <expected-error>Type incompatibility: function agg-var_samp gets incompatible input values: string and float</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="var_pop_mixed">
- <output-dir compare="Text">var_pop_mixed</output-dir>
- <expected-error>Type incompatibility: function agg-var_pop gets incompatible input values: string and float</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_var_pop_mixed">
- <output-dir compare="Text">serial_var_pop_mixed</output-dir>
- <expected-error>Type incompatibility: function agg-var_pop gets incompatible input values: string and float</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="kurtosis_mixed">
- <output-dir compare="Text">kurtosis_mixed</output-dir>
- <expected-error>Type incompatibility: function agg-kurtosis gets incompatible input values: string and float</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_kurtosis_mixed">
- <output-dir compare="Text">serial_kurtosis_mixed</output-dir>
- <expected-error>Type incompatibility: function agg-kurtosis gets incompatible input values: string and float</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="skewness_mixed">
- <output-dir compare="Text">skewness_mixed</output-dir>
- <expected-error>Type incompatibility: function agg-skewness gets incompatible input values: string and float</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_skewness_mixed">
- <output-dir compare="Text">serial_skewness_mixed</output-dir>
- <expected-error>Type incompatibility: function agg-skewness gets incompatible input values: string and float</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="agg_number">
- <output-dir compare="Text">agg_number</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="issue425_min_hetero_list_1">
- <output-dir compare="Text">issue425_min_hetero_list_1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="issue425_min_hetero_list">
- <output-dir compare="Text">issue425_min_hetero_list</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="sum/issue425_sum_hetero_list_1">
- <output-dir compare="Text">sum/issue425_sum_hetero_list_1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="sum/issue425_sum_hetero_list">
- <output-dir compare="Text">sum/issue425_sum_hetero_list</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="query-issue400">
- <output-dir compare="Text">query-issue400</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="issue395">
- <output-dir compare="Text">issue395</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="issue412_0">
- <output-dir compare="Text">issue412_0</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="issue412_1">
- <output-dir compare="Text">issue412_1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="avg_double">
- <output-dir compare="Text">avg_double</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="avg_double_null">
- <output-dir compare="Text">avg_double_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="avg_empty_01">
- <output-dir compare="Text">avg_empty_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="avg_empty_02">
- <output-dir compare="Text">avg_empty_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="avg_float">
- <output-dir compare="Text">avg_float</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="avg_float_null">
- <output-dir compare="Text">avg_float_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="avg_int16">
- <output-dir compare="Text">avg_int16</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="avg_int16_null">
- <output-dir compare="Text">avg_int16_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="avg_int32">
- <output-dir compare="Text">avg_int32</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="avg_int32_null">
- <output-dir compare="Text">avg_int32_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="avg_int64">
- <output-dir compare="Text">avg_int64</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="avg_int64_null">
- <output-dir compare="Text">avg_int64_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="avg_int8">
- <output-dir compare="Text">avg_int8</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="avg_int8_null">
- <output-dir compare="Text">avg_int8_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="avg_distinct">
- <output-dir compare="Text">avg_distinct</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="count_01">
- <output-dir compare="Text">count_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="count_dataset">
- <output-dir compare="Text">count_dataset</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="count_empty_01">
- <output-dir compare="Text">count_empty_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="count_empty_02">
- <output-dir compare="Text">count_empty_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="count_null">
- <output-dir compare="Text">count_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="count_distinct">
- <output-dir compare="Text">count_distinct</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="kurtosis_double">
- <output-dir compare="Text">kurtosis_double</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="kurtosis_double_null">
- <output-dir compare="Text">kurtosis_double_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="kurtosis_empty_01">
- <output-dir compare="Text">kurtosis_empty_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="kurtosis_empty_02">
- <output-dir compare="Text">kurtosis_empty_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="kurtosis_float">
- <output-dir compare="Text">kurtosis_float</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="kurtosis_float_null">
- <output-dir compare="Text">kurtosis_float_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="kurtosis_int16">
- <output-dir compare="Text">kurtosis_int16</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="kurtosis_int16_null">
- <output-dir compare="Text">kurtosis_int16_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="kurtosis_int32">
- <output-dir compare="Text">kurtosis_int32</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="kurtosis_int32_null">
- <output-dir compare="Text">kurtosis_int32_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="kurtosis_int64">
- <output-dir compare="Text">kurtosis_int64</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="kurtosis_int64_null">
- <output-dir compare="Text">kurtosis_int64_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="kurtosis_int8">
- <output-dir compare="Text">kurtosis_int8</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="kurtosis_int8_null">
- <output-dir compare="Text">kurtosis_int8_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="kurtosis_distinct">
- <output-dir compare="Text">kurtosis_distinct</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="max_empty_01">
- <output-dir compare="Text">max_empty_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="max_empty_02">
- <output-dir compare="Text">max_empty_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="max_distinct">
- <output-dir compare="Text">max_distinct</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="min_empty_01">
- <output-dir compare="Text">min_empty_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="min_empty_02">
- <output-dir compare="Text">min_empty_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="min_distinct">
- <output-dir compare="Text">min_distinct</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="scalar_avg">
- <output-dir compare="Text">scalar_avg</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="scalar_avg_empty">
- <output-dir compare="Text">scalar_avg_empty</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="scalar_avg_null">
- <output-dir compare="Text">scalar_avg_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="scalar_count">
- <output-dir compare="Text">scalar_count</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="scalar_count_empty">
- <output-dir compare="Text">scalar_count_empty</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="scalar_count_null">
- <output-dir compare="Text">scalar_count_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="scalar_kurtosis">
- <output-dir compare="Text">scalar_kurtosis</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="scalar_kurtosis_empty">
- <output-dir compare="Text">scalar_kurtosis_empty</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="scalar_kurtosis_null">
- <output-dir compare="Text">scalar_kurtosis_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="scalar_max">
- <output-dir compare="Text">scalar_max</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="scalar_max_empty">
- <output-dir compare="Text">scalar_max_empty</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="scalar_max_null">
- <output-dir compare="Text">scalar_max_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="scalar_min">
- <output-dir compare="Text">scalar_min</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="scalar_min_empty">
- <output-dir compare="Text">scalar_min_empty</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="scalar_min_null">
- <output-dir compare="Text">scalar_min_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="scalar_skewness">
- <output-dir compare="Text">scalar_skewness</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="scalar_skewness_empty">
- <output-dir compare="Text">scalar_skewness_empty</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="scalar_skewness_null">
- <output-dir compare="Text">scalar_skewness_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="scalar_stddev">
- <output-dir compare="Text">scalar_stddev</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="scalar_stddev_empty">
- <output-dir compare="Text">scalar_stddev_empty</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="scalar_stddev_null">
- <output-dir compare="Text">scalar_stddev_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="sum/scalar_sum">
- <output-dir compare="Text">sum/scalar_sum</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="sum/scalar_sum_empty">
- <output-dir compare="Text">sum/scalar_sum_empty</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="sum/scalar_sum_null">
- <output-dir compare="Text">sum/scalar_sum_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="sum/scalar_sum_type">
- <output-dir compare="Text">sum/scalar_sum_type</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="scalar_var">
- <output-dir compare="Text">scalar_var</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="scalar_var_empty">
- <output-dir compare="Text">scalar_var_empty</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="scalar_var_null">
- <output-dir compare="Text">scalar_var_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_avg_empty">
- <output-dir compare="Text">serial_avg_empty</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_avg_int8">
- <output-dir compare="Text">serial_avg_int8</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_avg_int8_null">
- <output-dir compare="Text">serial_avg_int8_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_avg_int16">
- <output-dir compare="Text">serial_avg_int16</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_avg_int16_null">
- <output-dir compare="Text">serial_avg_int16_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_avg_int32">
- <output-dir compare="Text">serial_avg_int32</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_avg_int32_null">
- <output-dir compare="Text">serial_avg_int32_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_avg_int64">
- <output-dir compare="Text">serial_avg_int64</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_avg_int64_null">
- <output-dir compare="Text">serial_avg_int64_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_avg_float">
- <output-dir compare="Text">serial_avg_float</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_avg_float_null">
- <output-dir compare="Text">serial_avg_float_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_avg_double">
- <output-dir compare="Text">serial_avg_double</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_avg_double_null">
- <output-dir compare="Text">serial_avg_double_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_kurtosis_double">
- <output-dir compare="Text">serial_kurtosis_double</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_kurtosis_double_null">
- <output-dir compare="Text">serial_kurtosis_double_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_kurtosis_empty">
- <output-dir compare="Text">serial_kurtosis_empty</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_kurtosis_float">
- <output-dir compare="Text">serial_kurtosis_float</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_kurtosis_float_null">
- <output-dir compare="Text">serial_kurtosis_float_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_kurtosis_int16">
- <output-dir compare="Text">serial_kurtosis_int16</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_kurtosis_int16_null">
- <output-dir compare="Text">serial_kurtosis_int16_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_kurtosis_int32">
- <output-dir compare="Text">serial_kurtosis_int32</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_kurtosis_int32_null">
- <output-dir compare="Text">serial_kurtosis_int32_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_kurtosis_int64">
- <output-dir compare="Text">serial_kurtosis_int64</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_kurtosis_int64_null">
- <output-dir compare="Text">serial_kurtosis_int64_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_kurtosis_int8">
- <output-dir compare="Text">serial_kurtosis_int8</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_kurtosis_int8_null">
- <output-dir compare="Text">serial_kurtosis_int8_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_skewness_double">
- <output-dir compare="Text">serial_skewness_double</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_skewness_double_null">
- <output-dir compare="Text">serial_skewness_double_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_skewness_empty">
- <output-dir compare="Text">serial_skewness_empty</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_skewness_float">
- <output-dir compare="Text">serial_skewness_float</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_skewness_float_null">
- <output-dir compare="Text">serial_skewness_float_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_skewness_int16">
- <output-dir compare="Text">serial_skewness_int16</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_skewness_int16_null">
- <output-dir compare="Text">serial_skewness_int16_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_skewness_int32">
- <output-dir compare="Text">serial_skewness_int32</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_skewness_int32_null">
- <output-dir compare="Text">serial_skewness_int32_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_skewness_int64">
- <output-dir compare="Text">serial_skewness_int64</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_skewness_int64_null">
- <output-dir compare="Text">serial_skewness_int64_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_skewness_int8">
- <output-dir compare="Text">serial_skewness_int8</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_skewness_int8_null">
- <output-dir compare="Text">serial_skewness_int8_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_stddev_double">
- <output-dir compare="Text">serial_stddev_double</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_stddev_double_null">
- <output-dir compare="Text">serial_stddev_double_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_stddev_empty">
- <output-dir compare="Text">serial_stddev_empty</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_stddev_float">
- <output-dir compare="Text">serial_stddev_float</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_stddev_float_null">
- <output-dir compare="Text">serial_stddev_float_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_stddev_int16">
- <output-dir compare="Text">serial_stddev_int16</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_stddev_int16_null">
- <output-dir compare="Text">serial_stddev_int16_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_stddev_int32">
- <output-dir compare="Text">serial_stddev_int32</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_stddev_int32_null">
- <output-dir compare="Text">serial_stddev_int32_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_stddev_int64">
- <output-dir compare="Text">serial_stddev_int64</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_stddev_int64_null">
- <output-dir compare="Text">serial_stddev_int64_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_stddev_int8">
- <output-dir compare="Text">serial_stddev_int8</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_stddev_int8_null">
- <output-dir compare="Text">serial_stddev_int8_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="sum/serial_sum_double">
- <output-dir compare="Text">sum/serial_sum_double</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="sum/serial_sum_double_null">
- <output-dir compare="Text">sum/serial_sum_double_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="sum/serial_sum_empty">
- <output-dir compare="Text">sum/serial_sum_empty</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="sum/serial_sum_float">
- <output-dir compare="Text">sum/serial_sum_float</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="sum/serial_sum_float_null">
- <output-dir compare="Text">sum/serial_sum_float_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="sum/serial_sum_int16">
- <output-dir compare="Text">sum/serial_sum_int16</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="sum/serial_sum_int16_null">
- <output-dir compare="Text">sum/serial_sum_int16_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="sum/serial_sum_int32">
- <output-dir compare="Text">sum/serial_sum_int32</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="sum/serial_sum_int32_null">
- <output-dir compare="Text">sum/serial_sum_int32_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="sum/serial_sum_int64">
- <output-dir compare="Text">sum/serial_sum_int64</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="sum/serial_sum_int64_null">
- <output-dir compare="Text">sum/serial_sum_int64_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="sum/serial_sum_int64_overflow">
- <output-dir compare="Text">sum/serial_sum_int64_overflow</output-dir>
- <expected-error>Overflow in agg-sum</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="sum/serial_sum_int8">
- <output-dir compare="Text">sum/serial_sum_int8</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="sum/serial_sum_int8_null">
- <output-dir compare="Text">sum/serial_sum_int8_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_var_double">
- <output-dir compare="Text">serial_var_double</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_var_double_null">
- <output-dir compare="Text">serial_var_double_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_var_empty">
- <output-dir compare="Text">serial_var_empty</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_var_float">
- <output-dir compare="Text">serial_var_float</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_var_float_null">
- <output-dir compare="Text">serial_var_float_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_var_int16">
- <output-dir compare="Text">serial_var_int16</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_var_int16_null">
- <output-dir compare="Text">serial_var_int16_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_var_int32">
- <output-dir compare="Text">serial_var_int32</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_var_int32_null">
- <output-dir compare="Text">serial_var_int32_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_var_int64">
- <output-dir compare="Text">serial_var_int64</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_var_int64_null">
- <output-dir compare="Text">serial_var_int64_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_var_int8">
- <output-dir compare="Text">serial_var_int8</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="serial_var_int8_null">
- <output-dir compare="Text">serial_var_int8_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="skewness_double">
- <output-dir compare="Text">skewness_double</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="skewness_double_null">
- <output-dir compare="Text">skewness_double_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="skewness_empty_01">
- <output-dir compare="Text">skewness_empty_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="skewness_empty_02">
- <output-dir compare="Text">skewness_empty_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="skewness_float">
- <output-dir compare="Text">skewness_float</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="skewness_float_null">
- <output-dir compare="Text">skewness_float_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="skewness_int16">
- <output-dir compare="Text">skewness_int16</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="skewness_int16_null">
- <output-dir compare="Text">skewness_int16_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="skewness_int32">
- <output-dir compare="Text">skewness_int32</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="skewness_int32_null">
- <output-dir compare="Text">skewness_int32_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="skewness_int64">
- <output-dir compare="Text">skewness_int64</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="skewness_int64_null">
- <output-dir compare="Text">skewness_int64_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="skewness_int8">
- <output-dir compare="Text">skewness_int8</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="skewness_int8_null">
- <output-dir compare="Text">skewness_int8_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="skewness_distinct">
- <output-dir compare="Text">skewness_distinct</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="stddev_double">
- <output-dir compare="Text">stddev_double</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="stddev_double_null">
- <output-dir compare="Text">stddev_double_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="stddev_empty_01">
- <output-dir compare="Text">stddev_empty_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="stddev_empty_02">
- <output-dir compare="Text">stddev_empty_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="stddev_float">
- <output-dir compare="Text">stddev_float</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="stddev_float_null">
- <output-dir compare="Text">stddev_float_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="stddev_int16">
- <output-dir compare="Text">stddev_int16</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="stddev_int16_null">
- <output-dir compare="Text">stddev_int16_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="stddev_int32">
- <output-dir compare="Text">stddev_int32</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="stddev_int32_null">
- <output-dir compare="Text">stddev_int32_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="stddev_int64">
- <output-dir compare="Text">stddev_int64</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="stddev_int64_null">
- <output-dir compare="Text">stddev_int64_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="stddev_int8">
- <output-dir compare="Text">stddev_int8</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="stddev_int8_null">
- <output-dir compare="Text">stddev_int8_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="stddev_distinct">
- <output-dir compare="Text">stddev_distinct</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="stddev_misc">
- <output-dir compare="Text">stddev_misc</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="stddev_pop_misc">
- <output-dir compare="Text">stddev_pop_misc</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="sum/sum_double">
- <output-dir compare="Text">sum/sum_double</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="sum/sum_double_null">
- <output-dir compare="Text">sum/sum_double_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="sum/sum_empty_01">
- <output-dir compare="Text">sum/sum_empty_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="sum/sum_empty_02">
- <output-dir compare="Text">sum/sum_empty_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="sum/sum_float">
- <output-dir compare="Text">sum/sum_float</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="sum/sum_float_null">
- <output-dir compare="Text">sum/sum_float_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="sum/sum_int16">
- <output-dir compare="Text">sum/sum_int16</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="sum/sum_int16_null">
- <output-dir compare="Text">sum/sum_int16_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="sum/sum_int32">
- <output-dir compare="Text">sum/sum_int32</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="sum/sum_int32_null">
- <output-dir compare="Text">sum/sum_int32_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="sum/sum_int64">
- <output-dir compare="Text">sum/sum_int64</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="sum/sum_int64_null">
- <output-dir compare="Text">sum/sum_int64_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="sum/sum_int64_null">
- <output-dir compare="Text">sum/sum_int64_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="sum/sum_int64_overflow">
- <output-dir compare="Text">sum/sum_int64_overflow</output-dir>
- <expected-error>Overflow in agg-sum</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="sum/sum_int8">
- <output-dir compare="Text">sum/sum_int8</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="sum/sum_int8_null">
- <output-dir compare="Text">sum/sum_int8_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="sum/sum_null-with-pred">
- <output-dir compare="Text">sum/sum_null-with-pred</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="sum/sum_numeric_null">
- <output-dir compare="Text">sum/sum_numeric_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="sum/sum_distinct">
- <output-dir compare="Text">sum/sum_distinct</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="var_double">
- <output-dir compare="Text">var_double</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="var_double_null">
- <output-dir compare="Text">var_double_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="var_empty_01">
- <output-dir compare="Text">var_empty_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="var_empty_02">
- <output-dir compare="Text">var_empty_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="var_float">
- <output-dir compare="Text">var_float</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="var_float_null">
- <output-dir compare="Text">var_float_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="var_int16">
- <output-dir compare="Text">var_int16</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="var_int16_null">
- <output-dir compare="Text">var_int16_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="var_int32">
- <output-dir compare="Text">var_int32</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="var_int32_null">
- <output-dir compare="Text">var_int32_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="var_int64">
- <output-dir compare="Text">var_int64</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="var_int64_null">
- <output-dir compare="Text">var_int64_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="var_int8">
- <output-dir compare="Text">var_int8</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="var_int8_null">
- <output-dir compare="Text">var_int8_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="var_distinct">
- <output-dir compare="Text">var_distinct</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="var_misc">
- <output-dir compare="Text">var_misc</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="var_pop_misc">
- <output-dir compare="Text">var_pop_misc</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="min_max_arrays">
- <output-dir compare="Text">min_max_arrays</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate">
- <compilation-unit name="non-pure-function">
- <output-dir compare="Text">non-pure-function</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="aggregate-sql">
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="min_max_arrays">
- <output-dir compare="Text">min_max_arrays</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="issue531_string_min_max">
- <output-dir compare="Text">issue531_string_min_max</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="agg_null">
- <output-dir compare="Text">agg_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="agg_null_rec">
- <output-dir compare="Text">agg_null_rec</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="agg_null_rec_1">
- <output-dir compare="Text">agg_null_rec_1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="agg_number_rec">
- <output-dir compare="Text">agg_number_rec</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql" check-warnings="true">
- <compilation-unit name="avg_mixed">
- <output-dir compare="Text">avg_mixed</output-dir>
- <expected-warn>Unsupported type: agg-avg cannot process input type string (in line 26, at column 16)</expected-warn>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql" check-warnings="true">
- <compilation-unit name="serial_avg_mixed">
- <output-dir compare="Text">serial_avg_mixed</output-dir>
- <expected-warn>Unsupported type: agg-avg cannot process input type string (in line 29, at column 38)</expected-warn>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="min_mixed">
- <output-dir compare="Text">min_mixed</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_stddev_mixed">
- <output-dir compare="Text">serial_stddev_mixed</output-dir>
- <expected-error>Invalid item type: function agg-stddev_samp cannot process item type string in an input array (or multiset)</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="stddev_mixed">
- <output-dir compare="Text">stddev_mixed</output-dir>
- <expected-error>Type incompatibility: function agg-stddev_samp gets incompatible input values: string and float</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_stddev_pop_mixed">
- <output-dir compare="Text">serial_stddev_pop_mixed</output-dir>
- <expected-error>Invalid item type: function agg-stddev_pop cannot process item type string in an input array (or multiset)</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="stddev_pop_mixed">
- <output-dir compare="Text">stddev_pop_mixed</output-dir>
- <expected-error>Type incompatibility: function agg-stddev_pop gets incompatible input values: string and float</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql" check-warnings="true">
- <compilation-unit name="sum/sum_mixed">
- <output-dir compare="Text">sum/sum_mixed</output-dir>
- <expected-warn>Unsupported type: agg-sum cannot process input type string (in line 27, at column 16)</expected-warn>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql" check-warnings="true">
- <compilation-unit name="sum/serial_sum_mixed">
- <output-dir compare="Text">sum/serial_sum_mixed</output-dir>
- <expected-warn>Unsupported type: agg-sum cannot process input type string (in line 29, at column 38)</expected-warn>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_var_mixed">
- <output-dir compare="Text">serial_var_mixed</output-dir>
- <expected-error>Invalid item type: function agg-var_samp cannot process item type string in an input array (or multiset)</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="var_mixed">
- <output-dir compare="Text">var_mixed</output-dir>
- <expected-error>Type incompatibility: function agg-var_samp gets incompatible input values: string and float</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_var_pop_mixed">
- <output-dir compare="Text">serial_var_pop_mixed</output-dir>
- <expected-error>Invalid item type: function agg-var_pop cannot process item type string in an input array (or multiset)</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="var_pop_mixed">
- <output-dir compare="Text">var_pop_mixed</output-dir>
- <expected-error>Type incompatibility: function agg-var_pop gets incompatible input values: string and float</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_skewness_mixed">
- <output-dir compare="Text">serial_skewness_mixed</output-dir>
- <expected-error>Invalid item type: function agg-skewness cannot process item type string in an input array (or multiset)</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="skewness_mixed">
- <output-dir compare="Text">skewness_mixed</output-dir>
- <expected-error>Type incompatibility: function agg-skewness gets incompatible input values: string and float</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="agg_number">
- <output-dir compare="Text">agg_number</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="issue425_min_hetero_list_1">
- <output-dir compare="Text">issue425_min_hetero_list_1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="issue425_min_hetero_list">
- <output-dir compare="Text">issue425_min_hetero_list</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="sum/issue425_sum_hetero_list_1">
- <output-dir compare="Text">sum/issue425_sum_hetero_list_1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="sum/issue425_sum_hetero_list">
- <output-dir compare="Text">sum/issue425_sum_hetero_list</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="query-issue400">
- <output-dir compare="Text">query-issue400</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="issue395">
- <output-dir compare="Text">issue395</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="issue412_0">
- <output-dir compare="Text">issue412_0</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="issue412_1">
- <output-dir compare="Text">issue412_1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="issue2348">
- <output-dir compare="Text">issue2348</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="issue2411">
- <output-dir compare="Text">issue2411</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="avg_double">
- <output-dir compare="Text">avg_double</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="avg_double_null">
- <output-dir compare="Text">avg_double_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="avg_empty_01">
- <output-dir compare="Text">avg_empty_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="avg_empty_02">
- <output-dir compare="Text">avg_empty_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="avg_float">
- <output-dir compare="Text">avg_float</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="avg_float_null">
- <output-dir compare="Text">avg_float_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="avg_int16">
- <output-dir compare="Text">avg_int16</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="avg_int16_null">
- <output-dir compare="Text">avg_int16_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="avg_int32">
- <output-dir compare="Text">avg_int32</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="avg_int32_null">
- <output-dir compare="Text">avg_int32_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="avg_int64">
- <output-dir compare="Text">avg_int64</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="avg_int64_null">
- <output-dir compare="Text">avg_int64_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="avg_int8">
- <output-dir compare="Text">avg_int8</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="avg_int8_null">
- <output-dir compare="Text">avg_int8_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="avg_distinct">
- <output-dir compare="Text">avg_distinct</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="count_01">
- <output-dir compare="Text">count_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="count_dataset">
- <output-dir compare="Text">count_dataset</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="count_empty_01">
- <output-dir compare="Text">count_empty_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="count_empty_02">
- <output-dir compare="Text">count_empty_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="count_null">
- <output-dir compare="Text">count_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="count_distinct">
- <output-dir compare="Text">count_distinct</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="kurtosis_double">
- <output-dir compare="Text">kurtosis_double</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="kurtosis_double_null">
- <output-dir compare="Text">kurtosis_double_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="kurtosis_empty_01">
- <output-dir compare="Text">kurtosis_empty_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="kurtosis_empty_02">
- <output-dir compare="Text">kurtosis_empty_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="kurtosis_float">
- <output-dir compare="Text">kurtosis_float</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="kurtosis_float_null">
- <output-dir compare="Text">kurtosis_float_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="kurtosis_int16">
- <output-dir compare="Text">kurtosis_int16</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="kurtosis_int16_null">
- <output-dir compare="Text">kurtosis_int16_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="kurtosis_int32">
- <output-dir compare="Text">kurtosis_int32</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="kurtosis_int32_null">
- <output-dir compare="Text">kurtosis_int32_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="kurtosis_int64">
- <output-dir compare="Text">kurtosis_int64</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="kurtosis_int64_null">
- <output-dir compare="Text">kurtosis_int64_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="kurtosis_int8">
- <output-dir compare="Text">kurtosis_int8</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="kurtosis_int8_null">
- <output-dir compare="Text">kurtosis_int8_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="kurtosis_distinct">
- <output-dir compare="Text">kurtosis_distinct</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="max_empty_01">
- <output-dir compare="Text">max_empty_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="max_empty_02">
- <output-dir compare="Text">max_empty_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="max_distinct">
- <output-dir compare="Text">max_distinct</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="min_empty_01">
- <output-dir compare="Text">min_empty_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="min_empty_02">
- <output-dir compare="Text">min_empty_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="min_distinct">
- <output-dir compare="Text">min_distinct</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="scalar_avg">
- <output-dir compare="Text">scalar_avg</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="scalar_avg_empty">
- <output-dir compare="Text">scalar_avg_empty</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="scalar_avg_null">
- <output-dir compare="Text">scalar_avg_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="scalar_count">
- <output-dir compare="Text">scalar_count</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="scalar_count_empty">
- <output-dir compare="Text">scalar_count_empty</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="scalar_count_null">
- <output-dir compare="Text">scalar_count_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="scalar_kurtosis">
- <output-dir compare="Text">scalar_kurtosis</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="scalar_kurtosis_empty">
- <output-dir compare="Text">scalar_kurtosis_empty</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="scalar_kurtosis_null">
- <output-dir compare="Text">scalar_kurtosis_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="scalar_max">
- <output-dir compare="Text">scalar_max</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="scalar_max_empty">
- <output-dir compare="Text">scalar_max_empty</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="scalar_max_null">
- <output-dir compare="Text">scalar_max_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="scalar_min">
- <output-dir compare="Text">scalar_min</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="scalar_min_empty">
- <output-dir compare="Text">scalar_min_empty</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="scalar_min_null">
- <output-dir compare="Text">scalar_min_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="scalar_skewness">
- <output-dir compare="Text">scalar_skewness</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="scalar_skewness_empty">
- <output-dir compare="Text">scalar_skewness_empty</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="scalar_skewness_null">
- <output-dir compare="Text">scalar_skewness_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="scalar_stddev">
- <output-dir compare="Text">scalar_stddev</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="scalar_stddev_empty">
- <output-dir compare="Text">scalar_stddev_empty</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="scalar_stddev_null">
- <output-dir compare="Text">scalar_stddev_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="sum/scalar_sum">
- <output-dir compare="Text">sum/scalar_sum</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="sum/scalar_sum_empty">
- <output-dir compare="Text">sum/scalar_sum_empty</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="sum/scalar_sum_null">
- <output-dir compare="Text">sum/scalar_sum_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="sum/scalar_sum_type">
- <output-dir compare="Text">sum/scalar_sum_type</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="scalar_var">
- <output-dir compare="Text">scalar_var</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="scalar_var_empty">
- <output-dir compare="Text">scalar_var_empty</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="scalar_var_null">
- <output-dir compare="Text">scalar_var_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_avg_empty">
- <output-dir compare="Text">serial_avg_empty</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_avg_int8">
- <output-dir compare="Text">serial_avg_int8</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_avg_int8_null">
- <output-dir compare="Text">serial_avg_int8_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_avg_int16">
- <output-dir compare="Text">serial_avg_int16</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_avg_int16_null">
- <output-dir compare="Text">serial_avg_int16_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_avg_int32">
- <output-dir compare="Text">serial_avg_int32</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_avg_int32_null">
- <output-dir compare="Text">serial_avg_int32_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_avg_int64">
- <output-dir compare="Text">serial_avg_int64</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_avg_int64_null">
- <output-dir compare="Text">serial_avg_int64_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_avg_float">
- <output-dir compare="Text">serial_avg_float</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_avg_float_null">
- <output-dir compare="Text">serial_avg_float_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_avg_double">
- <output-dir compare="Text">serial_avg_double</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_avg_double_null">
- <output-dir compare="Text">serial_avg_double_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_kurtosis_double">
- <output-dir compare="Text">serial_kurtosis_double</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_kurtosis_double_null">
- <output-dir compare="Text">serial_kurtosis_double_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_kurtosis_empty">
- <output-dir compare="Text">serial_kurtosis_empty</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_kurtosis_float">
- <output-dir compare="Text">serial_kurtosis_float</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_kurtosis_float_null">
- <output-dir compare="Text">serial_kurtosis_float_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_kurtosis_int16">
- <output-dir compare="Text">serial_kurtosis_int16</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_kurtosis_int16_null">
- <output-dir compare="Text">serial_kurtosis_int16_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_kurtosis_int32">
- <output-dir compare="Text">serial_kurtosis_int32</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_kurtosis_int32_null">
- <output-dir compare="Text">serial_kurtosis_int32_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_kurtosis_int64">
- <output-dir compare="Text">serial_kurtosis_int64</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_kurtosis_int64_null">
- <output-dir compare="Text">serial_kurtosis_int64_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_kurtosis_int8">
- <output-dir compare="Text">serial_kurtosis_int8</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_kurtosis_int8_null">
- <output-dir compare="Text">serial_kurtosis_int8_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_skewness_double">
- <output-dir compare="Text">serial_skewness_double</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_skewness_double_null">
- <output-dir compare="Text">serial_skewness_double_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_skewness_empty">
- <output-dir compare="Text">serial_skewness_empty</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_skewness_float">
- <output-dir compare="Text">serial_skewness_float</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_skewness_float_null">
- <output-dir compare="Text">serial_skewness_float_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_skewness_int16">
- <output-dir compare="Text">serial_skewness_int16</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_skewness_int16_null">
- <output-dir compare="Text">serial_skewness_int16_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_skewness_int32">
- <output-dir compare="Text">serial_skewness_int32</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_skewness_int32_null">
- <output-dir compare="Text">serial_skewness_int32_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_skewness_int64">
- <output-dir compare="Text">serial_skewness_int64</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_skewness_int64_null">
- <output-dir compare="Text">serial_skewness_int64_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_skewness_int8">
- <output-dir compare="Text">serial_skewness_int8</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_skewness_int8_null">
- <output-dir compare="Text">serial_skewness_int8_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_stddev_double">
- <output-dir compare="Text">serial_stddev_double</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_stddev_double_null">
- <output-dir compare="Text">serial_stddev_double_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_stddev_empty">
- <output-dir compare="Text">serial_stddev_empty</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_stddev_float">
- <output-dir compare="Text">serial_stddev_float</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_stddev_float_null">
- <output-dir compare="Text">serial_stddev_float_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_stddev_int16">
- <output-dir compare="Text">serial_stddev_int16</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_stddev_int16_null">
- <output-dir compare="Text">serial_stddev_int16_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_stddev_int32">
- <output-dir compare="Text">serial_stddev_int32</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_stddev_int32_null">
- <output-dir compare="Text">serial_stddev_int32_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_stddev_int64">
- <output-dir compare="Text">serial_stddev_int64</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_stddev_int64_null">
- <output-dir compare="Text">serial_stddev_int64_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_stddev_int8">
- <output-dir compare="Text">serial_stddev_int8</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_stddev_int8_null">
- <output-dir compare="Text">serial_stddev_int8_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="sum/serial_sum_double">
- <output-dir compare="Text">sum/serial_sum_double</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="sum/serial_sum_double_null">
- <output-dir compare="Text">sum/serial_sum_double_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="sum/serial_sum_empty">
- <output-dir compare="Text">sum/serial_sum_empty</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="sum/serial_sum_float">
- <output-dir compare="Text">sum/serial_sum_float</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="sum/serial_sum_float_null">
- <output-dir compare="Text">sum/serial_sum_float_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="sum/serial_sum_int16">
- <output-dir compare="Text">sum/serial_sum_int16</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="sum/serial_sum_int16_null">
- <output-dir compare="Text">sum/serial_sum_int16_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="sum/serial_sum_int32">
- <output-dir compare="Text">sum/serial_sum_int32</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="sum/serial_sum_int32_null">
- <output-dir compare="Text">sum/serial_sum_int32_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="sum/serial_sum_int64">
- <output-dir compare="Text">sum/serial_sum_int64</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="sum/serial_sum_int64_null">
- <output-dir compare="Text">sum/serial_sum_int64_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="sum/serial_sum_int64_overflow">
- <output-dir compare="Text">sum/serial_sum_int64_overflow</output-dir>
- <expected-error>Overflow in agg-sum</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="sum/serial_sum_int8">
- <output-dir compare="Text">sum/serial_sum_int8</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="sum/serial_sum_int8_null">
- <output-dir compare="Text">sum/serial_sum_int8_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_var_double">
- <output-dir compare="Text">serial_var_double</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_var_double_null">
- <output-dir compare="Text">serial_var_double_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_var_empty">
- <output-dir compare="Text">serial_var_empty</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_var_float">
- <output-dir compare="Text">serial_var_float</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_var_float_null">
- <output-dir compare="Text">serial_var_float_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_var_int16">
- <output-dir compare="Text">serial_var_int16</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_var_int16_null">
- <output-dir compare="Text">serial_var_int16_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_var_int32">
- <output-dir compare="Text">serial_var_int32</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_var_int32_null">
- <output-dir compare="Text">serial_var_int32_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_var_int64">
- <output-dir compare="Text">serial_var_int64</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_var_int64_null">
- <output-dir compare="Text">serial_var_int64_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_var_int8">
- <output-dir compare="Text">serial_var_int8</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="serial_var_int8_null">
- <output-dir compare="Text">serial_var_int8_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="skewness_double">
- <output-dir compare="Text">skewness_double</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="skewness_double_null">
- <output-dir compare="Text">skewness_double_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="skewness_empty_01">
- <output-dir compare="Text">skewness_empty_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="skewness_empty_02">
- <output-dir compare="Text">skewness_empty_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="skewness_float">
- <output-dir compare="Text">skewness_float</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="skewness_float_null">
- <output-dir compare="Text">skewness_float_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="skewness_int16">
- <output-dir compare="Text">skewness_int16</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="skewness_int16_null">
- <output-dir compare="Text">skewness_int16_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="skewness_int32">
- <output-dir compare="Text">skewness_int32</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="skewness_int32_null">
- <output-dir compare="Text">skewness_int32_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="skewness_int64">
- <output-dir compare="Text">skewness_int64</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="skewness_int64_null">
- <output-dir compare="Text">skewness_int64_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="skewness_int8">
- <output-dir compare="Text">skewness_int8</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="skewness_int8_null">
- <output-dir compare="Text">skewness_int8_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="skewness_distinct">
- <output-dir compare="Text">skewness_distinct</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="stddev_double">
- <output-dir compare="Text">stddev_double</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="stddev_double_null">
- <output-dir compare="Text">stddev_double_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="stddev_empty_01">
- <output-dir compare="Text">stddev_empty_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="stddev_empty_02">
- <output-dir compare="Text">stddev_empty_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="stddev_float">
- <output-dir compare="Text">stddev_float</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="stddev_float_null">
- <output-dir compare="Text">stddev_float_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="stddev_int16">
- <output-dir compare="Text">stddev_int16</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="stddev_int16_null">
- <output-dir compare="Text">stddev_int16_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="stddev_int32">
- <output-dir compare="Text">stddev_int32</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="stddev_int32_null">
- <output-dir compare="Text">stddev_int32_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="stddev_int64">
- <output-dir compare="Text">stddev_int64</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="stddev_int64_null">
- <output-dir compare="Text">stddev_int64_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="stddev_int8">
- <output-dir compare="Text">stddev_int8</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="stddev_int8_null">
- <output-dir compare="Text">stddev_int8_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="stddev_distinct">
- <output-dir compare="Text">stddev_distinct</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="stddev_misc">
- <output-dir compare="Text">stddev_misc</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="stddev_pop_misc">
- <output-dir compare="Text">stddev_pop_misc</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="sum/sum_double">
- <output-dir compare="Text">sum/sum_double</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="sum/sum_double_null">
- <output-dir compare="Text">sum/sum_double_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="sum/sum_empty_01">
- <output-dir compare="Text">sum/sum_empty_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="sum/sum_empty_02">
- <output-dir compare="Text">sum/sum_empty_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="sum/sum_float">
- <output-dir compare="Text">sum/sum_float</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="sum/sum_float_null">
- <output-dir compare="Text">sum/sum_float_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="sum/sum_int16">
- <output-dir compare="Text">sum/sum_int16</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="sum/sum_int16_null">
- <output-dir compare="Text">sum/sum_int16_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="sum/sum_int32">
- <output-dir compare="Text">sum/sum_int32</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="sum/sum_int32_null">
- <output-dir compare="Text">sum/sum_int32_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="sum/sum_int64">
- <output-dir compare="Text">sum/sum_int64</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="sum/sum_int64_null">
- <output-dir compare="Text">sum/sum_int64_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="sum/sum_int64_overflow">
- <output-dir compare="Text">sum/sum_int64_overflow</output-dir>
- <expected-error>Overflow in agg-sum</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="sum/sum_int8">
- <output-dir compare="Text">sum/sum_int8</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="sum/sum_int8_null">
- <output-dir compare="Text">sum/sum_int8_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="sum/sum_null-with-pred">
- <output-dir compare="Text">sum/sum_null-with-pred</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="sum/sum_numeric_null">
- <output-dir compare="Text">sum/sum_numeric_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="sum/sum_distinct">
- <output-dir compare="Text">sum/sum_distinct</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="var_double">
- <output-dir compare="Text">var_double</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="var_double_null">
- <output-dir compare="Text">var_double_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="var_empty_01">
- <output-dir compare="Text">var_empty_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="var_empty_02">
- <output-dir compare="Text">var_empty_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="var_float">
- <output-dir compare="Text">var_float</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="var_float_null">
- <output-dir compare="Text">var_float_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="var_int16">
- <output-dir compare="Text">var_int16</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="var_int16_null">
- <output-dir compare="Text">var_int16_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="var_int32">
- <output-dir compare="Text">var_int32</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="var_int32_null">
- <output-dir compare="Text">var_int32_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="var_int64">
- <output-dir compare="Text">var_int64</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="var_int64_null">
- <output-dir compare="Text">var_int64_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="var_int8">
- <output-dir compare="Text">var_int8</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="var_int8_null">
- <output-dir compare="Text">var_int8_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="var_distinct">
- <output-dir compare="Text">var_distinct</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="var_misc">
- <output-dir compare="Text">var_misc</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql">
- <compilation-unit name="var_pop_misc">
- <output-dir compare="Text">var_pop_misc</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="aggregate-sql-sugar">
- <test-case FilePath="aggregate-sql-sugar">
- <compilation-unit name="array_agg">
- <output-dir compare="Text">array_agg</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql-sugar">
- <compilation-unit name="array_agg_negative">
- <output-dir compare="Text">array_agg</output-dir>
- <expected-error>ASX1079: Compilation error: arrayagg is a SQL-92 aggregate function. The SQL++ core aggregate function strict_arrayagg could potentially express the intent.</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql-sugar" check-warnings="true">
- <compilation-unit name="avg_mixed">
- <output-dir compare="Text">avg_mixed</output-dir>
- <expected-warn>ASX0004: Unsupported type: agg-avg cannot process input type string (in line 26, at column 12)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: agg-avg cannot process input type string (in line 28, at column 19)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: agg-avg cannot process input type string (in line 29, at column 19)</expected-warn>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql-sugar">
- <compilation-unit name="distinct_mixed">
- <output-dir compare="Text">distinct_mixed</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql-sugar">
- <compilation-unit name="stddev">
- <output-dir compare="Text">stddev</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-sql-sugar">
- <compilation-unit name="var">
- <output-dir compare="Text">var</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="aggregate-subclause">
- <test-case FilePath="aggregate-subclause">
- <compilation-unit name="agg_filter_01">
- <output-dir compare="Text">agg_filter_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="aggregate-subclause">
- <compilation-unit name="agg_filter_02_neg">
- <output-dir compare="Text">agg_filter_01</output-dir>
- <expected-error>ASX1121: Illegal use of aggregate FILTER clause</expected-error>
- <expected-error>ASX1121: Illegal use of aggregate FILTER clause</expected-error>
- <expected-error>ASX1073: Cannot resolve alias reference for undefined identifier r</expected-error>
- <expected-error>ASX1121: Illegal use of aggregate FILTER clause</expected-error>
- <expected-error>ASX1121: Illegal use of aggregate FILTER clause</expected-error>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="array_fun">
- <test-case FilePath="array_fun">
- <compilation-unit name="array_prepend">
- <output-dir compare="Text">array_prepend</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array_fun">
- <compilation-unit name="array_append">
- <output-dir compare="Text">array_append</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array_fun">
- <compilation-unit name="array_position">
- <output-dir compare="Text">array_position</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array_fun">
- <compilation-unit name="array_repeat">
- <output-dir compare="Text">array_repeat</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array_fun">
- <compilation-unit name="array_reverse">
- <output-dir compare="Text">array_reverse</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array_fun">
- <compilation-unit name="array_contains">
- <output-dir compare="Text">array_contains</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array_fun">
- <compilation-unit name="array_insert">
- <output-dir compare="Text">array_insert</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array_fun">
- <compilation-unit name="array_put">
- <output-dir compare="Text">array_put</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array_fun">
- <compilation-unit name="array_remove">
- <output-dir compare="Text">array_remove</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array_fun">
- <compilation-unit name="array_distinct">
- <output-dir compare="Text">array_distinct</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array_fun">
- <compilation-unit name="array_sort">
- <output-dir compare="Text">array_sort</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array_fun">
- <compilation-unit name="array_concat">
- <output-dir compare="Text">array_concat</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array_fun">
- <compilation-unit name="array_flatten">
- <output-dir compare="Text">array_flatten</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array_fun">
- <compilation-unit name="array_ifnull">
- <output-dir compare="Text">array_ifnull</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array_fun">
- <compilation-unit name="array_intersect">
- <output-dir compare="Text">array_intersect</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array_fun">
- <compilation-unit name="array_range">
- <output-dir compare="Text">array_range</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array_fun">
- <compilation-unit name="array_replace">
- <output-dir compare="Text">array_replace</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array_fun">
- <compilation-unit name="array_star">
- <output-dir compare="Text">array_star</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array_fun">
- <compilation-unit name="array_symdiff">
- <output-dir compare="Text">array_symdiff</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array_fun">
- <compilation-unit name="array_symdiffn">
- <output-dir compare="Text">array_symdiffn</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array_fun">
- <compilation-unit name="array_union">
- <output-dir compare="Text">array_union</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array_fun">
- <compilation-unit name="array_slice/array_slice_double_argument">
- <output-dir compare="Text">array_slice/array_slice_double_argument</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array_fun">
- <compilation-unit name="array_slice/array_slice_int_argument">
- <output-dir compare="Text">array_slice/array_slice_int_argument</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array_fun">
- <compilation-unit name="array_slice/array_slice_negative_argument">
- <output-dir compare="Text">array_slice/array_slice_negative_argument</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array_fun">
- <compilation-unit name="array_slice/array_slice_missing_result">
- <output-dir compare="Text">array_slice/array_slice_missing_result</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array_fun">
- <compilation-unit name="array_slice/array_slice_null_result">
- <output-dir compare="Text">array_slice/array_slice_null_result</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array_fun">
- <compilation-unit name="array_slice/array_slice_exception_result">
- <output-dir compare="Text">array_slice/array_slice_exception_result</output-dir>
- <expected-error>ASX1081: Cannot find function with signature TinySocial.array_slice()</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array_fun">
- <compilation-unit name="array_except/001">
- <output-dir compare="Text">array_except/001</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array_fun">
- <compilation-unit name="array_except/002">
- <output-dir compare="Text">array_except/002</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array_fun">
- <compilation-unit name="array_except/003">
- <output-dir compare="Text">array_except/003</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array_fun">
- <compilation-unit name="array_except/004">
- <output-dir compare="Text">array_except/004</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array_fun">
- <compilation-unit name="array_except/005">
- <output-dir compare="Text">array_except/005</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array_fun">
- <compilation-unit name="array_except/006">
- <output-dir compare="Text">array_except/006</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array_fun">
- <compilation-unit name="array_errors">
- <output-dir compare="Text">array_errors</output-dir>
- <expected-error>Input contains different list types (in line 25, at column 8)</expected-error>
- <expected-error>Input contains different list types (in line 25, at column 8)</expected-error>
- <expected-error>Input contains different list types (in line 25, at column 8)</expected-error>
- <expected-error>Input contains different list types (in line 25, at column 8)</expected-error>
- <expected-error>Input contains different list types (in line 25, at column 8)</expected-error>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="bitwise">
- <test-case FilePath="bitwise">
- <compilation-unit name="bit_and_01">
- <output-dir compare="Text">bit_and_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="bitwise">
- <compilation-unit name="bit_and_02">
- <output-dir compare="Text">bit_and_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="bitwise">
- <compilation-unit name="bit_and_03">
- <output-dir compare="Text">bit_and_03</output-dir>
- <expected-error>Invalid number of arguments for function</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="bitwise">
- <compilation-unit name="bit_and_04">
- <output-dir compare="Text">bit_and_04</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="bitwise">
- <compilation-unit name="bit_or_01">
- <output-dir compare="Text">bit_or_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="bitwise">
- <compilation-unit name="bit_or_02">
- <output-dir compare="Text">bit_or_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="bitwise">
- <compilation-unit name="bit_or_03">
- <output-dir compare="Text">bit_or_03</output-dir>
- <expected-error>Invalid number of arguments for function</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="bitwise">
- <compilation-unit name="bit_or_04">
- <output-dir compare="Text">bit_or_04</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="bitwise">
- <compilation-unit name="bit_xor_01">
- <output-dir compare="Text">bit_xor_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="bitwise">
- <compilation-unit name="bit_xor_02">
- <output-dir compare="Text">bit_xor_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="bitwise">
- <compilation-unit name="bit_xor_03">
- <output-dir compare="Text">bit_xor_03</output-dir>
- <expected-error>Invalid number of arguments for function</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="bitwise">
- <compilation-unit name="bit_xor_04">
- <output-dir compare="Text">bit_xor_04</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="bitwise">
- <compilation-unit name="bit_not_01">
- <output-dir compare="Text">bit_not_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="bitwise">
- <compilation-unit name="bit_not_02">
- <output-dir compare="Text">bit_not_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="bitwise">
- <compilation-unit name="bit_not_03">
- <output-dir compare="Text">bit_not_03</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="bitwise">
- <compilation-unit name="bit_count_01">
- <output-dir compare="Text">bit_count_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="bitwise">
- <compilation-unit name="bit_count_02">
- <output-dir compare="Text">bit_count_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="bitwise">
- <compilation-unit name="bit_count_03">
- <output-dir compare="Text">bit_count_03</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="bitwise">
- <compilation-unit name="bit_set_01">
- <output-dir compare="Text">bit_set_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="bitwise">
- <compilation-unit name="bit_set_02">
- <output-dir compare="Text">bit_set_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="bitwise">
- <compilation-unit name="bit_set_03">
- <output-dir compare="Text">bit_set_03</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="bitwise">
- <compilation-unit name="bit_clear_01">
- <output-dir compare="Text">bit_clear_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="bitwise">
- <compilation-unit name="bit_clear_02">
- <output-dir compare="Text">bit_clear_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="bitwise">
- <compilation-unit name="bit_clear_03">
- <output-dir compare="Text">bit_clear_03</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="bitwise">
- <compilation-unit name="bit_shift_01">
- <output-dir compare="Text">bit_shift_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="bitwise">
- <compilation-unit name="bit_shift_02">
- <output-dir compare="Text">bit_shift_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="bitwise">
- <compilation-unit name="bit_shift_03">
- <output-dir compare="Text">bit_shift_03</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="bitwise">
- <compilation-unit name="bit_shift_04">
- <output-dir compare="Text">bit_shift_04</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="bitwise">
- <compilation-unit name="bit_test_01">
- <output-dir compare="Text">bit_test_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="bitwise">
- <compilation-unit name="bit_test_02">
- <output-dir compare="Text">bit_test_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="bitwise">
- <compilation-unit name="bit_test_03">
- <output-dir compare="Text">bit_test_03</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="bitwise">
- <compilation-unit name="bit_test_04">
- <output-dir compare="Text">bit_test_04</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="bitwise">
- <compilation-unit name="is_bit_set_01">
- <output-dir compare="Text">is_bit_set_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="bitwise">
- <compilation-unit name="is_bit_set_02">
- <output-dir compare="Text">is_bit_set_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="bitwise">
- <compilation-unit name="is_bit_set_03">
- <output-dir compare="Text">is_bit_set_03</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="boolean">
- <test-case FilePath="boolean">
- <compilation-unit name="and_01">
- <output-dir compare="Text">and_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="boolean">
- <compilation-unit name="and_null">
- <output-dir compare="Text">and_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="boolean">
- <compilation-unit name="and_null_false">
- <output-dir compare="Text">and_null_false</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="boolean">
- <compilation-unit name="not_01">
- <output-dir compare="Text">not_01</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="comparison">
- <test-case FilePath="comparison">
- <compilation-unit name="secondary_idx_lookup">
- <output-dir compare="Text">secondary_idx_lookup</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="year_month_duration_order">
- <output-dir compare="Text">year_month_duration_order</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="datetime_order">
- <output-dir compare="Text">datetime_order</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="datetime_range">
- <output-dir compare="Text">datetime_range</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="datetime_range_between">
- <output-dir compare="Text">datetime_range</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="datetime_tzeq">
- <output-dir compare="Text">datetime_tzeq</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="double">
- <output-dir compare="Text">double</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="double_gte_01">
- <output-dir compare="Text">double_gte_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="double_missing">
- <output-dir compare="Text">double_missing</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="double_null">
- <output-dir compare="Text">double_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="eq_01">
- <output-dir compare="Text">eq_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="float">
- <output-dir compare="Text">float</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="float_missing">
- <output-dir compare="Text">float_missing</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="float_null">
- <output-dir compare="Text">float_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="gt_01">
- <output-dir compare="Text">gt_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="gte_01">
- <output-dir compare="Text">gte_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="incompatible">
- <output-dir compare="Text">incompatible</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="int16">
- <output-dir compare="Text">int16</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="int16_missing">
- <output-dir compare="Text">int16_missing</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="int16_null">
- <output-dir compare="Text">int16_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="int32">
- <output-dir compare="Text">int32</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="int32_missing">
- <output-dir compare="Text">int32_missing</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="int32_null">
- <output-dir compare="Text">int32_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="int64">
- <output-dir compare="Text">int64</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="int64_missing">
- <output-dir compare="Text">int64_missing</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="int64_null">
- <output-dir compare="Text">int64_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="int8">
- <output-dir compare="Text">int8</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="int8_missing">
- <output-dir compare="Text">int8_missing</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="int8_null">
- <output-dir compare="Text">int8_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="int_not_between">
- <output-dir compare="Text">int_not_between</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="is_distinct_01">
- <output-dir compare="Text">is_distinct_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="like">
- <output-dir compare="Text">like</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="lt_01">
- <output-dir compare="Text">lt_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="lte_01">
- <output-dir compare="Text">lte_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="neq_01">
- <output-dir compare="Text">neq_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="neq_02">
- <output-dir compare="Text">neq_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="numeric-comparison_01">
- <output-dir compare="Text">numeric-comparison_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="string">
- <output-dir compare="Text">string</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="string_missing">
- <output-dir compare="Text">string_missing</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="string_null">
- <output-dir compare="Text">string_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="issue363_equality">
- <output-dir compare="Text">issue363_equality</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="issue363_inequality_duration">
- <output-dir compare="Text">issue363_inequality_duration</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="issue363_inequality_interval">
- <output-dir compare="Text">issue363_inequality_interval</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="issue363_inequality_point">
- <output-dir compare="Text">issue363_inequality_point</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="issue363_inequality_line">
- <output-dir compare="Text">issue363_inequality_line</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="issue363_inequality_polygon">
- <output-dir compare="Text">issue363_inequality_polygon</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="issue363_inequality_rectangle">
- <output-dir compare="Text">issue363_inequality_rectangle</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="issue363_inequality_circle">
- <output-dir compare="Text">issue363_inequality_circle</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="binary">
- <output-dir compare="Text">binary</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="binary_null">
- <output-dir compare="Text">binary_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="greatest_mixed">
- <output-dir compare="Text">greatest_mixed</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="least_mixed">
- <output-dir compare="Text">least_mixed</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="missingif">
- <output-dir compare="Text">missingif</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="nullif">
- <output-dir compare="Text">nullif</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="nanif">
- <output-dir compare="Text">nanif</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="posinfif">
- <output-dir compare="Text">posinfif</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="neginfif">
- <output-dir compare="Text">neginfif</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="arrays">
- <output-dir compare="Text">arrays</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison">
- <compilation-unit name="circle-point">
- <output-dir compare="Text">circle-point</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="comparison" check-warnings="true">
- <compilation-unit name="incomparable_types">
- <output-dir compare="Text">incomparable_types</output-dir>
- <expected-warn>Incomparable input types: string and bigint (in line 25, at column 13)</expected-warn>
- <expected-warn>Incomparable input types: array and bigint (in line 23, at column 7)</expected-warn>
- <expected-warn>Incomparable input types: point and point (in line 24, at column 18)</expected-warn>
- <expected-warn>Incomparable input types: bigint and string (in line 24, at column 46)</expected-warn>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="constructor">
- <test-case FilePath="constructor">
- <compilation-unit name="binary_01">
- <output-dir compare="Text">binary_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="constructor" check-warnings="true">
- <compilation-unit name="binary_02">
- <output-dir compare="Text">binary_02</output-dir>
- <expected-warn>ASX0006: Invalid format for binary in @#!1 (in line 24, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: hex() cannot process input type date (in line 25, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: hex() cannot process input type array (in line 26, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: hex() cannot process input type object (in line 27, at column 13)</expected-warn>
- <expected-warn>ASX0006: Invalid format for binary in @#!2 (in line 24, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: base64() cannot process input type date (in line 25, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: base64() cannot process input type array (in line 26, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: base64() cannot process input type object (in line 27, at column 13)</expected-warn>
- </compilation-unit>
- </test-case>
- <test-case FilePath="constructor">
- <compilation-unit name="add-null">
- <output-dir compare="Text">add-null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="constructor">
- <compilation-unit name="boolean_01">
- <output-dir compare="Text">boolean_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="constructor" check-warnings="true">
- <compilation-unit name="boolean_02">
- <output-dir compare="Text">boolean_02</output-dir>
- <expected-warn>ASX0006: Invalid format for boolean in FALSE (in line 24, at column 13)</expected-warn>
- <expected-warn>ASX0006: Invalid format for boolean in TRUE (in line 25, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: boolean() cannot process input type date (in line 26, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: boolean() cannot process input type array (in line 27, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: boolean() cannot process input type object (in line 28, at column 13)</expected-warn>
- </compilation-unit>
- </test-case>
- <test-case FilePath="constructor">
- <compilation-unit name="circle_01">
- <output-dir compare="Text">circle_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="constructor">
- <compilation-unit name="date_01">
- <output-dir compare="Text">date_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="constructor" check-warnings="true">
- <compilation-unit name="date_02">
- <output-dir compare="Text">date_02</output-dir>
- <expected-warn>ASX0006: Invalid format for date in @#! (in line 24, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: date() cannot process input type boolean (in line 25, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: date() cannot process input type bigint (in line 26, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: date() cannot process input type time (in line 27, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: date() cannot process input type array (in line 28, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: date() cannot process input type object (in line 29, at column 13)</expected-warn>
- <expected-warn>ASX0006: Invalid format for date in 12/31/2020 (in line 30, at column 13)</expected-warn>
- </compilation-unit>
- </test-case>
- <test-case FilePath="constructor">
- <compilation-unit name="datetime_01">
- <output-dir compare="Text">datetime_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="constructor" check-warnings="true">
- <compilation-unit name="datetime_02">
- <output-dir compare="Text">datetime_02</output-dir>
- <expected-warn>ASX0006: Invalid format for datetime in @#! (in line 24, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: datetime() cannot process input type boolean (in line 25, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: datetime() cannot process input type bigint (in line 26, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: datetime() cannot process input type array (in line 27, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: datetime() cannot process input type object (in line 28, at column 13)</expected-warn>
- <expected-warn>ASX0006: Invalid format for datetime in 1951-12-27T12:20:15Z (in line 29, at column 13)</expected-warn>
- </compilation-unit>
- </test-case>
- <test-case FilePath="constructor">
- <compilation-unit name="double_01">
- <output-dir compare="Text">double_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="constructor" check-warnings="true">
- <compilation-unit name="double_02">
- <output-dir compare="Text">double_02</output-dir>
- <expected-warn>ASX0006: Invalid format for double in @#! (in line 24, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: double() cannot process input type datetime (in line 25, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: double() cannot process input type date (in line 26, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: double() cannot process input type time (in line 27, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: double() cannot process input type array (in line 28, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: double() cannot process input type object (in line 29, at column 13)</expected-warn>
- </compilation-unit>
- </test-case>
- <test-case FilePath="constructor">
- <compilation-unit name="duration_01">
- <output-dir compare="Text">duration_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="constructor" check-warnings="true">
- <compilation-unit name="duration_02">
- <output-dir compare="Text">duration_02</output-dir>
- <expected-warn>ASX0006: Invalid format for duration in @#! (in line 24, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: duration() cannot process input type bigint (in line 25, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: duration() cannot process input type datetime (in line 26, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: duration() cannot process input type date (in line 27, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: duration() cannot process input type time (in line 28, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: duration() cannot process input type array (in line 29, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: duration() cannot process input type object (in line 30, at column 13)</expected-warn>
- <expected-warn>ASX0006: Invalid format for yearmonthduration in @#! (in line 24, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: year-month-duration() cannot process input type bigint (in line 25, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: year-month-duration() cannot process input type datetime (in line 26, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: year-month-duration() cannot process input type date (in line 27, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: year-month-duration() cannot process input type time (in line 28, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: year-month-duration() cannot process input type array (in line 29, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: year-month-duration() cannot process input type object (in line 30, at column 13)</expected-warn>
- <expected-warn>ASX0006: Invalid format for daytimeduration in @#! (in line 24, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: day-time-duration() cannot process input type bigint (in line 25, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: day-time-duration() cannot process input type datetime (in line 26, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: day-time-duration() cannot process input type date (in line 27, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: day-time-duration() cannot process input type time (in line 28, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: day-time-duration() cannot process input type array (in line 29, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: day-time-duration() cannot process input type object (in line 30, at column 13)</expected-warn>
- </compilation-unit>
- </test-case>
- <test-case FilePath="constructor">
- <compilation-unit name="float_01">
- <output-dir compare="Text">float_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="constructor" check-warnings="true">
- <compilation-unit name="float_02">
- <output-dir compare="Text">float_02</output-dir>
- <expected-warn>ASX0006: Invalid format for float in @#! (in line 24, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: float() cannot process input type datetime (in line 25, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: float() cannot process input type date (in line 26, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: float() cannot process input type time (in line 27, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: float() cannot process input type array (in line 28, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: float() cannot process input type object (in line 29, at column 13)</expected-warn>
- </compilation-unit>
- </test-case>
- <test-case FilePath="constructor">
- <compilation-unit name="int_01">
- <output-dir compare="Text">int_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="constructor" check-warnings="true">
- <compilation-unit name="int_02">
- <output-dir compare="Text">int_02</output-dir>
- <expected-warn>ASX0006: Invalid format for tinyint in @#! (in line 24, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: int8() cannot process input type datetime (in line 25, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: int8() cannot process input type date (in line 26, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: int8() cannot process input type time (in line 27, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: int8() cannot process input type array (in line 28, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: int8() cannot process input type object (in line 29, at column 13)</expected-warn>
- <expected-warn>ASX0006: Invalid format for smallint in @#! (in line 24, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: int16() cannot process input type datetime (in line 25, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: int16() cannot process input type date (in line 26, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: int16() cannot process input type time (in line 27, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: int16() cannot process input type array (in line 28, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: int16() cannot process input type object (in line 29, at column 13)</expected-warn>
- <expected-warn>ASX0006: Invalid format for integer in @#! (in line 24, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: int32() cannot process input type datetime (in line 25, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: int32() cannot process input type date (in line 26, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: int32() cannot process input type time (in line 27, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: int32() cannot process input type array (in line 28, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: int32() cannot process input type object (in line 29, at column 13)</expected-warn>
- <expected-warn>ASX0006: Invalid format for bigint in @#! (in line 24, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: int64() cannot process input type datetime (in line 25, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: int64() cannot process input type date (in line 26, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: int64() cannot process input type time (in line 27, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: int64() cannot process input type array (in line 28, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: int64() cannot process input type object (in line 29, at column 13)</expected-warn>
- </compilation-unit>
- </test-case>
- <test-case FilePath="constructor">
- <compilation-unit name="interval">
- <output-dir compare="Text">interval</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="constructor">
- <compilation-unit name="line_01">
- <output-dir compare="Text">line_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="constructor">
- <compilation-unit name="rectangle_01">
- <output-dir compare="Text">rectangle_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="constructor">
- <compilation-unit name="point_01">
- <output-dir compare="Text">point_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="constructor">
- <compilation-unit name="polygon_01">
- <output-dir compare="Text">polygon_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="constructor">
- <compilation-unit name="primitive-01">
- <output-dir compare="Text">primitive-01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="constructor">
- <compilation-unit name="primitive-02">
- <output-dir compare="Text">primitive-02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="constructor">
- <compilation-unit name="primitive-03">
- <output-dir compare="Text">primitive-03</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="constructor">
- <compilation-unit name="primitive-04">
- <output-dir compare="Text">primitive-04</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="constructor">
- <compilation-unit name="string_01">
- <output-dir compare="Text">string_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="constructor" check-warnings="true">
- <compilation-unit name="string_02">
- <output-dir compare="Text">string_02</output-dir>
- <expected-warn>ASX0004: Unsupported type: string() cannot process input type array (in line 24, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: string() cannot process input type object (in line 25, at column 13)</expected-warn>
- </compilation-unit>
- </test-case>
- <test-case FilePath="constructor">
- <compilation-unit name="time_01">
- <output-dir compare="Text">time_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="constructor" check-warnings="true">
- <compilation-unit name="time_02">
- <output-dir compare="Text">time_02</output-dir>
- <expected-warn>ASX0006: Invalid format for time in @#! (in line 24, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: time() cannot process input type boolean (in line 25, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: time() cannot process input type bigint (in line 26, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: time() cannot process input type date (in line 27, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: time() cannot process input type array (in line 28, at column 13)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: time() cannot process input type object (in line 29, at column 13)</expected-warn>
- <expected-warn>ASX0006: Invalid format for time in 11:58:59 (in line 30, at column 13)</expected-warn>
- </compilation-unit>
- </test-case>
- <test-case FilePath="constructor/uuid">
- <compilation-unit name="uuid_01">
- <output-dir compare="Text">uuid_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="constructor/uuid" check-warnings="true">
- <compilation-unit name="uuid_02">
- <output-dir compare="Text">uuid_02</output-dir>
- <expected-warn>ASX0006: Invalid format for uuid in 02a199ca-bf58-412e-bd9f-60a0c975a8a- (in line 24, at column 13)</expected-warn>
- <expected-warn>ASX0006: Invalid format for uuid in 12345 (in line 25, at column 13)</expected-warn>
- </compilation-unit>
- </test-case>
- <test-case FilePath="constructor">
- <compilation-unit name="polygon-from-open-list_issue1627">
- <output-dir compare="Text">polygon-from-open-list_issue1627</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="custord">
- <!--
- <test-case FilePath="custord">
- <compilation-unit name="co">
- <output-dir compare="Text">co</output-dir>
- </compilation-unit>
- </test-case>
- -->
- <test-case FilePath="custord">
- <compilation-unit name="customer_q_01">
- <output-dir compare="Text">customer_q_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="custord">
- <compilation-unit name="customer_q_02">
- <output-dir compare="Text">customer_q_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="custord">
- <compilation-unit name="customer_q_03">
- <output-dir compare="Text">customer_q_03</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="custord">
- <compilation-unit name="customer_q_04">
- <output-dir compare="Text">customer_q_04</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="custord">
- <compilation-unit name="customer_q_05">
- <output-dir compare="Text">customer_q_05</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="custord">
- <compilation-unit name="customer_q_06">
- <output-dir compare="Text">customer_q_06</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="custord">
- <compilation-unit name="customer_q_07">
- <output-dir compare="Text">customer_q_07</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="custord">
- <compilation-unit name="customer_q_08">
- <output-dir compare="Text">customer_q_08</output-dir>
- </compilation-unit>
- </test-case>
- <!--
- <test-case FilePath="custord">
- <compilation-unit name="denorm-cust-order_01">
- <output-dir compare="Text">denorm-cust-order_01</output-dir>
- </compilation-unit>
- </test-case>
- -->
- <test-case FilePath="custord">
- <compilation-unit name="denorm-cust-order_02">
- <output-dir compare="Text">denorm-cust-order_02</output-dir>
- </compilation-unit>
- </test-case>
- <!--
- <test-case FilePath="custord">
- <compilation-unit name="denorm-cust-order_03">
- <output-dir compare="Text">denorm-cust-order_03</output-dir>
- </compilation-unit>
- </test-case>
- -->
- <!--
- <test-case FilePath="custord">
- <compilation-unit name="freq-clerk">
- <output-dir compare="Text">freq-clerk</output-dir>
- </compilation-unit>
- </test-case>
- -->
- <test-case FilePath="custord">
- <compilation-unit name="join_q_01">
- <output-dir compare="Text">join_q_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="custord">
- <compilation-unit name="join_q_02">
- <output-dir compare="Text">join_q_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="custord">
- <compilation-unit name="join_q_03">
- <output-dir compare="Text">join_q_03</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="custord">
- <compilation-unit name="join_q_04">
- <output-dir compare="Text">join_q_04</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="custord">
- <compilation-unit name="join_q_05">
- <output-dir compare="Text">join_q_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="custord">
- <compilation-unit name="join_q_06">
- <output-dir compare="Text">join_q_06</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="custord">
- <compilation-unit name="join_q_07">
- <output-dir compare="Text">join_q_06</output-dir>
- <expected-error>Cannot find dataset c in dataverse test nor an alias with name c</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="custord">
- <compilation-unit name="join_q_08">
- <output-dir compare="Text">join_q_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="custord">
- <compilation-unit name="join_q_09">
- <output-dir compare="Text">join_q_01</output-dir>
- <expected-error>Cannot resolve ambiguous alias reference for identifier age</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="custord">
- <compilation-unit name="query-ASTERIXDB-1754">
- <output-dir compare="Text">query-ASTERIXDB-1754</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="custord">
- <compilation-unit name="load-test">
- <output-dir compare="Text">load-test</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="custord">
- <compilation-unit name="order_q_01">
- <output-dir compare="Text">order_q_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="custord">
- <compilation-unit name="order_q_02">
- <output-dir compare="Text">order_q_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="custord">
- <compilation-unit name="order_q_03">
- <output-dir compare="Text">order_q_03</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="custord">
- <compilation-unit name="order_q_04">
- <output-dir compare="Text">order_q_04</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="custord">
- <compilation-unit name="order_q_05">
- <output-dir compare="Text">order_q_05</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="custord">
- <compilation-unit name="order_q_06">
- <output-dir compare="Text">order_q_06</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="dapd">
- <test-case FilePath="dapd">
- <compilation-unit name="q1">
- <output-dir compare="Text">q1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dapd">
- <compilation-unit name="q2">
- <output-dir compare="Text">q2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dapd">
- <compilation-unit name="q2-2">
- <output-dir compare="Text">q2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dapd">
- <compilation-unit name="q2-2-negative">
- <output-dir compare="Text">q2</output-dir>
- <expected-error>Cannot find dataset e in dataverse test nor an alias with name e</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dapd">
- <compilation-unit name="q2-3">
- <output-dir compare="Text">q2</output-dir>
- <expected-error>Cannot resolve ambiguous alias reference for identifier sig_id</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dapd">
- <compilation-unit name="q2-4">
- <output-dir compare="Text">q2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dapd">
- <compilation-unit name="q2-5">
- <output-dir compare="Text">q2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dapd">
- <compilation-unit name="q2-6">
- <output-dir compare="Text">q2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dapd">
- <compilation-unit name="q2-7">
- <output-dir compare="Text">q2-7</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dapd">
- <compilation-unit name="q2-8">
- <output-dir compare="Text">q2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dapd">
- <compilation-unit name="q2-9">
- <output-dir compare="Text">q2-9</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dapd">
- <compilation-unit name="q2-10">
- <output-dir compare="Text">q2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dapd">
- <compilation-unit name="q2-11">
- <output-dir compare="Text">q2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dapd">
- <compilation-unit name="q2-12">
- <output-dir compare="Text">q2-12</output-dir>
- </compilation-unit>
- </test-case>
- <!--
- <test-case FilePath="dapd">
- <compilation-unit name="q3">
- <output-dir compare="Text">q3</output-dir>
- </compilation-unit>
- </test-case>
- -->
- </test-group>
- <test-group name="ddl">
- <test-case FilePath="ddl/create-index">
- <compilation-unit name="create-index-1">
- <output-dir compare="Text">create-index-1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="ddl/create-index">
- <compilation-unit name="create-index-2">
- <output-dir compare="Text">create-index-2</output-dir>
- <expected-error>Syntax error: In line 53 >>create primary index sec_primary_idx1 on LineItem type rtree;<< Encountered "rtree" at column 58.</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="ddl/create-index">
- <compilation-unit name="create-index-3">
- <output-dir compare="Text">create-index-3</output-dir>
- <expected-error>Syntax error: In line 53 >>create primary sec_primary_idx1 on LineItem;<< Encountered <IDENTIFIER> "sec_primary_idx1" at column 18.</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="ddl/create-index">
- <compilation-unit name="create-index-4">
- <output-dir compare="Text">create-index-4</output-dir>
- <expected-error>Syntax error: In line 53 >>create primary index if not exists sec_primary_idx1 if not exists on LineItem;<< Encountered <IDENTIFIER> "sec_primary_idx1" at column 37.</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="ddl/create-index">
- <compilation-unit name="create-index-5">
- <output-dir compare="Text">create-index-5</output-dir>
- <expected-error>Syntax error: In line 53 >>create primary index if exists sec_primary_idx1 on LineItem;<< Encountered "exists" at column 26.</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="ddl/create-index">
- <compilation-unit name="create-index-6">
- <output-dir compare="Text">none</output-dir>
- <expected-error>ASX1050: Cannot find dataset with name LineItemView1 in dataverse test (in line 55, at column 1)</expected-error>
- <expected-error>ASX1050: Cannot find dataset with name LineItemView2 in dataverse test (in line 60, at column 1)</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="ddl/create-index">
- <compilation-unit name="create-inverted-index-with-variable-length-primary-key">
- <output-dir compare="Text">create-inverted-index-with-variable-length-primary-key</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="ddl">
- <compilation-unit name="bad-type-ddl">
- <output-dir compare="Text">none</output-dir>
- <expected-error>ASX1079: Compilation error: Reserved type name $x</expected-error>
- <expected-error>ASX0013: Duplicate field name 'c' (in line 29, at column 19)</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="ddl">
- <compilation-unit name="create-dataset-inline-type-1">
- <output-dir compare="Text">create-dataset-inline-type-1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="ddl">
- <compilation-unit name="create-dataset-inline-type-2">
- <output-dir compare="Text">create-dataset-inline-type-2</output-dir>
- <expected-error>ASX1082: Cannot find datatype with name test.$d$t$i$Cust1</expected-error>
- <expected-error>ASX1082: Cannot find datatype with name test.$d$t$i$Cust2</expected-error>
- <expected-error>ASX1082: Cannot find datatype with name my_unknown_type</expected-error>
- <expected-error>ASX0013: Duplicate field name 'c_name' (in line 25, at column 22)</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="ddl">
- <compilation-unit name="drop-primary-index">
- <output-dir compare="Text">drop-primary-index</output-dir>
- <expected-error>Cannot drop index 'ds'. Drop dataset 'ds' to remove this index</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="ddl" check-warnings="true">
- <compilation-unit name="invalid-dataverse">
- <output-dir compare="Text">invalid-dataverse</output-dir>
- <source-location>false</source-location>
- <expected-warn>Cannot find dataverse with name fakeDataverse (in line 22, at column 1)</expected-warn>
- <expected-error>Cannot find dataverse with name fakeDataverse (in line 27, at column 1)</expected-error>
- <expected-warn>Cannot find dataverse with name fakeDataverse (in line 29, at column 1)</expected-warn>
- <expected-error>Cannot find dataverse with name fakeDataverse (in line 30, at column 1)</expected-error>
- <expected-error>Cannot find datatype with name fakeDataverse.myType</expected-error>
- <expected-error>Cannot find dataverse with name fakeDataverse (in line 30, at column 1)</expected-error>
- <expected-error>Cannot find dataverse with name fakeDataverse (in line 32, at column 1)</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="ddl">
- <compilation-unit name="invalid-dataverse-name">
- <output-dir compare="Text">none</output-dir>
- <expected-error>ASX1115: Invalid name for a database object: ''</expected-error>
- <expected-error>ASX1115: Invalid name for a database object: ' a'</expected-error>
- <expected-error>ASX1115: Invalid name for a database object: ' invalid'</expected-error>
- <expected-error>ASX1079: Compilation error: Invalid operation - Cannot create dataverse: asterix</expected-error>
- <expected-error>ASX1079: Compilation error: Invalid operation - Cannot create dataverse: algebricks</expected-error>
- <expected-error>ASX1115: Invalid name for a database object: '' (in line 24, at column 16)</expected-error>
- <expected-error>ASX1115: Invalid name for a database object: 'a/b' (in line 27, at column 16)</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="ddl">
- <compilation-unit name="invalid-dataset-name">
- <output-dir compare="Text">none</output-dir>
- <expected-error>ASX1115: Invalid name for a database object: ''</expected-error>
- <expected-error>ASX1115: Invalid name for a database object: ' a'</expected-error>
- <expected-error>ASX1115: Invalid name for a database object: ' invalid'</expected-error>
- <expected-error>ASX1115: Invalid name for a database object: 'a/b' (in line 29, at column 16)</expected-error>
- <expected-error>ASX1115: Invalid name for a database object: 'c/d' (in line 31, at column 14)</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="ddl">
- <compilation-unit name="invalid-feed-name">
- <output-dir compare="Text">none</output-dir>
- <expected-error>ASX1115: Invalid name for a database object: ''</expected-error>
- <expected-error>ASX1115: Invalid name for a database object: ' a'</expected-error>
- <expected-error>ASX1115: Invalid name for a database object: ' invalid'</expected-error>
- <expected-error>ASX1115: Invalid name for a database object: 'a/b' (in line 34, at column 13)</expected-error>
- <expected-error>ASX1115: Invalid name for a database object: 'c/d' (in line 42, at column 11)</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="ddl">
- <compilation-unit name="invalid-feed-policy-name">
- <output-dir compare="Text">none</output-dir>
- <expected-error>ASX1115: Invalid name for a database object: ''</expected-error>
- <expected-error>ASX1115: Invalid name for a database object: ' a'</expected-error>
- <expected-error>ASX1115: Invalid name for a database object: 'a/b' (in line 29, at column 1)</expected-error>
- <expected-error>ASX1115: Invalid name for a database object: 'c/d' (in line 32, at column 23)</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="ddl">
- <compilation-unit name="invalid-index-name">
- <output-dir compare="Text">none</output-dir>
- <expected-error>ASX1115: Invalid name for a database object: ''</expected-error>
- <expected-error>ASX1115: Invalid name for a database object: ' a'</expected-error>
- <expected-error>ASX1115: Invalid name for a database object: ' invalid'</expected-error>
- <expected-error>ASX1115: Invalid name for a database object: 'a/b' (in line 29, at column 19)</expected-error>
- <expected-error>ASX1115: Invalid name for a database object: 'c/d' (in line 33, at column 12)</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="ddl">
- <compilation-unit name="invalid-nodegroup-name">
- <output-dir compare="Text">none</output-dir>
- <expected-error>ASX1115: Invalid name for a database object: ''</expected-error>
- <expected-error>ASX1115: Invalid name for a database object: ' a'</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="ddl">
- <compilation-unit name="invalid-type-name">
- <output-dir compare="Text">none</output-dir>
- <expected-error>ASX1115: Invalid name for a database object: ''</expected-error>
- <expected-error>ASX1115: Invalid name for a database object: ' a'</expected-error>
- <expected-error>ASX1115: Invalid name for a database object: ' invalid'</expected-error>
- <expected-error>ASX1115: Invalid name for a database object: 'a/b' (in line 29, at column 13)</expected-error>
- <expected-error>ASX1115: Invalid name for a database object: 'c/d' (in line 33, at column 11)</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="ddl">
- <compilation-unit name="invalid-udf-name">
- <output-dir compare="Text">none</output-dir>
- <expected-error>ASX1115: Invalid name for a database object: ''</expected-error>
- <expected-error>ASX1115: Invalid name for a database object: ' a'</expected-error>
- <expected-error>ASX1115: Invalid name for a database object: ' invalid'</expected-error>
- <expected-error>ASX1115: Invalid name for a database object: 'a/b' (in line 29, at column 17)</expected-error>
- <expected-error>ASX1115: Invalid name for a database object: 'c/d' (in line 33, at column 15)</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="ddl">
- <compilation-unit name="invalid-view-name">
- <output-dir compare="Text">none</output-dir>
- <expected-error>ASX1115: Invalid name for a database object: '' (in line 29, at column 1)</expected-error>
- <expected-error>ASX1115: Invalid name for a database object: ' a' (in line 29, at column 1)</expected-error>
- <expected-error>ASX1115: Invalid name for a database object: ' invalid' (in line 29, at column 1)</expected-error>
- <expected-error>ASX1115: Invalid name for a database object: 'a/b' (in line 29, at column 13)</expected-error>
- <expected-error>ASX1115: Invalid name for a database object: 'c/d' (in line 32, at column 11)</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="ddl">
- <compilation-unit name="dataset-and-index-same-dataverse">
- <output-dir compare="Text">dataset-and-index-same-dataverse</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="ddl" check-warnings="true">
- <compilation-unit name="drop_dataset_invalid_dataverse">
- <output-dir compare="Text">drop_dataset_invalid_dataverse</output-dir>
- <expected-error>ASX1063: Cannot find dataverse with name fakeDataverse (in line 22, at column 1)</expected-error>
- <expected-error>ASX1063: Cannot find dataverse with name fakeDataverse (in line 22, at column 1)</expected-error>
- <expected-error>ASX1063: Cannot find dataverse with name fakeDataverse (in line 22, at column 1)</expected-error>
- <expected-error>ASX1063: Cannot find dataverse with name fakeDataverse (in line 22, at column 1)</expected-error>
- <expected-error>ASX1050: Cannot find dataset with name fakeDataset1 in dataverse realDataverse (in line 22, at column 1)</expected-error>
- <expected-warn>ASX1063: Cannot find dataverse with name fakeDataverse (in line 22, at column 1)</expected-warn>
- <expected-warn>ASX1063: Cannot find dataverse with name fakeDataverse (in line 22, at column 1)</expected-warn>
- <expected-warn>ASX1063: Cannot find dataverse with name fakeDataverse (in line 22, at column 1)</expected-warn>
- </compilation-unit>
- </test-case>
- <test-case FilePath="ddl">
- <compilation-unit name="create_dataset_with_filter_on_meta">
- <output-dir compare="Text">create_dataset_with_filter_on_meta</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="ddl">
- <compilation-unit name="index-bad-fields">
- <output-dir compare="Text">index-bad-fields</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="ddl">
- <compilation-unit name="index-cast-null">
- <placeholder name="with" value="" />
- <output-dir compare="Text">index-cast-null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="ddl">
- <compilation-unit name="index-cast-null">
- <placeholder name="with" value="WITH {'merge-policy': {'name': 'correlated-prefix','parameters': { 'max-mergable-component-size': 16384, 'max-tolerance-component-count': 3 }}}" />
- <output-dir compare="Text">index-cast-null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="ddl">
- <compilation-unit name="index-cast-null-negative">
- <output-dir compare="Text">index-cast-null-negative</output-dir>
- <expected-error>CAST modifier is only allowed for B-Tree indexes</expected-error>
- <expected-error>CAST modifier cannot be specified together with ENFORCED</expected-error>
- <expected-error>CAST modifier is used without specifying the type of the indexed field</expected-error>
- <expected-error>Typed index on 'typed_f2' field could be created only for open datatype</expected-error>
- <expected-error>Parameter invalid_date cannot be set</expected-error>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="dml">
- <test-case FilePath="dml">
- <compilation-unit name="insert-with-autogenerated-pk_adm-with-sec-primary-index">
- <output-dir compare="Text">insert-with-autogenerated-pk_adm-with-sec-primary-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="insert-returning-fieldname-qualified">
- <output-dir compare="Text">insert-returning-fieldname</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="compact-dataset-and-its-indexes">
- <output-dir compare="Text">compact-dataset-and-its-indexes</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="using-constant-merge-policy">
- <output-dir compare="Text">using-constant-merge-policy</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="using-prefix-merge-policy">
- <output-dir compare="Text">using-prefix-merge-policy</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="using-concurrent-merge-policy">
- <output-dir compare="Text">using-concurrent-merge-policy</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="using-correlated-prefix-merge-policy">
- <output-dir compare="Text">using-correlated-prefix-merge-policy</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="using-correlated-prefix-merge-policy-with-feed">
- <output-dir compare="Text">using-correlated-prefix-merge-policy</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="using-no-merge-policy">
- <output-dir compare="Text">using-no-merge-policy</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="query-issue382">
- <output-dir compare="Text">query-issue382</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="query-issue433">
- <output-dir compare="Text">query-issue433</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="query-issue288">
- <output-dir compare="Text">query-issue288</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="query-issue205">
- <output-dir compare="Text">query-issue205</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="delete-from-loaded-dataset-with-index">
- <output-dir compare="Text">delete-from-loaded-dataset-with-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="delete-from-loaded-dataset">
- <output-dir compare="Text">delete-from-loaded-dataset</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="delete-syntax-change">
- <output-dir compare="Text">delete-syntax-change</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="drop-empty-secondary-indexes">
- <output-dir compare="Text">drop-empty-secondary-indexes</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="drop-index">
- <output-dir compare="Text">drop-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="create-drop-cltype">
- <output-dir compare="Text">create-drop-cltype</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="create-drop-opntype">
- <output-dir compare="Text">create-drop-opntype</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="empty-load-with-index">
- <output-dir compare="Text">empty-load-with-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="insert-into-empty-dataset">
- <output-dir compare="Text">insert-into-empty-dataset</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="insert-into-empty-dataset-with-index">
- <output-dir compare="Text">insert-into-empty-dataset-with-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="insert-syntax">
- <output-dir compare="Text">insert-syntax</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="insert-and-scan-dataset">
- <output-dir compare="Text">insert-and-scan-dataset</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="insert-and-scan-dataset-with-index">
- <output-dir compare="Text">insert-and-scan-dataset-with-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="recreate-index">
- <output-dir compare="Text">recreate-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="insert-and-scan-joined-datasets">
- <output-dir compare="Text">insert-and-scan-joined-datasets</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="insert-into-loaded-dataset-with-index_01">
- <output-dir compare="Text">insert-into-loaded-dataset-with-index_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="insert-into-loaded-dataset-with-index_02">
- <output-dir compare="Text">insert-into-loaded-dataset-with-index_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="insert-into-loaded-dataset_01">
- <output-dir compare="Text">insert-into-loaded-dataset_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="insert-into-loaded-dataset_02">
- <output-dir compare="Text">insert-into-loaded-dataset_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="insert-src-dst-01">
- <output-dir compare="Text">insert-src-dst-01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="insert">
- <output-dir compare="Text">insert</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="insert-duplicated-keys">
- <output-dir compare="Text">insert-duplicated-keys</output-dir>
- <expected-error>Inserting duplicate keys into the primary storage</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="insert-duplicated-keys-with-pk-index">
- <output-dir compare="Text">insert-duplicated-keys-with-pk-index</output-dir>
- <expected-error>Inserting duplicate keys into the primary storage</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="insert_less_nc">
- <output-dir compare="Text">insert_less_nc</output-dir>
- </compilation-unit>
- </test-case>
- <!--
- <test-case FilePath="dml">
- <compilation-unit name="load-from-hdfs">
- <output-dir compare="Text">load-from-hdfs</output-dir>
- </compilation-unit>
- </test-case>
- -->
- <test-case FilePath="dml">
- <compilation-unit name="insert-with-autogenerated-pk_adm-with-sec-primary-index">
- <output-dir compare="Text">insert-with-autogenerated-pk_adm-with-sec-primary-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="insert-with-autogenerated-pk_adm_01">
- <output-dir compare="Text">insert-with-autogenerated-pk_adm_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="insert-with-autogenerated-pk_adm_02">
- <output-dir compare="Text">insert-with-autogenerated-pk_adm_02</output-dir>
- <expected-error>Field type string cannot be promoted to type uuid</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="insert-with-autogenerated-pk_adm_03">
- <output-dir compare="Text">insert-with-autogenerated-pk_adm_03</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="load-with-autogenerated-pk_txt_01">
- <output-dir compare="Text">load-with-autogenerated-pk_txt_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="load-with-autogenerated-pk_adm_01">
- <output-dir compare="Text">load-with-autogenerated-pk_adm_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="load-with-autogenerated-pk_adm_02">
- <output-dir compare="Text">load-with-autogenerated-pk_adm_02</output-dir>
- <expected-error>ASX3058: This record is closed, you can not add extra fields! new field name: id</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="load-with-autogenerated-pk_adm_03">
- <output-dir compare="Text">load-with-autogenerated-pk_adm_03</output-dir>
- <expected-error>ASX3058: This record is closed, you can not add extra fields! new field name: id</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="load-with-autogenerated-pk_adm_04">
- <output-dir compare="Text">load-with-autogenerated-pk_adm_04</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="load-with-autogenerated-pk_csv_01">
- <output-dir compare="Text">load-with-autogenerated-pk_csv_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="load-with-autogenerated-pk_csv_02">
- <output-dir compare="Text">load-with-autogenerated-pk_csv_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="load-with-autogenerated-no-field">
- <output-dir compare="Text">load-with-autogenerated-no-field</output-dir>
- <expected-error>ASX1014: Field 'not_id' is not found</expected-error>
- <expected-error>ASX1014: Field 'not_id' is not found</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="insert-return-records">
- <output-dir compare="Text">insert-return-records</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="insert-returning-udf">
- <output-dir compare="Text">insert-returning-fieldname</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="insert-returning-fieldname">
- <output-dir compare="Text">insert-returning-fieldname</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="insert-returning-fieldname-implicit">
- <output-dir compare="Text">insert-returning-fieldname</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="insert-returning-fieldname-implicit-2">
- <output-dir compare="Text">insert-returning-fieldname</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="insert_uuid_autogenerate">
- <output-dir compare="Text">insert_uuid_autogenerate</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="upsert_uuid_autogenerate">
- <output-dir compare="Text">upsert_uuid_autogenerate</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="insert_uuid_manual">
- <output-dir compare="Text">insert_uuid_manual</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="upsert_uuid_manual">
- <output-dir compare="Text">upsert_uuid_manual</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="insert_uuid_manual_exists">
- <output-dir compare="Text">insert_uuid_manual_exists</output-dir>
- <expected-error>Inserting duplicate keys into the primary storage</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="upsert_uuid_manual_exists">
- <output-dir compare="Text">upsert_uuid_manual_exists</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="insert_uuid_manual_exists_select">
- <output-dir compare="Text">insert_uuid_manual_exists_select</output-dir>
- <expected-error>Inserting duplicate keys into the primary storage</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="upsert_uuid_manual_exists_select">
- <output-dir compare="Text">upsert_uuid_manual_exists_select</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="insert_nested_uuid_autogenerate">
- <output-dir compare="Text">insert_nested_uuid_autogenerate</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="upsert_nested_uuid_autogenerate">
- <output-dir compare="Text">upsert_nested_uuid_autogenerate</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="insert_nested_uuid_manual">
- <output-dir compare="Text">insert_nested_uuid_manual</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="upsert_nested_uuid_manual">
- <output-dir compare="Text">upsert_nested_uuid_manual</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="insert_nested_uuid_manual_exists">
- <output-dir compare="Text">insert_nested_uuid_manual_exists</output-dir>
- <expected-error>Inserting duplicate keys into the primary storage</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="upsert_nested_uuid_manual_exists">
- <output-dir compare="Text">upsert_nested_uuid_manual_exists</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="insert_nested_uuid_manual_exists_select">
- <output-dir compare="Text">insert_nested_uuid_manual_exists_select</output-dir>
- <expected-error>Inserting duplicate keys into the primary storage</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="upsert_nested_uuid_manual_exists_select">
- <output-dir compare="Text">upsert_nested_uuid_manual_exists_select</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="insert-with-bad-return">
- <output-dir compare="Text">insert-with-bad-return</output-dir>
- <expected-error>A returning expression cannot contain dataset access</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="load-with-index">
- <output-dir compare="Text">load-with-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="load-with-ngram-index">
- <output-dir compare="Text">load-with-ngram-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="load-with-rtree-index">
- <output-dir compare="Text">load-with-rtree-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="load-with-word-index">
- <output-dir compare="Text">load-with-word-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="opentype-c2o-recursive">
- <output-dir compare="Text">opentype-c2o-recursive</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="opentype-c2o">
- <output-dir compare="Text">opentype-c2o</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="opentype-closed-optional">
- <output-dir compare="Text">opentype-closed-optional</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="opentype-insert">
- <output-dir compare="Text">opentype-insert</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="opentype-insert2">
- <output-dir compare="Text">opentype-insert2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="opentype-noexpand">
- <output-dir compare="Text">opentype-noexpand</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="opentype-o2c-recursive">
- <output-dir compare="Text">opentype-o2c-recursive</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="opentype-o2c">
- <output-dir compare="Text">opentype-o2c</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="opentype-o2o">
- <output-dir compare="Text">opentype-o2o</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="scan-delete-btree-secondary-index-nullable">
- <output-dir compare="Text">scan-delete-btree-secondary-index-nullable</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit
- name="scan-delete-btree-correlated-secondary-index-nullable">
- <output-dir compare="Text">scan-delete-btree-secondary-index-nullable</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="scan-delete-rtree-secondary-index-nullable">
- <output-dir compare="Text">scan-delete-rtree-secondary-index-nullable</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="scan-delete-rtree-correlated-secondary-index-nullable">
- <output-dir compare="Text">scan-delete-rtree-secondary-index-nullable</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="scan-delete-rtree-secondary-index">
- <output-dir compare="Text">scan-delete-rtree-secondary-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="scan-delete-rtree-correlated-secondary-index">
- <output-dir compare="Text">scan-delete-rtree-secondary-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="scan-insert-btree-secondary-index-nullable">
- <output-dir compare="Text">scan-insert-btree-secondary-index-nullable</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="scan-insert-btree-correlated-secondary-index-nullable">
- <output-dir compare="Text">scan-delete-btree-secondary-index-nullable</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="scan-insert-rtree-secondary-index-nullable">
- <output-dir compare="Text">scan-insert-rtree-secondary-index-nullable</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="scan-insert-rtree-correlated-secondary-index-nullable">
- <output-dir compare="Text">scan-insert-rtree-secondary-index-nullable</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="scan-insert-rtree-secondary-index">
- <output-dir compare="Text">scan-insert-rtree-secondary-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="scan-insert-rtree-correlated-secondary-index">
- <output-dir compare="Text">scan-insert-rtree-secondary-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="scan-insert-inverted-index-ngram-secondary-index">
- <output-dir compare="Text">scan-insert-inverted-index-ngram-secondary-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="scan-insert-inverted-index-ngram-secondary-index-string-as-primary-key">
- <output-dir compare="Text">scan-insert-inverted-index-ngram-secondary-index-string-as-primary-key</output-dir>
- </compilation-unit>
- </test-case>
-
- <test-case FilePath="dml">
- <compilation-unit name="scan-insert-inverted-index-ngram-correlated-secondary-index">
- <output-dir compare="Text">scan-insert-inverted-index-ngram-secondary-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="scan-insert-inverted-index-word-secondary-index">
- <output-dir compare="Text">scan-insert-inverted-index-word-secondary-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="scan-insert-inverted-index-word-secondary-index-string-as-primary-key">
- <output-dir compare="Text">scan-insert-inverted-index-word-secondary-index-string-as-primary-key</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="scan-insert-inverted-index-word-correlated-secondary-index">
- <output-dir compare="Text">scan-insert-inverted-index-word-secondary-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="scan-insert-inverted-index-word-correlated-secondary-index-string-as-primary-key">
- <output-dir compare="Text">scan-insert-inverted-index-word-secondary-index-string-as-primary-key</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="scan-insert-inverted-index-ngram-secondary-index-nullable">
- <output-dir compare="Text">scan-insert-inverted-index-ngram-secondary-index-nullable</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="scan-insert-inverted-index-ngram-secondary-index-nullable-string-as-primary-key">
- <output-dir compare="Text">scan-insert-inverted-index-ngram-secondary-index-nullable-string-as-primary-key</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="scan-insert-inverted-index-ngram-correlated-secondary-index-nullable">
- <output-dir compare="Text">scan-insert-inverted-index-ngram-secondary-index-nullable</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="scan-insert-inverted-index-ngram-correlated-secondary-index-nullable-string-as-primary-key">
- <output-dir compare="Text">scan-insert-inverted-index-ngram-secondary-index-nullable-string-as-primary-key</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="scan-insert-inverted-index-word-secondary-index-nullable">
- <output-dir compare="Text">scan-insert-inverted-index-word-secondary-index-nullable</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="scan-insert-inverted-index-word-secondary-index-nullable-string-as-primary-key">
- <output-dir compare="Text">scan-insert-inverted-index-word-secondary-index-nullable-string-as-primary-key</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="scan-insert-inverted-index-word-correlated-secondary-index-nullable">
- <output-dir compare="Text">scan-insert-inverted-index-word-secondary-index-nullable</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="scan-delete-inverted-index-ngram-secondary-index">
- <output-dir compare="Text">scan-delete-inverted-index-ngram-secondary-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="scan-delete-inverted-index-ngram-secondary-index-string-as-primary-key">
- <output-dir compare="Text">scan-delete-inverted-index-ngram-secondary-index-string-as-primary-key</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="scan-delete-inverted-index-ngram-correlated-secondary-index">
- <output-dir compare="Text">scan-delete-inverted-index-ngram-secondary-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="scan-delete-inverted-index-word-secondary-index">
- <output-dir compare="Text">scan-delete-inverted-index-word-secondary-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="scan-delete-inverted-index-word-correlated-secondary-index">
- <output-dir compare="Text">scan-delete-inverted-index-word-secondary-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="scan-delete-inverted-index-ngram-secondary-index-nullable">
- <output-dir compare="Text">scan-delete-inverted-index-ngram-secondary-index-nullable</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="scan-delete-inverted-index-ngram-correlated-secondary-index-nullable">
- <output-dir compare="Text">scan-delete-inverted-index-ngram-secondary-index-nullable</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="scan-delete-inverted-index-word-secondary-index-nullable">
- <output-dir compare="Text">scan-delete-inverted-index-word-secondary-index-nullable</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="scan-delete-inverted-index-word-correlated-secondary-index-nullable">
- <output-dir compare="Text">scan-delete-inverted-index-word-secondary-index-nullable</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="load-with-index-open">
- <output-dir compare="Text">load-with-index-open</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="load-with-ngram-index-open">
- <output-dir compare="Text">load-with-ngram-index-open</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="load-with-rtree-index-open">
- <output-dir compare="Text">load-with-rtree-index-open</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="load-with-word-index-open">
- <output-dir compare="Text">load-with-word-index-open</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="scan-delete-btree-secondary-index-open">
- <output-dir compare="Text">scan-delete-btree-secondary-index-open</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="scan-delete-btree-correlated-secondary-index-open">
- <output-dir compare="Text">scan-delete-btree-secondary-index-open</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="scan-delete-inverted-index-ngram-secondary-index-open">
- <output-dir compare="Text">scan-delete-inverted-index-ngram-secondary-index-open</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="scan-delete-inverted-index-ngram-correlated-secondary-index-open">
- <output-dir compare="Text">scan-delete-inverted-index-ngram-secondary-index-open</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="scan-delete-inverted-index-word-secondary-index-open">
- <output-dir compare="Text">scan-delete-inverted-index-word-secondary-index-open</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="scan-delete-inverted-index-word-correlated-secondary-index-open">
- <output-dir compare="Text">scan-delete-inverted-index-word-secondary-index-open</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="scan-delete-rtree-secondary-index-open">
- <output-dir compare="Text">scan-delete-rtree-secondary-index-open</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="scan-delete-rtree-correlated-secondary-index-open">
- <output-dir compare="Text">scan-delete-rtree-secondary-index-open</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="scan-insert-btree-secondary-index-open">
- <output-dir compare="Text">scan-insert-btree-secondary-index-open</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="scan-insert-btree-correlated-secondary-index-open">
- <output-dir compare="Text">scan-delete-btree-secondary-index-open</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="scan-insert-inverted-index-ngram-secondary-index-open">
- <output-dir compare="Text">scan-insert-inverted-index-ngram-secondary-index-open</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="scan-insert-inverted-index-ngram-correlated-secondary-index-open">
- <output-dir compare="Text">scan-insert-inverted-index-ngram-secondary-index-open</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="scan-insert-inverted-index-word-secondary-index-open">
- <output-dir compare="Text">scan-insert-inverted-index-word-secondary-index-open</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="scan-insert-inverted-index-word-correlated-secondary-index-open">
- <output-dir compare="Text">scan-insert-inverted-index-word-secondary-index-open</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="scan-insert-rtree-secondary-index-open">
- <output-dir compare="Text">scan-insert-rtree-secondary-index-open</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="scan-insert-rtree-correlated-secondary-index-open">
- <output-dir compare="Text">scan-insert-rtree-secondary-index-open</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="delete-multi-statement">
- <output-dir compare="Text">delete-multi-statement</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="upsert-dataset-with-meta">
- <output-dir compare="Text">upsert-dataset-with-meta</output-dir>
- <expected-error>upsert into dataset is not supported on datasets with meta records</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="upsert-return-custom-result">
- <output-dir compare="Text">upsert-return-custom-result</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="upsert-returning-fieldname">
- <output-dir compare="Text">insert-returning-fieldname</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="upsert-returning-fieldname-implicit">
- <output-dir compare="Text">insert-returning-fieldname</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="upsert-returning-fieldname-implicit-2">
- <output-dir compare="Text">insert-returning-fieldname</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="delete-dataset-with-meta">
- <output-dir compare="Text">delete-dataset-with-meta</output-dir>
- <expected-error>delete from dataset is not supported on datasets with meta records</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="insert-dataset-with-meta">
- <output-dir compare="Text">insert-dataset-with-meta</output-dir>
- <expected-error>insert into dataset is not supported on datasets with meta records</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="insert-duplicated-keys-from-query">
- <output-dir compare="Text">insert-duplicated-keys-from-query</output-dir>
- <expected-error>HYR0033: Inserting duplicate keys into the primary storage</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="load-with-autogenerated-no-field">
- <output-dir compare="Text">load-with-autogenerated-no-field</output-dir>
- <expected-error>ASX1014: Field 'not_id' is not found</expected-error>
- <expected-error>ASX1014: Field 'not_id' is not found</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="load-with-index-open_02">
- <output-dir compare="Text">load-with-index-open_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="query-ASTERIXDB-867">
- <output-dir compare="Text">query-ASTERIXDB-867</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="query-ASTERIXDB-1406">
- <output-dir compare="Text">query-ASTERIXDB-1406</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="query-issue382">
- <output-dir compare="Text">query-issue382</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="create-index-unknown-key">
- <output-dir compare="Text">index-unknown-key</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="insert-into-index-unknown-key">
- <output-dir compare="Text">index-unknown-key</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="upsert-into-index-unknown-key">
- <output-dir compare="Text">index-unknown-key</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="dml">
- <compilation-unit name="load-into-index-unknown-key">
- <output-dir compare="Text">index-unknown-key</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="employee">
- <test-case FilePath="employee">
- <compilation-unit name="q_01">
- <output-dir compare="Text">q_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="employee">
- <compilation-unit name="q_02">
- <output-dir compare="Text">q_02</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="failure">
- <test-case FilePath="failure">
- <compilation-unit name="group_by_failure">
- <output-dir compare="Text">group_by_failure</output-dir>
- <expected-error>Injected failure in inject-failure</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="failure">
- <compilation-unit name="group_by_hash_failure">
- <output-dir compare="Text">group_by_hash_failure</output-dir>
- <expected-error>Injected failure in inject-failure</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="failure">
- <compilation-unit name="q01_pricing_summary_report_failure">
- <output-dir compare="Text">q01_pricing_summary_report_failure</output-dir>
- <expected-error>Injected failure in inject-failure</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="failure">
- <compilation-unit name="q18_large_volume_customer_failure">
- <output-dir compare="Text">q18_large_volume_customer_failure</output-dir>
- <expected-error>Injected failure in inject-failure</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="failure">
- <compilation-unit name="order_by_failure">
- <output-dir compare="Text">order_by_failure</output-dir>
- <expected-error>Injected failure in inject-failure</expected-error>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="fuzzyjoin">
- <test-case FilePath="fuzzyjoin">
- <compilation-unit name="basic-1_1">
- <output-dir compare="Text">basic-1_1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="fuzzyjoin">
- <compilation-unit name="basic-1_1_2">
- <output-dir compare="Text">basic-1_1_2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="fuzzyjoin">
- <compilation-unit name="basic-1_1_3">
- <output-dir compare="Text">basic-1_1_3</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="fuzzyjoin">
- <compilation-unit name="dblp-string-as-primary-key">
- <output-dir compare="Text">dblp-string-as-primary-key</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <!--
- <test-group name="flwor">
- <test-case FilePath="flwor">
- <compilation-unit name="for01">
- <output-dir compare="Text">for01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="for02">
- <output-dir compare="Text">for02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="for03">
- <output-dir compare="Text">for03</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="for04">
- <output-dir compare="Text">for04</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="for05">
- <output-dir compare="Text">for05</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="for06">
- <output-dir compare="Text">for06</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="for07">
- <output-dir compare="Text">for07</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="for08">
- <output-dir compare="Text">for08</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="for09">
- <output-dir compare="Text">for09</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="for10">
- <output-dir compare="Text">for10</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="for11">
- <output-dir compare="Text">for11</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="for12">
- <output-dir compare="Text">for12</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="for13">
- <output-dir compare="Text">for13</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="for14">
- <output-dir compare="Text">for14</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="for15">
- <output-dir compare="Text">for15</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="for16">
- <output-dir compare="Text">for16</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="for17">
- <output-dir compare="Text">for17</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="for18">
- <output-dir compare="Text">for18</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="for19">
- <output-dir compare="Text">for19</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="grpby01">
- <output-dir compare="Text">grpby01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="grpby02">
- <output-dir compare="Text">grpby02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="let01">
- <output-dir compare="Text">let01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="let02">
- <output-dir compare="Text">let02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="let03">
- <output-dir compare="Text">let03</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="let04">
- <output-dir compare="Text">let04</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="let05">
- <output-dir compare="Text">let05</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="let06">
- <output-dir compare="Text">let06</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="let07">
- <output-dir compare="Text">let07</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="let08">
- <output-dir compare="Text">let08</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="let09">
- <output-dir compare="Text">let09</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="let10">
- <output-dir compare="Text">let10</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="let11">
- <output-dir compare="Text">let11</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="let12">
- <output-dir compare="Text">let12</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="let13">
- <output-dir compare="Text">let13</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="let14">
- <output-dir compare="Text">let14</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="let15">
- <output-dir compare="Text">let15</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="let16">
- <output-dir compare="Text">let16</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="let17">
- <output-dir compare="Text">let17</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="let18">
- <output-dir compare="Text">let18</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="let19">
- <output-dir compare="Text">let19</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="let20">
- <output-dir compare="Text">let20</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="let21">
- <output-dir compare="Text">let21</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="let22">
- <output-dir compare="Text">let22</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="let23">
- <output-dir compare="Text">let23</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="let24">
- <output-dir compare="Text">let24</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="let25">
- <output-dir compare="Text">let25</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="let26">
- <output-dir compare="Text">let26</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="let27">
- <output-dir compare="Text">let27</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="let28">
- <output-dir compare="Text">let28</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="let29">
- <output-dir compare="Text">let29</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="let30">
- <output-dir compare="Text">let30</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="let31">
- <output-dir compare="Text">let31</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="let32">
- <output-dir compare="Text">let32</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="order-by-01">
- <output-dir compare="Text">order-by-01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="order-by-02">
- <output-dir compare="Text">order-by-02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="order-by-03">
- <output-dir compare="Text">order-by-03</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="order-by-04">
- <output-dir compare="Text">order-by-04</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="order-by-05">
- <output-dir compare="Text">order-by-05</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="order-by-06">
- <output-dir compare="Text">order-by-06</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="order-by-07">
- <output-dir compare="Text">order-by-07</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="order-by-08">
- <output-dir compare="Text">order-by-08</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="order-by-09">
- <output-dir compare="Text">order-by-09</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="order-by-10">
- <output-dir compare="Text">order-by-10</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="order-by-11">
- <output-dir compare="Text">order-by-11</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="order-by-12">
- <output-dir compare="Text">order-by-12</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="ret-01">
- <output-dir compare="Text">ret-01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="ret-02">
- <output-dir compare="Text">ret-02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="ret-03">
- <output-dir compare="Text">ret-03</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="ret-04">
- <output-dir compare="Text">ret-04</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="ret-05">
- <output-dir compare="Text">ret-05</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="ret-06">
- <output-dir compare="Text">ret-06</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="ret-07">
- <output-dir compare="Text">ret-07</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="ret-08">
- <output-dir compare="Text">ret-08</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="ret-09">
- <output-dir compare="Text">ret-09</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="ret-10">
- <output-dir compare="Text">ret-10</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="ret-11">
- <output-dir compare="Text">ret-11</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="ret-12">
- <output-dir compare="Text">ret-12</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="ret-13">
- <output-dir compare="Text">ret-13</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="ret-14">
- <output-dir compare="Text">ret-14</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="ret-15">
- <output-dir compare="Text">ret-15</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="writers">
- <test-case FilePath="writers">
- <compilation-unit name="print_01">
- <output-dir compare="Text">print_01</output-dir>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="ret-16">
- <output-dir compare="Text">ret-16</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="ret-17">
- <output-dir compare="Text">ret-17</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="ret-18">
- <output-dir compare="Text">ret-18</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="flwor">
- <compilation-unit name="ret-19">
- <output-dir compare="Text">ret-19</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- -->
- <test-group name="fulltext">
- <test-case FilePath="fulltext">
- <compilation-unit name="fulltext-01">
- <output-dir compare="Text">fulltext-01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="fulltext">
- <compilation-unit name="fulltext-02">
- <output-dir compare="Text">fulltext-02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="fulltext">
- <compilation-unit name="fulltext-04">
- <output-dir compare="Text">fulltext-04</output-dir>
- <expected-error>ASX1010: Phrase search in Full-text is not yet supported. Only one keyword per expression is permitted</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="fulltext">
- <compilation-unit name="fulltext-05">
- <output-dir compare="Text">fulltext-05</output-dir>
- <expected-error>ASX1010: Phrase search in Full-text is not yet supported. Only one keyword per expression is permitted</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="fulltext">
- <compilation-unit name="fulltext-06">
- <output-dir compare="Text">fulltext-06</output-dir>
- <expected-error>ASX1010: Phrase search in Full-text is not yet supported. Only one keyword per expression is permitted</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="fulltext">
- <compilation-unit name="fulltext-08">
- <output-dir compare="Text">fulltext-08</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="fulltext">
- <compilation-unit name="fulltext-09">
- <output-dir compare="Text">fulltext-09</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="fulltext">
- <compilation-unit name="fulltext-index-01">
- <output-dir compare="Text">fulltext-index-01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="fulltext">
- <compilation-unit name="fulltext-index-02">
- <output-dir compare="Text">fulltext-index-02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="fulltext">
- <compilation-unit name="fulltext-index-04">
- <output-dir compare="Text">fulltext-index-04</output-dir>
- <expected-error>ASX1010: Phrase search in Full-text is not yet supported. Only one keyword per expression is permitted</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="fulltext">
- <compilation-unit name="fulltext-index-05">
- <output-dir compare="Text">fulltext-index-05</output-dir>
- <expected-error>ASX1010: Phrase search in Full-text is not yet supported. Only one keyword per expression is permitted</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="fulltext">
- <compilation-unit name="fulltext-index-06">
- <output-dir compare="Text">fulltext-index-06</output-dir>
- <expected-error>ASX1010: Phrase search in Full-text is not yet supported. Only one keyword per expression is permitted</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="fulltext">
- <compilation-unit name="fulltext-index-08">
- <output-dir compare="Text">fulltext-index-08</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="fulltext">
- <compilation-unit name="fulltext-index-09">
- <output-dir compare="Text">fulltext-index-09</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="fulltext">
- <compilation-unit name="fulltext-index-large-data">
- <output-dir compare="Text">fulltext-index-large-data</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="fulltext">
- <compilation-unit name="stopwords-full-text-filter-1">
- <output-dir compare="Text">stopwords-full-text-filter-1</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="global-aggregate">
- <test-case FilePath="global-aggregate">
- <compilation-unit name="q01">
- <output-dir compare="Text">q01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="global-aggregate">
- <compilation-unit name="q02">
- <output-dir compare="Text">q02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="global-aggregate">
- <compilation-unit name="q03">
- <output-dir compare="Text">q01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="global-aggregate">
- <compilation-unit name="q04">
- <output-dir compare="Text">q01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="global-aggregate">
- <compilation-unit name="q05_error">
- <output-dir compare="Text">q01</output-dir>
- <expected-error>ASX1073: Cannot resolve alias reference for undefined identifier u (in line 22, at column 8)</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="global-aggregate">
- <compilation-unit name="q06_error">
- <output-dir compare="Text">q01</output-dir>
- <expected-error>ASX0037: Type mismatch: expected value of type array or multiset, but got the value of type string (in line 22, at column 8)</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="global-aggregate">
- <compilation-unit name="q07_error">
- <output-dir compare="Text">q01</output-dir>
- <expected-error>count is a SQL-92 aggregate function. The SQL++ core aggregate function array_count could potentially express the intent.</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="global-aggregate">
- <compilation-unit name="q08">
- <output-dir compare="Text">q08</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="global-aggregate">
- <compilation-unit name="q09">
- <output-dir compare="Text">q09</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="global-aggregate">
- <compilation-unit name="q10">
- <output-dir compare="Text">q09</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="global-aggregate">
- <compilation-unit name="q11">
- <output-dir compare="Text">q01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="global-aggregate">
- <compilation-unit name="q12_error">
- <output-dir compare="Text">q01</output-dir>
- <expected-error>The parameter * can only be used in count().</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="global-aggregate">
- <compilation-unit name="query-ASTERIXDB-159">
- <output-dir compare="Text">query-ASTERIXDB-159</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="global-aggregate">
- <compilation-unit name="query-ASTERIXDB-1626">
- <output-dir compare="Text">query-ASTERIXDB-1626</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="global-aggregate">
- <compilation-unit name="query-ASTERIXDB-1626-2">
- <output-dir compare="Text">query-ASTERIXDB-1626-2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="global-aggregate">
- <compilation-unit name="query-ASTERIXDB-2525">
- <output-dir compare="Text">query-ASTERIXDB-2525</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="group-by">
- <test-case FilePath="group-by">
- <compilation-unit name="gby-array">
- <output-dir compare="Text">gby-array</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="group-by">
- <compilation-unit name="gby-case-01">
- <output-dir compare="Text">gby-case-01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="group-by">
- <compilation-unit name="gby-cross-join">
- <output-dir compare="Text">gby-cross-join</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="group-by">
- <compilation-unit name="gby-nested-01">
- <output-dir compare="Text">gby-nested-01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="group-by">
- <compilation-unit name="gby-record">
- <output-dir compare="Text">gby-record</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="group-by">
- <compilation-unit name="core-01">
- <output-dir compare="Text">core-01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="group-by">
- <compilation-unit name="core-01-error">
- <output-dir compare="Text">none</output-dir>
- <expected-error>Cannot find dataset e in dataverse gby nor an alias with name e</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="group-by">
- <compilation-unit name="core-02-error">
- <output-dir compare="Text">none</output-dir>
- <expected-error>Cannot find dataset f in dataverse gby nor an alias with name f</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="group-by">
- <compilation-unit name="core-02">
- <output-dir compare="Text">core-02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="group-by">
- <compilation-unit name="core-03">
- <output-dir compare="Text">core-03</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="group-by">
- <compilation-unit name="core-04">
- <output-dir compare="Text">core-04</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="group-by">
- <compilation-unit name="core-05">
- <output-dir compare="Text">core-05</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="group-by">
- <compilation-unit name="core-06">
- <output-dir compare="Text">core-06</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="group-by">
- <compilation-unit name="core-07-error">
- <output-dir compare="Text">none</output-dir>
- <expected-error><![CDATA[ASX1001: Syntax error: In line 25 >>GROUP BY x, y;<< Encountered "GROUP" at column 1]]></expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="group-by">
- <compilation-unit name="sugar-01">
- <output-dir compare="Text">core-01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="group-by">
- <compilation-unit name="sugar-01-negative">
- <output-dir compare="Text">core-01</output-dir>
- <expected-error>ASX0037: Type mismatch: expected value of type array or multiset, but got the value of type bigint (in line 26, at column 26)</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="group-by">
- <compilation-unit name="sugar-01-2">
- <output-dir compare="Text">core-01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="group-by">
- <compilation-unit name="sugar-02">
- <output-dir compare="Text">core-02</output-dir>
- <expected-error>Cannot resolve ambiguous alias reference for identifier deptId</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="group-by">
- <compilation-unit name="sugar-02-2">
- <output-dir compare="Text">core-02</output-dir>
- <expected-error>ASX1073: Cannot resolve alias reference for undefined identifier deptId (in line 28, at column 8)</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="group-by">
- <compilation-unit name="sugar-03">
- <output-dir compare="Text">core-03</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="group-by">
- <compilation-unit name="sugar-03-2">
- <output-dir compare="Text">core-03</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="group-by">
- <compilation-unit name="sugar-04">
- <output-dir compare="Text">core-04</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="group-by">
- <compilation-unit name="sugar-04-2">
- <output-dir compare="Text">core-04</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="group-by">
- <compilation-unit name="sugar-05">
- <output-dir compare="Text">core-05</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="group-by">
- <compilation-unit name="sugar-05-2">
- <output-dir compare="Text">core-05</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="group-by">
- <compilation-unit name="sugar-06-distinct">
- <output-dir compare="Text">sugar-06-distinct</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="group-by">
- <compilation-unit name="sugar-07-negative">
- <output-dir compare="Text">core-01</output-dir>
- <expected-error>Cannot resolve ambiguous alias reference for identifier IDENT</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="group-by">
- <compilation-unit name="sugar-08-negative">
- <output-dir compare="Text">core-01</output-dir>
- <expected-error>ASX1103: Illegal use of identifier: x</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="group-by">
- <compilation-unit name="sugar-09">
- <output-dir compare="Text">sugar-09</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="group-by">
- <compilation-unit name="two-step-agg-01">
- <output-dir compare="Text">two-step-agg-01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="group-by">
- <compilation-unit name="null">
- <output-dir compare="Text">null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="group-by">
- <compilation-unit name="gby-expr">
- <output-dir compare="Text">gby-expr</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="group-by">
- <compilation-unit name="grouping-sets-1">
- <output-dir compare="Text">grouping-sets-1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="group-by">
- <compilation-unit name="grouping-sets-2">
- <output-dir compare="Text">grouping-sets-2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="group-by">
- <compilation-unit name="grouping-sets-3-negative">
- <output-dir compare="Text">grouping-sets-2</output-dir>
- <expected-error>ASX1120: Unexpected alias: v21</expected-error>
- <expected-error>ASX1120: Unexpected alias: v22</expected-error>
- <expected-error>ASX1120: Unexpected alias: v23</expected-error>
- <expected-error>ASX1087: Invalid number of arguments for function grouping</expected-error>
- <expected-error>ASX1119: Invalid argument to grouping() function</expected-error>
- <expected-error>ASX1119: Invalid argument to grouping() function</expected-error>
- <expected-error>ASX1119: Invalid argument to grouping() function</expected-error>
- <expected-error>ASX1118: Too many grouping sets in group by clause: 512. Maximum allowed: 128.</expected-error>
- <expected-error>ASX1129: Cannot compile SELECT variable.* with GROUP BY GROUPING SETS/ROLLUP/CUBE followed by ORDER BY/LIMIT</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="group-by">
- <compilation-unit name="having">
- <output-dir compare="Text">core-02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="group-by">
- <compilation-unit name="having-2">
- <output-dir compare="Text">core-02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="group-by">
- <compilation-unit name="policy">
- <output-dir compare="Text">policy</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="group-by">
- <compilation-unit name="policy-02">
- <output-dir compare="Text">policy-02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="group-by">
- <compilation-unit name="policy-03">
- <output-dir compare="Text">policy-03</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="group-by">
- <compilation-unit name="policy-04">
- <output-dir compare="Text">policy-04</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="group-by" check-warnings="true">
- <compilation-unit name="policy-05">
- <output-dir compare="Text">policy-05</output-dir>
- <expected-warn>Unsupported type: agg-sum cannot process input type object (in line 29, at column 23)</expected-warn>
- </compilation-unit>
- </test-case>
- <test-case FilePath="group-by">
- <compilation-unit name="listify">
- <output-dir compare="Text">none</output-dir>
- <expected-error>The byte size of a single group</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="group-by">
- <compilation-unit name="listify-2">
- <output-dir compare="Text">none</output-dir>
- <expected-error>The byte size of a single group</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="group-by">
- <compilation-unit name="listify-3">
- <output-dir compare="Text">listify-3</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="group-by">
- <compilation-unit name="redundant-var-in-groupby">
- <output-dir compare="Text">redundant-var-in-groupby</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="group-by">
- <compilation-unit name="group-by-all-ASTERIXDB-2611">
- <output-dir compare="Text">group-by-all-ASTERIXDB-2611</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="group-by">
- <compilation-unit name="hash-group-by-decor">
- <output-dir compare="Text">hash-group-by-decor</output-dir>
- </compilation-unit>
- </test-case>
- <!--test-case FilePath="group-by">
- <compilation-unit name="query-ASTERIXDB-3016">
- <output-dir compare="Text">query-ASTERIXDB-3016</output-dir>
- </compilation-unit>
- </test-case-->
- </test-group>
- <test-group name="index-join">
- <test-case FilePath="index-join">
- <compilation-unit name="btree-index-nested-loop-join">
- <output-dir compare="Text">btree-index-nested-loop-join</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-join">
- <compilation-unit name="btree-primary-equi-join">
- <output-dir compare="Text">btree-primary-equi-join</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-join">
- <compilation-unit name="btree-pidx-to-sidx-idxonly-equi-join_01">
- <output-dir compare="Text">btree-pidx-to-sidx-idxonly-equi-join_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-join">
- <compilation-unit name="btree-secondary-equi-join_01">
- <output-dir compare="Text">btree-secondary-equi-join_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-join">
- <compilation-unit name="btree-secondary-equi-join_02">
- <output-dir compare="Text">btree-secondary-equi-join_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-join">
- <compilation-unit name="btree-secondary-equi-join_03">
- <output-dir compare="Text">btree-secondary-equi-join_03</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-join">
- <compilation-unit name="btree-secondary-equi-join_04">
- <output-dir compare="Text">btree-secondary-equi-join_04</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-join">
- <compilation-unit name="btree-sidx-idxonly-to-pidx-equi-join_01">
- <output-dir compare="Text">btree-sidx-idxonly-to-pidx-equi-join_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-join">
- <compilation-unit name="btree-sidx-idxonly-to-sidx-idxonly-equi-join_01">
- <output-dir compare="Text">btree-sidx-idxonly-to-sidx-idxonly-equi-join_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-join">
- <compilation-unit name="btree-sidx-non-idxonly-to-pidx-equi-join_01">
- <output-dir compare="Text">btree-sidx-non-idxonly-to-pidx-equi-join_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-join">
- <compilation-unit name="btree-sidx-non-idxonly-to-sidx-idxonly-equi-join_01">
- <output-dir compare="Text">btree-sidx-non-idxonly-to-sidx-idxonly-equi-join_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-join">
- <compilation-unit name="btree-secondary-self-equi-join_01">
- <output-dir compare="Text">btree-secondary-self-equi-join_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-join" check-warnings="true">
- <compilation-unit name="hints-indexnl-params">
- <output-dir compare="Text">hints-indexnl-params</output-dir>
- <expected-warn><![CDATA[ASX1132: Invalid specification for hint indexnl. ASX1001: Syntax error: In line 1 >>(8, idx_tenk2_1k_2k)<< Encountered <INTEGER_LITERAL> "8" at column 2. (in line 35, at column 21)]]></expected-warn>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-join">
- <compilation-unit name="rtree-spatial-intersect-point_01">
- <output-dir compare="Text">rtree-spatial-intersect-point_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-join">
- <compilation-unit name="rtree-spatial-intersect-point_02">
- <output-dir compare="Text">rtree-spatial-intersect-point_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-join">
- <compilation-unit name="rtree-spatial-intersect-point_03">
- <output-dir compare="Text">rtree-spatial-intersect-point_03</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-join">
- <compilation-unit name="rtree-spatial-intersect-point_04">
- <output-dir compare="Text">rtree-spatial-intersect-point_04</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-join">
- <compilation-unit name="rtree-spatial-intersect-point_05">
- <output-dir compare="Text">rtree-spatial-intersect-point_05</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-join">
- <compilation-unit name="rtree-spatial-self-intersect-point">
- <output-dir compare="Text">rtree-spatial-self-intersect-point</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-join">
- <compilation-unit name="btree-multiple-join">
- <output-dir compare="Text">btree-multiple-join</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="index-selection">
- <test-case FilePath="index-selection">
- <compilation-unit name="btree-index-composite-key">
- <output-dir compare="Text">btree-index-composite-key</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="btree-index-composite-key-02">
- <output-dir compare="Text">btree-index-composite-key-02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="btree-index-composite-key-03">
- <output-dir compare="Text">btree-index-composite-key-03</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="btree-index-composite-key-04">
- <output-dir compare="Text">btree-index-composite-key-04</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="btree-sec-primary-index">
- <output-dir compare="Text">btree-sec-primary-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="btree-sec-primary-index-01">
- <output-dir compare="Text">btree-sec-primary-index-01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="btree-sec-primary-index-02">
- <output-dir compare="Text">btree-sec-primary-index-02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="btree-sec-primary-index-03">
- <output-dir compare="Text">btree-sec-primary-index-03</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="btree-sec-primary-index-04">
- <output-dir compare="Text">btree-sec-primary-index-04</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="btree-index-composite-key-mixed-intervals">
- <output-dir compare="Text">btree-index-composite-key-mixed-intervals</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="btree-index-rewrite-multiple">
- <output-dir compare="Text">btree-index-rewrite-multiple</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="btree-sidx-composite-idxonly-01">
- <output-dir compare="Text">btree-sidx-composite-idxonly-01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="btree-sidx-composite-idxonly-02">
- <output-dir compare="Text">btree-sidx-composite-idxonly-02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="btree-sidx-composite-idxonly-03">
- <output-dir compare="Text">btree-sidx-composite-idxonly-03</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="btree-sidx-composite-idxonly-04">
- <output-dir compare="Text">btree-sidx-composite-idxonly-04</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="btree-sidx-idxonly-01">
- <output-dir compare="Text">btree-sidx-idxonly-01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="btree-sidx-non-idxonly-01">
- <output-dir compare="Text">btree-sidx-non-idxonly-01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="cust-index-age-nullable">
- <output-dir compare="Text">cust-index-age-nullable</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="intersection-misc-01">
- <output-dir compare="Text">intersection-misc-01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="intersection-with-between">
- <output-dir compare="Text">intersection-with-between</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="inverted-index-ngram-contains">
- <output-dir compare="Text">inverted-index-ngram-contains</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="inverted-index-ngram-edit-distance-panic">
- <output-dir compare="Text">inverted-index-ngram-edit-distance-panic</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="inverted-index-ngram-edit-distance">
- <output-dir compare="Text">inverted-index-ngram-edit-distance</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="inverted-index-ngram-edit-distance-large-data">
- <output-dir compare="Text">inverted-index-ngram-edit-distance-large-data</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="inverted-index-ngram-edit-distance-word-tokens">
- <output-dir compare="Text">inverted-index-ngram-edit-distance-word-tokens</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="inverted-index-ngram-jaccard">
- <output-dir compare="Text">inverted-index-ngram-jaccard</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="inverted-index-ngram-edit-distance-contains">
- <output-dir compare="Text">inverted-index-ngram-edit-distance-contains</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="inverted-index-olist-edit-distance-panic">
- <output-dir compare="Text">inverted-index-olist-edit-distance-panic</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="inverted-index-olist-edit-distance">
- <output-dir compare="Text">inverted-index-olist-edit-distance</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="inverted-index-olist-jaccard">
- <output-dir compare="Text">inverted-index-olist-jaccard</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="inverted-index-ulist-jaccard">
- <output-dir compare="Text">inverted-index-ulist-jaccard</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="inverted-index-word-contains">
- <output-dir compare="Text">inverted-index-word-contains</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="inverted-index-word-jaccard">
- <output-dir compare="Text">inverted-index-word-jaccard</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="orders-index-custkey-conjunctive-open">
- <output-dir compare="Text">orders-index-custkey-conjunctive-open</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="orders-index-custkey-conjunctive">
- <output-dir compare="Text">orders-index-custkey-conjunctive</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="orders-index-custkey-open">
- <output-dir compare="Text">orders-index-custkey-open</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="orders-index-custkey">
- <output-dir compare="Text">orders-index-custkey</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="range-search-open">
- <output-dir compare="Text">range-search-open</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="range-search">
- <output-dir compare="Text">range-search</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="rtree-secondary-index-nullable">
- <output-dir compare="Text">rtree-secondary-index-nullable</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="rtree-secondary-index-open">
- <output-dir compare="Text">rtree-secondary-index-open</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="rtree-secondary-index">
- <output-dir compare="Text">rtree-secondary-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="rtree-secondary-index-circular-query">
- <output-dir compare="Text">rtree-secondary-index-circular-query</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="rtree-sidx-idxonly-01">
- <output-dir compare="Text">rtree-sidx-idxonly-01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="rtree-sidx-non-idxonly-01">
- <output-dir compare="Text">rtree-sidx-non-idxonly-01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="rtree-sidx-non-idxonly-02">
- <output-dir compare="Text">rtree-sidx-non-idxonly-02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="disjunctive-predicate-1">
- <output-dir compare="Text">disjunctive-predicate-1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="dataset-with-meta">
- <output-dir compare="Text">dataset-with-meta</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection" check-warnings="true">
- <compilation-unit name="hints-skip-index">
- <output-dir compare="Text">hints-skip-index</output-dir>
- <expected-warn><![CDATA[ASX1132: Invalid specification for hint skip-index. ASX1001: Syntax error: In line 1 >>(13, idx_1k)<< Encountered <INTEGER_LITERAL> "13" at column 2. (in line 32, at column 19)]]></expected-warn>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection" check-warnings="true">
- <compilation-unit name="hints-use-index">
- <output-dir compare="Text">hints-use-index</output-dir>
- <expected-warn><![CDATA[ASX1132: Invalid specification for hint use-index. ASX1001: Syntax error: In line 1 >>(18, idx_1k_2k)<< Encountered <INTEGER_LITERAL> "18" at column 2. (in line 33, at column 15)]]></expected-warn>
- <expected-warn><![CDATA[ASX1132: Invalid specification for hint use-index. ASX1001: Syntax error: In line 1 >>()<< Encountered ")" at column 2. (in line 33, at column 15)]]></expected-warn>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="inverted-index-ngram-edit-distance-with-two-ngram-index">
- <output-dir compare="Text">inverted-index-ngram-edit-distance-with-two-ngram-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="intersection">
- <output-dir compare="Text">intersection</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="intersection-with-filter">
- <output-dir compare="Text">intersection-with-filter</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="intersection_with_nodegroup">
- <output-dir compare="Text">intersection</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="verify">
- <output-dir compare="Text">verify</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="inverted-index-join">
- <test-case FilePath="inverted-index-join">
- <compilation-unit name="ngram-edit-distance">
- <output-dir compare="Text">ngram-edit-distance</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="inverted-index-join">
- <compilation-unit name="ngram-edit-distance-inline">
- <output-dir compare="Text">ngram-edit-distance-inline</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="inverted-index-join">
- <compilation-unit name="ngram-jaccard">
- <output-dir compare="Text">ngram-jaccard</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="inverted-index-join">
- <compilation-unit name="ngram-jaccard-inline">
- <output-dir compare="Text">ngram-jaccard-inline</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="inverted-index-join">
- <compilation-unit name="olist-edit-distance">
- <output-dir compare="Text">olist-edit-distance</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="inverted-index-join">
- <compilation-unit name="olist-edit-distance-inline">
- <output-dir compare="Text">olist-edit-distance-inline</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="inverted-index-join">
- <compilation-unit name="olist-jaccard">
- <output-dir compare="Text">olist-jaccard</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="inverted-index-join">
- <compilation-unit name="olist-jaccard-inline">
- <output-dir compare="Text">olist-jaccard-inline</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="inverted-index-join">
- <compilation-unit name="ulist-jaccard">
- <output-dir compare="Text">ulist-jaccard</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="inverted-index-join">
- <compilation-unit name="ulist-jaccard-inline">
- <output-dir compare="Text">ulist-jaccard-inline</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="inverted-index-join">
- <compilation-unit name="word-jaccard">
- <output-dir compare="Text">word-jaccard</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="inverted-index-join">
- <compilation-unit name="word-jaccard-inline">
- <output-dir compare="Text">word-jaccard-inline</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="cast-default-null">
- <placeholder name="with" value="" />
- <output-dir compare="Text">cast-default-null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-selection">
- <compilation-unit name="cast-default-null">
- <placeholder name="with" value="WITH {'merge-policy': {'name': 'correlated-prefix','parameters': { 'max-mergable-component-size': 16384, 'max-tolerance-component-count': 3 }}}" />
- <output-dir compare="Text">cast-default-null</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="inverted-index-join-noeqjoin">
- <test-case FilePath="inverted-index-join-noeqjoin">
- <compilation-unit name="ngram-edit-distance">
- <output-dir compare="Text">ngram-edit-distance</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="inverted-index-join-noeqjoin">
- <compilation-unit name="ngram-edit-distance-inline">
- <output-dir compare="Text">ngram-edit-distance-inline</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="inverted-index-join-noeqjoin">
- <compilation-unit name="ngram-jaccard">
- <output-dir compare="Text">ngram-jaccard</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="inverted-index-join-noeqjoin">
- <compilation-unit name="ngram-jaccard-inline">
- <output-dir compare="Text">ngram-jaccard-inline</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="inverted-index-join-noeqjoin">
- <compilation-unit name="olist-edit-distance">
- <output-dir compare="Text">olist-edit-distance</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="inverted-index-join-noeqjoin">
- <compilation-unit name="olist-edit-distance-inline">
- <output-dir compare="Text">olist-edit-distance-inline</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="inverted-index-join-noeqjoin">
- <compilation-unit name="olist-jaccard">
- <output-dir compare="Text">olist-jaccard</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="inverted-index-join-noeqjoin">
- <compilation-unit name="olist-jaccard-inline">
- <output-dir compare="Text">olist-jaccard-inline</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="inverted-index-join-noeqjoin">
- <compilation-unit name="ulist-jaccard">
- <output-dir compare="Text">ulist-jaccard</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="inverted-index-join-noeqjoin">
- <compilation-unit name="ulist-jaccard-inline">
- <output-dir compare="Text">ulist-jaccard-inline</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="inverted-index-join-noeqjoin">
- <compilation-unit name="word-jaccard">
- <output-dir compare="Text">word-jaccard</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="inverted-index-join-noeqjoin">
- <compilation-unit name="word-jaccard-inline">
- <output-dir compare="Text">word-jaccard-inline</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="join">
- <test-case FilePath="join">
- <compilation-unit name="cross-join-01">
- <output-dir compare="Text">cross-join-01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="join">
- <compilation-unit name="cross-join-02-negative">
- <output-dir compare="Text">none</output-dir>
- <expected-error>ASX1077: Cannot find dataset x in dataverse Default nor an alias with name x (in line 26, at column 39)</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="join">
- <compilation-unit name="hash_join_array">
- <output-dir compare="Text">hash_join_array</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="join">
- <compilation-unit name="hash_join_missing">
- <output-dir compare="Text">hash_join_missing</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="join">
- <compilation-unit name="hash_join_record">
- <output-dir compare="Text">hash_join_record</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="join">
- <compilation-unit name="join-with-empty-dataset">
- <output-dir compare="Text">join-with-empty-dataset</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="list">
- <test-case FilePath="list">
- <compilation-unit name="any-collection-member_01">
- <output-dir compare="Text">any-collection-member_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="list">
- <compilation-unit name="array_length">
- <output-dir compare="Text">array_length</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="list">
- <compilation-unit name="enforcing_item_type">
- <output-dir compare="Text">enforcing_item_type</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="list">
- <compilation-unit name="exists">
- <output-dir compare="Text">exists</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="list">
- <compilation-unit name="get-item_01">
- <output-dir compare="Text">get-item_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="list">
- <compilation-unit name="get-item_02">
- <output-dir compare="Text">get-item_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="list">
- <compilation-unit name="get-item_03">
- <output-dir compare="Text">get-item_03</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="list">
- <compilation-unit name="len_01">
- <output-dir compare="Text">len_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="list">
- <compilation-unit name="len_null_01">
- <output-dir compare="Text">len_null_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="list">
- <compilation-unit name="list-range">
- <output-dir compare="Text">list-range</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="list">
- <compilation-unit name="list-slice_01">
- <output-dir compare="Text">list-slice_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="list">
- <compilation-unit name="list-slice_02">
- <output-dir compare="Text">list-slice_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="list">
- <compilation-unit name="list-slice_03">
- <output-dir compare="Text">list-slice_03</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="list">
- <compilation-unit name="list-star_01">
- <output-dir compare="Text">list-star_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="list">
- <compilation-unit name="listify_01">
- <output-dir compare="Text">listify_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="list">
- <compilation-unit name="listify_02">
- <output-dir compare="Text">listify_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="list">
- <compilation-unit name="listify_03">
- <output-dir compare="Text">listify_03</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="list">
- <compilation-unit name="ordered-list-constructor_01">
- <output-dir compare="Text">ordered-list-constructor_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="list">
- <compilation-unit name="ordered-list-constructor_02">
- <output-dir compare="Text">ordered-list-constructor_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="list">
- <compilation-unit name="ordered-list-constructor_03">
- <output-dir compare="Text">ordered-list-constructor_03</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="list">
- <compilation-unit name="ordered-list-constructor_04">
- <output-dir compare="Text">ordered-list-constructor_04</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="list">
- <compilation-unit name="ordered-list-constructor_05">
- <output-dir compare="Text">ordered-list-constructor_05</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="list">
- <compilation-unit name="ordered-list-constructor_06_negative">
- <output-dir compare="Text">none</output-dir>
- <expected-error><![CDATA[ASX1001: Syntax error: In line 26 >> from [r, r+1] x select value count(x),<< Encountered "," at column 42]]></expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="list">
- <compilation-unit name="scan-collection_01">
- <output-dir compare="Text">scan-collection_01</output-dir>
- </compilation-unit>
- </test-case>
- <!-- <test-case FilePath="list">
- <compilation-unit name="union_01">
- <output-dir compare="Text">union_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="list">
- <compilation-unit name="union_02">
- <output-dir compare="Text">union_02</output-dir>
- </compilation-unit>
- </test-case>
- -->
- <test-case FilePath="list">
- <compilation-unit name="unordered-list-constructor_01">
- <output-dir compare="Text">unordered-list-constructor_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="list">
- <compilation-unit name="unordered-list-constructor_02">
- <output-dir compare="Text">unordered-list-constructor_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="list">
- <compilation-unit name="unordered-list-constructor_03">
- <output-dir compare="Text">unordered-list-constructor_03</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="list">
- <compilation-unit name="query-issue428">
- <output-dir compare="Text">query-issue428</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="list">
- <compilation-unit name="query-ASTERIXDB-159-2">
- <output-dir compare="Text">query-ASTERIXDB-159-2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="list">
- <compilation-unit name="query-ASTERIXDB-159-3">
- <output-dir compare="Text">query-ASTERIXDB-159-3</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="list">
- <compilation-unit name="query-ASTERIXDB-1131-2">
- <output-dir compare="Text">query-ASTERIXDB-1131-2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="list">
- <compilation-unit name="query-ASTERIXDB-1131">
- <output-dir compare="Text">query-ASTERIXDB-1131</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="list">
- <compilation-unit name="query-ASTERIXDB-1212-2-open">
- <output-dir compare="Text">query-ASTERIXDB-1212-2-open</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="list">
- <compilation-unit name="query-ASTERIXDB-1212-2">
- <output-dir compare="Text">query-ASTERIXDB-1212-2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="list">
- <compilation-unit name="query-ASTERIXDB-1212-open">
- <output-dir compare="Text">query-ASTERIXDB-1212-open</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="list">
- <compilation-unit name="query-ASTERIXDB-1212">
- <output-dir compare="Text">query-ASTERIXDB-1212</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="list">
- <compilation-unit name="query-ASTERIXDB-673">
- <output-dir compare="Text">query-ASTERIXDB-673</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="list">
- <compilation-unit name="var-in-list">
- <output-dir compare="Text">var-in-list</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="misc">
- <test-case FilePath="misc">
- <compilation-unit name="big_in_list/000">
- <output-dir compare="Text">big_in_list/000</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="big_in_list/001">
- <output-dir compare="Text">big_in_list/001</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="multiple_and/000">
- <output-dir compare="Text">multiple_and/000</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="record-serialization-ASTERIXDB-2567">
- <output-dir compare="Text">record-serialization-ASTERIXDB-2567</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="record-serialization-ASTERIXDB-2613">
- <output-dir compare="Text">record-serialization-ASTERIXDB-2613</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="field_access-ASTERIXDB-2289">
- <output-dir compare="Text">field_access-ASTERIXDB-2289</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="comp-ASTERIXDB-2415">
- <output-dir compare="Text">query-ASTERIXDB-1671</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="comp-ASTERIXDB-2412">
- <output-dir compare="Text">comp-ASTERIXDB-2412</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="field_access_union-ASTERIXDB-2288">
- <output-dir compare="Text">field_access_union-ASTERIXDB-2288</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="constant_folding">
- <output-dir compare="Text">constant_folding</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="join-ASTERIXDB-2686">
- <output-dir compare="Text">join-ASTERIXDB-2686</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="poll-dynamic">
- <output-dir compare="Text">poll-dynamic</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="validate-expected">
- <output-dir compare="Text">validate-expected</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="dataset-resources">
- <output-dir compare="Text">dataset-resources</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="ping">
- <output-dir compare="Text">ping</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="case_01">
- <output-dir compare="Text">case_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="case_02">
- <output-dir compare="Text">case_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="case_03">
- <output-dir compare="Text">case_03</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="case_04">
- <output-dir compare="Text">case_04</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="case_05">
- <output-dir compare="Text">case_05</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="case_06">
- <output-dir compare="Text">case_06</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="case_07">
- <output-dir compare="Text">case_05</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="case_08">
- <output-dir compare="Text">case_08</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="case_09">
- <output-dir compare="Text">case_09</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="dataset_nodegroup">
- <output-dir compare="Text">dataset_nodegroup</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="ensure_result_numeric_type">
- <output-dir compare="Text">ensure_result_numeric_type</output-dir>
- <expected-error>expected < 3.0</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="partition-by-nonexistent-field">
- <output-dir compare="Text">partition-by-nonexistent-field</output-dir>
- <expected-error>Field 'id' is not found</expected-error>
- <expected-error>Cannot find dataset with name testds in dataverse test</expected-error>
- <expected-error>Cannot find dataset testds in dataverse test nor an alias with name testds</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="float_01">
- <output-dir compare="Text">float_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="flushtest">
- <output-dir compare="Text">flushtest</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="groupby-orderby-count">
- <output-dir compare="Text">groupby-orderby-count</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="identifier_01">
- <output-dir compare="Text">identifier_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="ifthenelse_01">
- <output-dir compare="Text">ifthenelse_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="is-null_01">
- <output-dir compare="Text">is-null_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="nested-loop-join_01">
- <output-dir compare="Text">nested-loop-join_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="query_issue267">
- <output-dir compare="Text">query_issue267</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="random">
- <output-dir compare="Text">random</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="stable_sort">
- <output-dir compare="Text">stable_sort</output-dir>
- </compilation-unit>
- </test-case>
- <!--
- <test-case FilePath="misc">
- <compilation-unit name="range_01">
- <output-dir compare="Text">range_01</output-dir>
- </compilation-unit>
- </test-case>
- -->
- <!--
- <test-case FilePath="misc">
- <compilation-unit name="tid_01">
- <output-dir compare="Text">tid_01</output-dir>
- </compilation-unit>
- </test-case>
- -->
- <test-case FilePath="misc">
- <compilation-unit name="year_01">
- <output-dir compare="Text">year_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="string_eq_01">
- <output-dir compare="Text">string_eq_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="prefix-search">
- <output-dir compare="Text">prefix-search</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="query-ASTERIXDB-1490">
- <output-dir compare="Text">query-ASTERIXDB-1490</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="query-ASTERIXDB-971">
- <output-dir compare="Text">query-ASTERIXDB-971-sqlpp</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="query-ASTERIXDB-1531">
- <output-dir compare="Text">query-ASTERIXDB-1531</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="query-ASTERIXDB-1577">
- <output-dir compare="Text">query-ASTERIXDB-1577</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="query-ASTERIXDB-1671">
- <output-dir compare="Text">query-ASTERIXDB-1671</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="query-ASTERIXDB-1671-2">
- <output-dir compare="Text">query-ASTERIXDB-1671</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="query-ASTERIXDB-2354">
- <output-dir compare="Text">query-ASTERIXDB-2354</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="query-ASTERIXDB-2355">
- <output-dir compare="Text">none</output-dir>
- <expected-error><![CDATA[ASX1001: Syntax error: In line 22 >> %%%<< Encountered "%" at column 2.]]></expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="query-ASTERIXDB-2380">
- <output-dir compare="Text">query-ASTERIXDB-2380</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="query-ASTERIXDB-2550">
- <output-dir compare="Text">none</output-dir>
- <expected-error>ASX0037: Type mismatch: expected value of type array or multiset, but got the value of type object (in line 28, at column 2)</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="query-ASTERIXDB-2886">
- <output-dir compare="Text">query-ASTERIXDB-2886</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="unsupported_parameter">
- <output-dir compare="Text">none</output-dir>
- <expected-error>Query parameter compiler.joinmem is not supported</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="unsupported_parameter_value">
- <output-dir compare="Text">none</output-dir>
- <expected-error>The given string: 4LS is not a byte unit string (e.g., 320KB or 1024)</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="uuid">
- <output-dir compare="Text">uuid</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="p_sort_join">
- <output-dir compare="Text">p_sort_join</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="p_sort_seq_merge">
- <output-dir compare="Text">p_sort_seq_merge</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="p_sort_num_samples">
- <output-dir compare="Text">p_sort_num_samples</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="p_sort_static_range_map">
- <output-dir compare="Text">p_sort_static_range_map</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="active_requests">
- <output-dir compare="Text">active_requests</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="jobs">
- <output-dir compare="Text">jobs</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="completed_requests">
- <output-dir compare="Text">completed_requests</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="dump_index">
- <output-dir compare="Text">dump_index</output-dir>
- <expected-error>Cannot find index with name noindex</expected-error>
- <expected-error>Cannot find dataset with name nodataset in dataverse test</expected-error>
- <expected-error>Cannot find dataset with name ds in dataverse nodataverse</expected-error>
- <expected-error>Unsupported type: dump-index cannot process input type null</expected-error>
- <expected-error>Unsupported type: dump-index cannot process input type null</expected-error>
- <expected-error>Unsupported type: dump-index cannot process input type null</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="metadata_only_01">
- <output-dir compare="Text">metadata_only_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="metadata_only_02">
- <output-dir compare="Text">metadata_only_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="cast-ASTERIXDB-2458">
- <output-dir compare="Text">cast-ASTERIXDB-2458</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="insert_nulls_with_secondary_idx">
- <output-dir compare="Text">insert_nulls_with_secondary_idx</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="query-ASTERIXDB-2700">
- <output-dir compare="Text">query-ASTERIXDB-2700</output-dir>
- </compilation-unit>
- </test-case>
- <!--
- <test-case FilePath="misc">
- <compilation-unit name="query-ASTERIXDB-1203">
- <output-dir compare="Text">query-ASTERIXDB-1203</output-dir>
- </compilation-unit>
- </test-case>
- -->
- <test-case FilePath="misc">
- <compilation-unit name="query-ASTERIXDB-819-2">
- <output-dir compare="Text">query-ASTERIXDB-819-2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="query-ASTERIXDB-819">
- <output-dir compare="Text">query-ASTERIXDB-819</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="misc">
- <compilation-unit name="query-ASTERIXDB-865">
- <output-dir compare="Text">query-ASTERIXDB-865</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="multipart-dataverse">
- <test-case FilePath="multipart-dataverse">
- <compilation-unit name="index_1">
- <output-dir compare="Text">index_1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="multipart-dataverse">
- <compilation-unit name="resolution_1">
- <output-dir compare="Text">resolution_1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="multipart-dataverse">
- <compilation-unit name="special_chars_1">
- <output-dir compare="Text">special_chars_1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="multipart-dataverse">
- <compilation-unit name="special_chars_2">
- <output-dir compare="Text">special_chars_2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="multipart-dataverse">
- <compilation-unit name="udf_1">
- <output-dir compare="Text">udf_1</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="index">
- <test-group name="index/validations">
- <test-case FilePath="index/validations">
- <compilation-unit name="keys-same-as-pk-but-different-order">
- <output-dir compare="Text">keys-same-as-pk-but-different-order</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index/validations">
- <compilation-unit name="keys-same-as-pk-in-same-order">
- <output-dir compare="Text">keys-same-as-pk-in-same-order</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index/validations">
- <compilation-unit name="repetitive-keys">
- <output-dir compare="Text">repetitive-keys</output-dir>
- <expected-error>Cannot create index with the same field '[value]' specified more than once.</expected-error>
- </compilation-unit>
- </test-case>
- </test-group>
- </test-group>
- <test-group name="open-index-enforced">
- <test-group name="open-index-enforced/error-checking">
- <test-case FilePath="open-index-enforced/error-checking">
- <compilation-unit name="enforced-field-name-collision">
- <output-dir compare="Text">enforced-field-name-collision</output-dir>
- <expected-error>Cannot create enforced index on '[value]' field. The field is closed type.</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-index-enforced/error-checking">
- <compilation-unit name="enforced-field-type-collision">
- <output-dir compare="Text">enforced-field-type-collision</output-dir>
- <expected-error>Cannot create enforced index on '[value]' field. The field is closed type.</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-index-enforced/error-checking">
- <compilation-unit name="missing-enforce-statement">
- <output-dir compare="Text">missing-enforce-statement</output-dir>
- <expected-error>ASX1042: Cannot create non-enforced typed index of this kind: RTREE</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-index-enforced/error-checking">
- <compilation-unit name="missing-optionality">
- <output-dir compare="Text">missing-optionality</output-dir>
- <expected-error>Cannot create enforced index on '[value]' field with non-optional type</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-index-enforced/error-checking">
- <compilation-unit name="index-on-closed-type">
- <output-dir compare="Text">index-on-closed-type</output-dir>
- <expected-error>ASX1014: Field 'value' is not found (in line 33, at column 34)</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-index-enforced/error-checking">
- <compilation-unit name="index-type-collision">
- <output-dir compare="Text">index-type-collision</output-dir>
- <expected-error>Cannot create index testIdx2 , enforced index testIdx1 on field(s) 'value' is already defined with type(s) 'integer'</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-index-enforced/error-checking">
- <compilation-unit name="index-type-promotion-collision">
- <output-dir compare="Text">index-type-promotion-collision</output-dir>
- <expected-error>Cannot create index testIdx2 , enforced index testIdx1 on field(s) 'value' is already defined with type(s) 'bigint'</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-index-enforced/error-checking">
- <compilation-unit name="object-type-collision">
- <output-dir compare="Text">object-type-collision</output-dir>
- <expected-error>ASX1051: Cannot create enforced index on '[value]' field. The field is closed type.</expected-error>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="open-index-enforced/index-join">
- <test-case FilePath="open-index-enforced/index-join">
- <compilation-unit name="btree-secondary-equi-join">
- <output-dir compare="Text">btree-secondary-equi-join</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-index-enforced/index-join">
- <compilation-unit name="ngram-edit-distance">
- <output-dir compare="Text">ngram-edit-distance</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-index-enforced/index-join">
- <compilation-unit name="ngram-edit-distance-inline">
- <output-dir compare="Text">ngram-edit-distance-inline</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-index-enforced/index-join">
- <compilation-unit name="ngram-jaccard">
- <output-dir compare="Text">ngram-jaccard</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-index-enforced/index-join">
- <compilation-unit name="ngram-jaccard-inline">
- <output-dir compare="Text">ngram-jaccard-inline</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-index-enforced/index-join">
- <compilation-unit name="rtree-spatial-intersect-point">
- <output-dir compare="Text">rtree-spatial-intersect-point</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-index-enforced/index-join">
- <compilation-unit name="word-jaccard">
- <output-dir compare="Text">word-jaccard</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-index-enforced/index-join">
- <compilation-unit name="word-jaccard-inline">
- <output-dir compare="Text">word-jaccard-inline</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="open-index-enforced/index-leftouterjoin">
- <test-case FilePath="open-index-enforced/index-leftouterjoin">
- <compilation-unit name="probe-pidx-with-join-btree-sidx1">
- <output-dir compare="Text">probe-pidx-with-join-btree-sidx1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-index-enforced/index-leftouterjoin">
- <compilation-unit name="probe-pidx-with-join-btree-sidx2">
- <output-dir compare="Text">probe-pidx-with-join-btree-sidx2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-index-enforced/index-leftouterjoin">
- <compilation-unit name="probe-pidx-with-join-invidx-sidx2">
- <output-dir compare="Text">probe-pidx-with-join-invidx-sidx2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-index-enforced/index-leftouterjoin">
- <compilation-unit name="probe-pidx-with-join-rtree-sidx1">
- <output-dir compare="Text">probe-pidx-with-join-rtree-sidx1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-index-enforced/index-leftouterjoin">
- <compilation-unit name="probe-pidx-with-join-rtree-sidx2">
- <output-dir compare="Text">probe-pidx-with-join-rtree-sidx2</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="open-index-enforced/index-selection">
- <test-case FilePath="open-index-enforced/index-selection">
- <compilation-unit name="btree-index-composite-key">
- <output-dir compare="Text">btree-index-composite-key</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-index-enforced/index-selection">
- <compilation-unit name="btree-index-composite-key-mixed-intervals">
- <output-dir compare="Text">btree-index-composite-key-mixed-intervals</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-index-enforced/index-selection">
- <compilation-unit name="btree-index-rewrite-multiple">
- <output-dir compare="Text">btree-index-rewrite-multiple</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-index-enforced/index-selection">
- <compilation-unit name="inverted-index-ngram-contains">
- <output-dir compare="Text">inverted-index-ngram-contains</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-index-enforced/index-selection">
- <compilation-unit name="inverted-index-ngram-edit-distance">
- <output-dir compare="Text">inverted-index-ngram-edit-distance</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-index-enforced/index-selection">
- <compilation-unit name="inverted-index-ngram-edit-distance-contains">
- <output-dir compare="Text">inverted-index-ngram-edit-distance-contains</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-index-enforced/index-selection">
- <compilation-unit name="inverted-index-ngram-edit-distance-panic">
- <output-dir compare="Text">inverted-index-ngram-edit-distance-panic</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-index-enforced/index-selection">
- <compilation-unit name="inverted-index-ngram-edit-distance-word-tokens">
- <output-dir compare="Text">inverted-index-ngram-edit-distance-word-tokens</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-index-enforced/index-selection">
- <compilation-unit name="inverted-index-ngram-jaccard">
- <output-dir compare="Text">inverted-index-ngram-jaccard</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-index-enforced/index-selection">
- <compilation-unit name="inverted-index-word-contains">
- <output-dir compare="Text">inverted-index-word-contains</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-index-enforced/index-selection">
- <compilation-unit name="inverted-index-word-jaccard">
- <output-dir compare="Text">inverted-index-word-jaccard</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-index-enforced/index-selection">
- <compilation-unit name="orders-index-custkey">
- <output-dir compare="Text">orders-index-custkey</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-index-enforced/index-selection">
- <compilation-unit name="orders-index-custkey-conjunctive">
- <output-dir compare="Text">orders-index-custkey-conjunctive</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-index-enforced/index-selection">
- <compilation-unit name="range-search">
- <output-dir compare="Text">range-search</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-index-enforced/index-selection">
- <compilation-unit name="rtree-secondary-index">
- <output-dir compare="Text">rtree-secondary-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-index-enforced/index-selection">
- <compilation-unit name="multi-index-composite-key">
- <output-dir compare="Text">multi-index-composite-key</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-index-enforced/index-selection">
- <compilation-unit name="multi-index">
- <output-dir compare="Text">multi-index</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="open-index-enforced/type-checking">
- <test-case FilePath="open-index-enforced/type-checking">
- <compilation-unit name="enforced-type-delete">
- <output-dir compare="Text">enforced-type-delete</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-index-enforced/type-checking">
- <compilation-unit name="enforced-type-upsert">
- <output-dir compare="Text">enforced-type-upsert</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="open-index-non-enforced/index-selection">
- <test-case FilePath="open-index-non-enforced/index-selection">
- <compilation-unit name="btree-index-01">
- <output-dir compare="Text">btree-index-01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-index-non-enforced/index-selection">
- <compilation-unit name="btree-index-02">
- <output-dir compare="Text">btree-index-02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-index-non-enforced/index-selection">
- <compilation-unit name="btree-index-03">
- <output-dir compare="Text">btree-index-03</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-index-non-enforced/index-selection">
- <compilation-unit name="btree-index-04">
- <output-dir compare="Text">btree-index-04</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-index-non-enforced/index-selection">
- <compilation-unit name="btree-index-composite-key-03">
- <output-dir compare="Text">btree-index-composite-key-03</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-index-non-enforced/index-selection">
- <compilation-unit name="btree-index-composite-key-04">
- <output-dir compare="Text">btree-index-composite-key-04</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-index-non-enforced/correlated-index-selection">
- <compilation-unit name="btree-index-01">
- <output-dir compare="Text">btree-index-01</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="open-index-non-enforced/index-join">
- <test-case FilePath="open-index-non-enforced/index-join">
- <compilation-unit name="btree-equi-join-01">
- <output-dir compare="Text">btree-equi-join-01</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- </test-group>
- <test-group name="nested-open-index">
- <test-group name="nested-open-index/index-join">
- <test-case FilePath="nested-open-index/index-join">
- <compilation-unit name="btree-secondary-equi-join">
- <output-dir compare="Text">btree-secondary-equi-join</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-open-index/index-join">
- <compilation-unit name="ngram-edit-distance">
- <output-dir compare="Text">ngram-edit-distance</output-dir>
- </compilation-unit>
- </test-case>
- <!-- <test-case FilePath="nested-open-index/index-join">
- <compilation-unit name="ngram-edit-distance-inline">
- <output-dir compare="Text">ngram-edit-distance-inline</output-dir>
- </compilation-unit>
- </test-case> -->
- <test-case FilePath="nested-open-index/index-join">
- <compilation-unit name="ngram-jaccard">
- <output-dir compare="Text">ngram-jaccard</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-open-index/index-join">
- <compilation-unit name="ngram-jaccard-inline">
- <output-dir compare="Text">ngram-jaccard-inline</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-open-index/index-join">
- <compilation-unit name="rtree-spatial-intersect-point">
- <output-dir compare="Text">rtree-spatial-intersect-point</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-open-index/index-join">
- <compilation-unit name="word-jaccard">
- <output-dir compare="Text">word-jaccard</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-open-index/index-join">
- <compilation-unit name="word-jaccard-inline">
- <output-dir compare="Text">word-jaccard-inline</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-open-index/index-join">
- <compilation-unit name="btree-secondary-non-enforced-equi-join">
- <output-dir compare="Text">btree-secondary-non-enforced-equi-join</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-open-index/index-join">
- <compilation-unit name="btree-secondary-non-enforced-equi-join-2">
- <output-dir compare="Text">btree-secondary-non-enforced-equi-join-2</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="nested-open-index/index-leftouterjoin">
- <test-case FilePath="nested-open-index/index-leftouterjoin">
- <compilation-unit name="probe-pidx-with-join-btree-sidx1">
- <output-dir compare="Text">probe-pidx-with-join-btree-sidx1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-open-index/index-leftouterjoin">
- <compilation-unit name="probe-pidx-with-join-btree-sidx2">
- <output-dir compare="Text">probe-pidx-with-join-btree-sidx2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-open-index/index-leftouterjoin">
- <compilation-unit name="probe-pidx-with-join-invidx-sidx2">
- <output-dir compare="Text">probe-pidx-with-join-invidx-sidx2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-open-index/index-leftouterjoin">
- <compilation-unit name="probe-pidx-with-join-rtree-sidx1">
- <output-dir compare="Text">probe-pidx-with-join-rtree-sidx1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-open-index/index-leftouterjoin">
- <compilation-unit name="probe-pidx-with-join-rtree-sidx2">
- <output-dir compare="Text">probe-pidx-with-join-rtree-sidx2</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="nested-open-index/index-selection">
- <test-case FilePath="nested-open-index/index-selection">
- <compilation-unit name="btree-index-composite-key">
- <output-dir compare="Text">btree-index-composite-key</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-open-index/index-selection">
- <compilation-unit name="btree-index-composite-key-mixed-intervals">
- <output-dir compare="Text">btree-index-composite-key-mixed-intervals</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-open-index/index-selection">
- <compilation-unit name="btree-index-rewrite-multiple">
- <output-dir compare="Text">btree-index-rewrite-multiple</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-open-index/index-selection">
- <compilation-unit name="inverted-index-ngram-contains">
- <output-dir compare="Text">inverted-index-ngram-contains</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-open-index/index-selection">
- <compilation-unit name="inverted-index-ngram-edit-distance">
- <output-dir compare="Text">inverted-index-ngram-edit-distance</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-open-index/index-selection">
- <compilation-unit name="inverted-index-ngram-edit-distance-contains">
- <output-dir compare="Text">inverted-index-ngram-edit-distance-contains</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-open-index/index-selection">
- <compilation-unit name="inverted-index-ngram-edit-distance-panic">
- <output-dir compare="Text">inverted-index-ngram-edit-distance-panic</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-open-index/index-selection">
- <compilation-unit name="inverted-index-ngram-edit-distance-word-tokens">
- <output-dir compare="Text">inverted-index-ngram-edit-distance-word-tokens</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-open-index/index-selection">
- <compilation-unit name="inverted-index-ngram-jaccard">
- <output-dir compare="Text">inverted-index-ngram-jaccard</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-open-index/index-selection">
- <compilation-unit name="inverted-index-word-contains">
- <output-dir compare="Text">inverted-index-word-contains</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-open-index/index-selection">
- <compilation-unit name="inverted-index-word-jaccard">
- <output-dir compare="Text">inverted-index-word-jaccard</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-open-index/index-selection">
- <compilation-unit name="orders-index-custkey">
- <output-dir compare="Text">orders-index-custkey</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-open-index/index-selection">
- <compilation-unit name="orders-index-custkey-conjunctive">
- <output-dir compare="Text">orders-index-custkey-conjunctive</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-open-index/index-selection">
- <compilation-unit name="range-search">
- <output-dir compare="Text">range-search</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-open-index/index-selection">
- <compilation-unit name="rtree-secondary-index">
- <output-dir compare="Text">rtree-secondary-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-open-index/index-selection">
- <compilation-unit name="non-enforced-01">
- <output-dir compare="Text">non-enforced-01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-open-index/index-selection">
- <compilation-unit name="non-enforced-02">
- <output-dir compare="Text">non-enforced-02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-open-index/index-selection">
- <compilation-unit name="non-enforced-03">
- <output-dir compare="Text">non-enforced-03</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-open-index/index-selection">
- <compilation-unit name="non-enforced-04">
- <output-dir compare="Text">non-enforced-04</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="nested-open-index/highly-open-highly-nested">
- <test-case FilePath="nested-open-index/highly-open-highly-nested">
- <compilation-unit name="bottom-closed-top-closed">
- <output-dir compare="Text">bottom-closed-top-closed</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-open-index/highly-open-highly-nested">
- <compilation-unit name="bottom-closed-top-open">
- <output-dir compare="Text">bottom-closed-top-open</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-open-index/highly-open-highly-nested">
- <compilation-unit name="bottom-open-top-closed">
- <output-dir compare="Text">bottom-open-top-closed</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-open-index/highly-open-highly-nested">
- <compilation-unit name="bottom-open-top-open">
- <output-dir compare="Text">bottom-open-top-open</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- </test-group>
- <test-group name="nested-index">
- <test-group name="nested-index/index-join">
- <test-case FilePath="nested-index/index-join">
- <compilation-unit name="btree-primary-equi-join">
- <output-dir compare="Text">btree-primary-equi-join</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index/index-join">
- <compilation-unit name="btree-secondary-equi-join">
- <output-dir compare="Text">btree-secondary-equi-join</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index/index-join">
- <compilation-unit name="ngram-edit-distance">
- <output-dir compare="Text">ngram-edit-distance</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index/index-join">
- <compilation-unit name="ngram-edit-distance-inline">
- <output-dir compare="Text">ngram-edit-distance-inline</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index/index-join">
- <compilation-unit name="ngram-jaccard">
- <output-dir compare="Text">ngram-jaccard</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index/index-join">
- <compilation-unit name="ngram-jaccard-inline">
- <output-dir compare="Text">ngram-jaccard-inline</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index/index-join">
- <compilation-unit name="rtree-spatial-intersect-point">
- <output-dir compare="Text">rtree-spatial-intersect-point</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index/index-join">
- <compilation-unit name="word-jaccard">
- <output-dir compare="Text">word-jaccard</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index/index-join">
- <compilation-unit name="word-jaccard-inline">
- <output-dir compare="Text">word-jaccard-inline</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="nested-index/index-leftouterjoin">
- <test-case FilePath="nested-index/index-leftouterjoin">
- <compilation-unit name="probe-pidx-with-join-btree-sidx1">
- <output-dir compare="Text">probe-pidx-with-join-btree-sidx1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index/index-leftouterjoin">
- <compilation-unit name="probe-pidx-with-join-btree-sidx2">
- <output-dir compare="Text">probe-pidx-with-join-btree-sidx2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index/index-leftouterjoin">
- <compilation-unit name="probe-pidx-with-join-invidx-sidx1">
- <output-dir compare="Text">probe-pidx-with-join-invidx-sidx1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index/index-leftouterjoin">
- <compilation-unit name="probe-pidx-with-join-invidx-sidx2">
- <output-dir compare="Text">probe-pidx-with-join-invidx-sidx2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index/index-leftouterjoin">
- <compilation-unit name="probe-pidx-with-join-rtree-sidx1">
- <output-dir compare="Text">probe-pidx-with-join-rtree-sidx1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index/index-leftouterjoin">
- <compilation-unit name="probe-pidx-with-join-rtree-sidx2">
- <output-dir compare="Text">probe-pidx-with-join-rtree-sidx2</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="nested-index/index-selection">
- <test-case FilePath="nested-index/index-selection">
- <compilation-unit name="btree-index-composite-key">
- <output-dir compare="Text">btree-index-composite-key</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index/index-selection">
- <compilation-unit name="btree-sec-primary-index">
- <output-dir compare="Text">btree-sec-primary-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index/index-selection">
- <compilation-unit name="btree-index-composite-key-mixed-intervals">
- <output-dir compare="Text">btree-index-composite-key-mixed-intervals</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index/index-selection">
- <compilation-unit name="btree-index-rewrite-multiple">
- <output-dir compare="Text">btree-index-rewrite-multiple</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index/index-selection">
- <compilation-unit name="cust-index-age-nullable">
- <output-dir compare="Text">cust-index-age-nullable</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index/index-selection">
- <compilation-unit name="inverted-index-ngram-contains">
- <output-dir compare="Text">inverted-index-ngram-contains</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index/index-selection">
- <compilation-unit name="inverted-index-ngram-edit-distance">
- <output-dir compare="Text">inverted-index-ngram-edit-distance</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index/index-selection">
- <compilation-unit name="inverted-index-ngram-edit-distance-contains">
- <output-dir compare="Text">inverted-index-ngram-edit-distance-contains</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index/index-selection">
- <compilation-unit name="inverted-index-ngram-edit-distance-panic">
- <output-dir compare="Text">inverted-index-ngram-edit-distance-panic</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index/index-selection">
- <compilation-unit name="inverted-index-ngram-edit-distance-word-tokens">
- <output-dir compare="Text">inverted-index-ngram-edit-distance-word-tokens</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index/index-selection">
- <compilation-unit name="inverted-index-ngram-jaccard">
- <output-dir compare="Text">inverted-index-ngram-jaccard</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index/index-selection">
- <compilation-unit name="inverted-index-olist-edit-distance">
- <output-dir compare="Text">inverted-index-olist-edit-distance</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index/index-selection">
- <compilation-unit name="inverted-index-olist-edit-distance-panic">
- <output-dir compare="Text">inverted-index-olist-edit-distance-panic</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index/index-selection">
- <compilation-unit name="inverted-index-olist-jaccard">
- <output-dir compare="Text">inverted-index-olist-jaccard</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index/index-selection">
- <compilation-unit name="inverted-index-ulist-jaccard">
- <output-dir compare="Text">inverted-index-ulist-jaccard</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index/index-selection">
- <compilation-unit name="inverted-index-word-contains">
- <output-dir compare="Text">inverted-index-word-contains</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index/index-selection">
- <compilation-unit name="inverted-index-word-jaccard">
- <output-dir compare="Text">inverted-index-word-jaccard</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index/index-selection">
- <compilation-unit name="orders-index-custkey">
- <output-dir compare="Text">orders-index-custkey</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index/index-selection">
- <compilation-unit name="orders-index-custkey-conjunctive">
- <output-dir compare="Text">orders-index-custkey-conjunctive</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index/index-selection">
- <compilation-unit name="orders-index-custkey-conjunctive-open">
- <output-dir compare="Text">orders-index-custkey-conjunctive-open</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index/index-selection">
- <compilation-unit name="orders-index-custkey-open">
- <output-dir compare="Text">orders-index-custkey-open</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index/index-selection">
- <compilation-unit name="range-search">
- <output-dir compare="Text">range-search</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index/index-selection">
- <compilation-unit name="range-search-open">
- <output-dir compare="Text">range-search-open</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index/index-selection">
- <compilation-unit name="rtree-secondary-index">
- <output-dir compare="Text">rtree-secondary-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index/index-selection">
- <compilation-unit name="rtree-secondary-index-nullable">
- <output-dir compare="Text">rtree-secondary-index-nullable</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index/index-selection">
- <compilation-unit name="rtree-secondary-index-open">
- <output-dir compare="Text">rtree-secondary-index-open</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index/index-selection">
- <compilation-unit name="rtree-secondary-index-optional">
- <output-dir compare="Text">rtree-secondary-index-open</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- </test-group>
- <test-group name="nested-index-dml">
- <test-case FilePath="nested-index-dml">
- <compilation-unit name="compact-dataset-and-its-indexes">
- <output-dir compare="Text">compact-dataset-and-its-indexes</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index-dml">
- <compilation-unit name="nested-uuid-load">
- <output-dir compare="Text">nested-uuid-load</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index-dml">
- <compilation-unit name="nested-uuid-insert">
- <output-dir compare="Text">nested-uuid-insert</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index-dml">
- <compilation-unit name="delete-from-loaded-dataset-with-index">
- <output-dir compare="Text">delete-from-loaded-dataset-with-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index-dml">
- <compilation-unit name="delete-from-loaded-dataset-with-sec-primary-index">
- <output-dir compare="Text">delete-from-loaded-dataset-with-sec-primary-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index-dml">
- <compilation-unit name="drop-index">
- <output-dir compare="Text">drop-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index-dml">
- <compilation-unit name="insert-into-empty-dataset-with-index">
- <output-dir compare="Text">insert-into-empty-dataset-with-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index-dml">
- <compilation-unit name="insert-into-empty-dataset-with-sec-primary-index">
- <output-dir compare="Text">insert-into-empty-dataset-with-sec-primary-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index-dml">
- <compilation-unit name="insert-into-loaded-dataset-with-index_01">
- <output-dir compare="Text">insert-into-loaded-dataset-with-index_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index-dml">
- <compilation-unit name="insert-into-loaded-dataset-with-index_02">
- <output-dir compare="Text">insert-into-loaded-dataset-with-index_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index-dml">
- <compilation-unit name="insert-into-loaded-dataset-with-sec-primary-index">
- <output-dir compare="Text">insert-into-loaded-dataset-with-sec-primary-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index-dml">
- <compilation-unit name="load-with-index">
- <output-dir compare="Text">load-with-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index-dml">
- <compilation-unit name="load-with-sec-primary-index">
- <output-dir compare="Text">load-with-sec-primary-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index-dml">
- <compilation-unit name="load-with-ngram-index">
- <output-dir compare="Text">load-with-ngram-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index-dml">
- <compilation-unit name="load-with-rtree-index">
- <output-dir compare="Text">load-with-rtree-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index-dml">
- <compilation-unit name="load-with-word-index">
- <output-dir compare="Text">load-with-word-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index-dml">
- <compilation-unit name="scan-delete-btree-secondary-index-nullable">
- <output-dir compare="Text">scan-delete-btree-secondary-index-nullable</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index-dml">
- <compilation-unit name="scan-delete-rtree-secondary-index-nullable">
- <output-dir compare="Text">scan-delete-rtree-secondary-index-nullable</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index-dml">
- <compilation-unit name="scan-delete-rtree-secondary-index">
- <output-dir compare="Text">scan-delete-rtree-secondary-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index-dml">
- <compilation-unit name="scan-insert-btree-secondary-index-nullable">
- <output-dir compare="Text">scan-insert-btree-secondary-index-nullable</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index-dml">
- <compilation-unit name="scan-insert-rtree-secondary-index-nullable">
- <output-dir compare="Text">scan-insert-rtree-secondary-index-nullable</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index-dml">
- <compilation-unit name="scan-insert-rtree-secondary-index">
- <output-dir compare="Text">scan-insert-rtree-secondary-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index-dml">
- <compilation-unit name="scan-insert-inverted-index-ngram-secondary-index">
- <output-dir compare="Text">scan-insert-inverted-index-ngram-secondary-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index-dml">
- <compilation-unit name="scan-insert-inverted-index-word-secondary-index">
- <output-dir compare="Text">scan-insert-inverted-index-word-secondary-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index-dml">
- <compilation-unit name="scan-insert-inverted-index-ngram-secondary-index-nullable">
- <output-dir compare="Text">scan-insert-inverted-index-ngram-secondary-index-nullable</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index-dml">
- <compilation-unit name="scan-insert-inverted-index-word-secondary-index-nullable">
- <output-dir compare="Text">scan-insert-inverted-index-word-secondary-index-nullable</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index-dml">
- <compilation-unit name="scan-delete-inverted-index-ngram-secondary-index">
- <output-dir compare="Text">scan-delete-inverted-index-ngram-secondary-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index-dml">
- <compilation-unit name="scan-delete-inverted-index-word-secondary-index">
- <output-dir compare="Text">scan-delete-inverted-index-word-secondary-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index-dml">
- <compilation-unit name="scan-delete-inverted-index-ngram-secondary-index-nullable">
- <output-dir compare="Text">scan-delete-inverted-index-ngram-secondary-index-nullable</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nested-index-dml">
- <compilation-unit name="scan-delete-inverted-index-word-secondary-index-nullable">
- <output-dir compare="Text">scan-delete-inverted-index-word-secondary-index-nullable</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="array-index">
- <test-group name="array-index/error-handling">
- <test-case FilePath="array-index/error-handling">
- <compilation-unit name="index-include-unknown-key">
- <output-dir compare="Text">index-include-unknown-key</output-dir>
- <expected-error>ASX1079: Compilation error: Array indexes must specify EXCLUDE UNKNOWN KEY</expected-error>
- <expected-error>ASX1079: Compilation error: Array indexes must specify EXCLUDE UNKNOWN KEY</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/error-handling">
- <compilation-unit name="index-two-array-fields">
- <output-dir compare="Text">index-two-array-fields</output-dir>
- <expected-error>ASX1079: Compilation error: Cannot create composite index with multiple array fields using different arrays</expected-error>
- <expected-error>ASX1079: Compilation error: Cannot create composite index with multiple array fields using different arrays</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/error-handling">
- <compilation-unit name="invalid-array-path">
- <output-dir compare="Text">invalid-array-path</output-dir>
- <expected-error>ASX0037: Type mismatch: expected value of type array or multiset, but got the value of type CheckinType_checkin_time:</expected-error>
- <expected-error>ASX0037: Type mismatch: expected value of type array or multiset, but got the value of type string</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/error-handling">
- <compilation-unit name="index-on-closed-array">
- <output-dir compare="Text">index-on-closed-array</output-dir>
- <expected-error>ASX1014: Field 'date' is not found</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/error-handling">
- <compilation-unit name="index-with-enforced-type">
- <output-dir compare="Text">index-with-enforced-type</output-dir>
- <expected-error>ASX1155: Incompatible index type ARRAY</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/error-handling">
- <compilation-unit name="no-field-type">
- <output-dir compare="Text">no-field-type</output-dir>
- <expected-error>ASX1079: Compilation error: cannot find type of elements of field 'open_array_f'</expected-error>
- <expected-error>ASX1079: Compilation error: cannot find type of elements of field 'nested_array'</expected-error>
- <expected-error>ASX1079: Compilation error: cannot find type of elements of field 'nested_array'</expected-error>
- <expected-error>ASX1079: Compilation error: cannot find type of field 'proj1'</expected-error>
- <expected-error>ASX1079: Compilation error: cannot find type of field 'proj1'</expected-error>
- <expected-error>ASX1079: Compilation error: cannot find type of field 'proj1'</expected-error>
- <expected-error>ASX1079: Compilation error: cannot find type of field 'proj1'</expected-error>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="array-index/metadata">
- <test-case FilePath="array-index/metadata/closed">
- <compilation-unit name="use-case-1">
- <output-dir compare="Text">use-case-1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/metadata/open">
- <compilation-unit name="use-case-1">
- <output-dir compare="Text">use-case-1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/metadata/closed">
- <compilation-unit name="use-case-2">
- <output-dir compare="Text">use-case-2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/metadata/open">
- <compilation-unit name="use-case-2">
- <output-dir compare="Text">use-case-2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/metadata/closed">
- <compilation-unit name="use-case-3">
- <output-dir compare="Text">use-case-3</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/metadata/open">
- <compilation-unit name="use-case-3">
- <output-dir compare="Text">use-case-3</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/metadata/closed">
- <compilation-unit name="use-case-4">
- <output-dir compare="Text">use-case-4</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/metadata/open">
- <compilation-unit name="use-case-4">
- <output-dir compare="Text">use-case-4</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/metadata/open">
- <compilation-unit name="complex-structures">
- <output-dir compare="Text">complex-structures</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/metadata/closed">
- <compilation-unit name="with-composite-sk">
- <output-dir compare="Text">with-composite-sk</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/metadata/open">
- <compilation-unit name="with-composite-sk">
- <output-dir compare="Text">with-composite-sk</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/metadata/closed">
- <compilation-unit name="with-composite-array-different-indicators">
- <output-dir compare="Text">with-composite-array-different-indicators</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/metadata/open">
- <compilation-unit name="with-composite-array-different-indicators">
- <output-dir compare="Text">with-composite-array-different-indicators</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/metadata/closed">
- <compilation-unit name="with-3-level-record-path">
- <output-dir compare="Text">with-3-level-record-path</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/metadata/open">
- <compilation-unit name="with-3-level-record-path">
- <output-dir compare="Text">with-3-level-record-path</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="array-index/bulk-loading/on-index-creation">
- <test-case FilePath="array-index/bulk-loading/on-index-creation/closed">
- <compilation-unit name="use-case-1">
- <output-dir compare="Text">use-case-1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/bulk-loading/on-index-creation/open">
- <compilation-unit name="use-case-1">
- <output-dir compare="Text">use-case-1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/bulk-loading/on-index-creation/closed">
- <compilation-unit name="use-case-2">
- <output-dir compare="Text">use-case-2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/bulk-loading/on-index-creation/open">
- <compilation-unit name="use-case-2">
- <output-dir compare="Text">use-case-2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/bulk-loading/on-index-creation/closed">
- <compilation-unit name="use-case-3">
- <output-dir compare="Text">use-case-3</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/bulk-loading/on-index-creation/open">
- <compilation-unit name="use-case-3">
- <output-dir compare="Text">use-case-3</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/bulk-loading/on-index-creation/closed">
- <compilation-unit name="use-case-4">
- <output-dir compare="Text">use-case-4</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/bulk-loading/on-index-creation/open">
- <compilation-unit name="use-case-4">
- <output-dir compare="Text">use-case-4</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/bulk-loading/on-index-creation/closed">
- <compilation-unit name="with-composite-pk">
- <output-dir compare="Text">with-composite-pk</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/bulk-loading/on-index-creation/closed">
- <compilation-unit name="with-filter-fields">
- <output-dir compare="Text">with-filter-fields</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/bulk-loading/on-index-creation/closed">
- <compilation-unit name="with-3-level-record-path">
- <output-dir compare="Text">with-3-level-record-path</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/bulk-loading/on-index-creation/open">
- <compilation-unit name="composite-atomic">
- <output-dir compare="Text">composite-atomic</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="array-index/bulk-loading/after-index-creation">
- <test-case FilePath="array-index/bulk-loading/after-index-creation">
- <compilation-unit name="use-case-1">
- <output-dir compare="Text">use-case-1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/bulk-loading/after-index-creation">
- <compilation-unit name="use-case-2">
- <output-dir compare="Text">use-case-2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/bulk-loading/after-index-creation">
- <compilation-unit name="use-case-3">
- <output-dir compare="Text">use-case-3</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/bulk-loading/after-index-creation">
- <compilation-unit name="use-case-4">
- <output-dir compare="Text">use-case-4</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/bulk-loading/after-index-creation">
- <compilation-unit name="with-3-level-record-path">
- <output-dir compare="Text">with-3-level-record-path</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/bulk-loading/after-index-creation">
- <compilation-unit name="with-composite-pk">
- <output-dir compare="Text">with-composite-pk</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/bulk-loading/after-index-creation">
- <compilation-unit name="with-filter-fields">
- <output-dir compare="Text">with-filter-fields</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/bulk-loading/after-index-creation">
- <compilation-unit name="with-open-index">
- <output-dir compare="Text">with-open-index</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="array-index/insert-upsert-delete">
- <test-case FilePath="array-index/insert-upsert-delete/closed">
- <compilation-unit name="use-case-1">
- <output-dir compare="Text">use-case-1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/insert-upsert-delete/open">
- <compilation-unit name="use-case-1">
- <output-dir compare="Text">use-case-1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/insert-upsert-delete/closed">
- <compilation-unit name="use-case-2">
- <output-dir compare="Text">use-case-2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/insert-upsert-delete/open">
- <compilation-unit name="use-case-2">
- <output-dir compare="Text">use-case-2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/insert-upsert-delete/closed">
- <compilation-unit name="use-case-3">
- <output-dir compare="Text">use-case-3</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/insert-upsert-delete/open">
- <compilation-unit name="use-case-3">
- <output-dir compare="Text">use-case-3</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/insert-upsert-delete/closed">
- <compilation-unit name="use-case-4">
- <output-dir compare="Text">use-case-4</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/insert-upsert-delete/open">
- <compilation-unit name="use-case-4">
- <output-dir compare="Text">use-case-4</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/insert-upsert-delete/closed">
- <compilation-unit name="with-composite-sk">
- <output-dir compare="Text">with-composite-sk</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/insert-upsert-delete/open">
- <compilation-unit name="with-composite-sk">
- <output-dir compare="Text">with-composite-sk</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/insert-upsert-delete/closed">
- <compilation-unit name="with-additional-atomic-index">
- <output-dir compare="Text">with-additional-atomic-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/insert-upsert-delete/open">
- <compilation-unit name="with-additional-atomic-index">
- <output-dir compare="Text">with-additional-atomic-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/insert-upsert-delete/closed">
- <compilation-unit name="with-filter-fields">
- <output-dir compare="Text">with-filter-fields</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="array-index/join-quantified-queries">
- <test-case FilePath="array-index/join-quantified-queries">
- <compilation-unit name="use-case-1">
- <output-dir compare="Text">use-case-1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/join-quantified-queries">
- <compilation-unit name="use-case-2">
- <output-dir compare="Text">use-case-2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/join-quantified-queries">
- <compilation-unit name="use-case-3">
- <output-dir compare="Text">use-case-3</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/join-quantified-queries">
- <compilation-unit name="use-case-4">
- <output-dir compare="Text">use-case-4</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="array-index/join-unnest-queries">
- <test-case FilePath="array-index/join-unnest-queries">
- <compilation-unit name="use-case-1">
- <output-dir compare="Text">use-case-1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/join-unnest-queries">
- <compilation-unit name="use-case-2">
- <output-dir compare="Text">use-case-2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/join-unnest-queries">
- <compilation-unit name="use-case-3">
- <output-dir compare="Text">use-case-3</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/join-unnest-queries">
- <compilation-unit name="use-case-4">
- <output-dir compare="Text">use-case-4</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/join-unnest-queries">
- <compilation-unit name="with-open-index">
- <output-dir compare="Text">with-open-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/join-unnest-queries">
- <compilation-unit name="loj-subquery">
- <output-dir compare="Text">loj-subquery</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="array-index/select-unnest-queries">
- <test-case FilePath="array-index/select-unnest-queries/closed">
- <compilation-unit name="use-case-1">
- <output-dir compare="Text">use-case-1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/select-unnest-queries/open">
- <compilation-unit name="use-case-1">
- <output-dir compare="Text">use-case-1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/select-unnest-queries/closed">
- <compilation-unit name="use-case-2">
- <output-dir compare="Text">use-case-2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/select-unnest-queries/open">
- <compilation-unit name="use-case-2">
- <output-dir compare="Text">use-case-2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/select-unnest-queries/closed">
- <compilation-unit name="use-case-3">
- <output-dir compare="Text">use-case-3</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/select-unnest-queries/open">
- <compilation-unit name="use-case-3">
- <output-dir compare="Text">use-case-3</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/select-unnest-queries/closed">
- <compilation-unit name="use-case-4">
- <output-dir compare="Text">use-case-4</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/select-unnest-queries/open">
- <compilation-unit name="use-case-4">
- <output-dir compare="Text">use-case-4</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/select-unnest-queries/closed">
- <compilation-unit name="with-3-level-record-path">
- <output-dir compare="Text">with-3-level-record-path</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/select-unnest-queries/open">
- <compilation-unit name="with-3-level-record-path">
- <output-dir compare="Text">with-3-level-record-path</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/select-unnest-queries/closed">
- <compilation-unit name="with-composite-sk">
- <output-dir compare="Text">with-composite-sk</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/select-unnest-queries/open">
- <compilation-unit name="with-composite-sk">
- <output-dir compare="Text">with-composite-sk</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/select-unnest-queries/closed">
- <compilation-unit name="with-composite-pk">
- <output-dir compare="Text">with-composite-pk</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/select-unnest-queries/closed">
- <compilation-unit name="with-filter-fields">
- <output-dir compare="Text">with-filter-fields</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/select-unnest-queries/open">
- <compilation-unit name="using-feed-new-index">
- <output-dir compare="Text">using-feed</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/select-unnest-queries/open">
- <compilation-unit name="using-feed-old-index">
- <output-dir compare="Text">using-feed</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="array-index/select-quantified-queries">
- <test-case FilePath="array-index/select-quantified-queries">
- <compilation-unit name="use-case-1">
- <output-dir compare="Text">use-case-1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/select-quantified-queries">
- <compilation-unit name="use-case-2">
- <output-dir compare="Text">use-case-2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/select-quantified-queries">
- <compilation-unit name="use-case-3">
- <output-dir compare="Text">use-case-3</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/select-quantified-queries">
- <compilation-unit name="use-case-4">
- <output-dir compare="Text">use-case-4</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/select-quantified-queries">
- <compilation-unit name="with-composite-pk">
- <output-dir compare="Text">with-composite-pk</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/select-quantified-queries">
- <compilation-unit name="with-open-index">
- <output-dir compare="Text">with-open-index</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="array-index/composite-index-queries">
- <test-case FilePath="array-index">
- <compilation-unit name="composite-index-queries">
- <output-dir compare="Text">composite-index-queries</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- </test-group>
- <test-group name="nestrecords">
- <test-case FilePath="nestrecords">
- <compilation-unit name="nestrecord">
- <output-dir compare="Text">nestrecord</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nestrecords">
- <compilation-unit name="nested-optional-pk">
- <output-dir compare="Text">nested-optional-pk</output-dir>
- <expected-error>ASX1021: The primary key field 'nested.id' cannot be nullable</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nestrecords">
- <compilation-unit name="query-ASTERIXDB-1025">
- <output-dir compare="Text">query-ASTERIXDB-1025</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="null-missing">
- <test-case FilePath="null-missing">
- <compilation-unit name="array">
- <output-dir compare="Text">array</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="null-missing">
- <compilation-unit name="boolean">
- <output-dir compare="Text">boolean</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="null-missing">
- <compilation-unit name="ceil">
- <output-dir compare="Text">ceil</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="null-missing">
- <compilation-unit name="comparison">
- <output-dir compare="Text">comparison</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="null-missing">
- <compilation-unit name="field-access">
- <output-dir compare="Text">field-access</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="null-missing">
- <compilation-unit name="group-by">
- <output-dir compare="Text">group-by</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="null-missing">
- <compilation-unit name="group-by-from-dataset">
- <output-dir compare="Text">group-by-from-dataset</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="null-missing">
- <compilation-unit name="length">
- <output-dir compare="Text">length</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="null-missing">
- <compilation-unit name="lower">
- <output-dir compare="Text">lower</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="null-missing">
- <compilation-unit name="order-by">
- <output-dir compare="Text">order-by</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="null-missing">
- <compilation-unit name="order-by-2">
- <output-dir compare="Text">order-by-2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="null-missing">
- <compilation-unit name="order-by-3-negative">
- <output-dir compare="Text">none</output-dir>
- <expected-error><![CDATA[ASX1001: Syntax error: In line 33 >>ORDER BY d.a NULLS;<< Encountered ";" at column 19]]></expected-error>
- <expected-error><![CDATA[ASX1001: Syntax error: Unexpected token: NULLS (in line 33, at column 20)]]></expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="null-missing">
- <compilation-unit name="order-by-from-dataset">
- <output-dir compare="Text">order-by-from-dataset</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="null-missing">
- <compilation-unit name="order-by-from-dataset-2">
- <output-dir compare="Text">order-by-from-dataset-2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="null-missing">
- <compilation-unit name="scan-collection">
- <output-dir compare="Text">scan-collection</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="null-missing">
- <compilation-unit name="spatial-intersect">
- <output-dir compare="Text">spatial-intersect</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="null-missing">
- <compilation-unit name="substr">
- <output-dir compare="Text">substr</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="null-missing">
- <compilation-unit name="upper">
- <output-dir compare="Text">upper</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="null-missing">
- <compilation-unit name="is">
- <output-dir compare="Text">is</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="null-missing">
- <compilation-unit name="ifmissing">
- <output-dir compare="Text">ifmissing</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="null-missing">
- <compilation-unit name="ifnull">
- <output-dir compare="Text">ifnull</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="null-missing">
- <compilation-unit name="ifmissingornull">
- <output-dir compare="Text">ifmissingornull</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="null-missing">
- <compilation-unit name="coalesce">
- <output-dir compare="Text">coalesce</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="null-missing">
- <compilation-unit name="query-ASTERIXDB-1689">
- <output-dir compare="Text">query-ASTERIXDB-1689</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="null-missing">
- <compilation-unit name="print-ASTERIXDB-1885">
- <output-dir compare="Text">print-ASTERIXDB-1885</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="numeric">
- <test-case FilePath="numeric">
- <compilation-unit name="caret0">
- <output-dir compare="Text">caret0</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="caret1">
- <output-dir compare="Text">caret1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="abs0">
- <output-dir compare="Text">abs0</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="abs1">
- <output-dir compare="Text">abs1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="abs2">
- <output-dir compare="Text">abs2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="abs3">
- <output-dir compare="Text">abs3</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="abs4">
- <output-dir compare="Text">abs4</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="add_double">
- <output-dir compare="Text">add_double</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="add_float">
- <output-dir compare="Text">add_float</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="add_int16">
- <output-dir compare="Text">add_int16</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="add_int32">
- <output-dir compare="Text">add_int32</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="add_int64">
- <output-dir compare="Text">add_int64</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="add_int8">
- <output-dir compare="Text">add_int8</output-dir>
- </compilation-unit>
- </test-case>
- <!--
- <test-case FilePath="numeric">
- <compilation-unit name="issue_1166">
- <output-dir compare="Text">issue_1166</output-dir>
- </compilation-unit>
- </test-case>
- -->
- <test-case FilePath="numeric">
- <compilation-unit name="ceiling0">
- <output-dir compare="Text">ceiling0</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="ceiling1">
- <output-dir compare="Text">ceiling1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="ceiling2">
- <output-dir compare="Text">ceiling2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="ceiling3">
- <output-dir compare="Text">ceiling3</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="ceiling4">
- <output-dir compare="Text">ceiling4</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="const">
- <output-dir compare="Text">const</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="degrees">
- <output-dir compare="Text">degrees</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="divide_double">
- <output-dir compare="Text">divide_double</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="divide_float">
- <output-dir compare="Text">divide_float</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="divide_int16">
- <output-dir compare="Text">divide_int16</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="divide_int32">
- <output-dir compare="Text">divide_int32</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="divide_int64">
- <output-dir compare="Text">divide_int64</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="divide_int8">
- <output-dir compare="Text">divide_int8</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="div_mod_case_insensitive">
- <output-dir compare="Text">div_mod_case_insensitive</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="floor0">
- <output-dir compare="Text">floor0</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="floor1">
- <output-dir compare="Text">floor1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="floor2">
- <output-dir compare="Text">floor2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="floor3">
- <output-dir compare="Text">floor3</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="floor4">
- <output-dir compare="Text">floor4</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="ifinf">
- <output-dir compare="Text">ifinf</output-dir>
- <expected-error>Invalid number of arguments for function if-inf (in line 25, at column 14)</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="ifnan">
- <output-dir compare="Text">ifnan</output-dir>
- <expected-error>Invalid number of arguments for function if-nan (in line 25, at column 14)</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="ifnanorinf">
- <output-dir compare="Text">ifnanorinf</output-dir>
- <expected-error>Invalid number of arguments for function if-nan-or-inf (in line 25, at column 14)</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="infinity">
- <output-dir compare="Text">infinity</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="multiply_double">
- <output-dir compare="Text">multiply_double</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="multiply_float">
- <output-dir compare="Text">multiply_float</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="multiply_int16">
- <output-dir compare="Text">multiply_int16</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="multiply_int32">
- <output-dir compare="Text">multiply_int32</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="multiply_int64">
- <output-dir compare="Text">multiply_int64</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="multiply_int8">
- <output-dir compare="Text">multiply_int8</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="power">
- <output-dir compare="Text">power</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="query-ASTERIXDB-2530">
- <output-dir compare="Text">query-ASTERIXDB-2530</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="radians">
- <output-dir compare="Text">radians</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="round-half-to-even0">
- <output-dir compare="Text">round-half-to-even0</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="round-half-to-even1">
- <output-dir compare="Text">round-half-to-even1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="round-half-to-even2">
- <output-dir compare="Text">round-half-to-even2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="round-half-to-even20">
- <output-dir compare="Text">round-half-to-even20</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="round-half-to-even21">
- <output-dir compare="Text">round-half-to-even21</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="round-half-to-even22">
- <output-dir compare="Text">round-half-to-even22</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="round-half-to-even23">
- <output-dir compare="Text">round-half-to-even23</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="round-half-to-even24">
- <output-dir compare="Text">round-half-to-even24</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="round-half-to-even3">
- <output-dir compare="Text">round-half-to-even3</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="round-half-to-even4">
- <output-dir compare="Text">round-half-to-even4</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="round-half-to-even5">
- <output-dir compare="Text">round-half-to-even5</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="round-half-up">
- <output-dir compare="Text">round-half-up</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="round0">
- <output-dir compare="Text">round0</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="round1">
- <output-dir compare="Text">round1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="round2">
- <output-dir compare="Text">round2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="round3">
- <output-dir compare="Text">round3</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="round4">
- <output-dir compare="Text">round4</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="round5_with_digit_int8">
- <output-dir compare="Text">round5_with_digit_int8</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="round6_with_digit_int16">
- <output-dir compare="Text">round6_with_digit_int16</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="round7_with_digit_int32">
- <output-dir compare="Text">round7_with_digit_int32</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="round8_with_digit_int64">
- <output-dir compare="Text">round8_with_digit_int64</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="round9_with_digit_float">
- <output-dir compare="Text">round9_with_digit_float</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="round10_with_digit_double">
- <output-dir compare="Text">round10_with_digit_double</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="round11_invalid">
- <output-dir compare="Text">round11_invalid</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="subtract_double">
- <output-dir compare="Text">subtract_double</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="subtract_float">
- <output-dir compare="Text">subtract_float</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="subtract_int16">
- <output-dir compare="Text">subtract_int16</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="subtract_int32">
- <output-dir compare="Text">subtract_int32</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="subtract_int64">
- <output-dir compare="Text">subtract_int64</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="subtract_int8">
- <output-dir compare="Text">subtract_int8</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="trunc">
- <output-dir compare="Text">trunc</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="unary-minus_double_02">
- <output-dir compare="Text">unary-minus_double_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="unary-minus_float_02">
- <output-dir compare="Text">unary-minus_float_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="unary-minus_int_02">
- <output-dir compare="Text">unary-minus_int_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="unary-minus_null">
- <output-dir compare="Text">unary-minus_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="query-issue355"> <!-- @Yingyi, This one fails in the aql tests!! -->
- <output-dir compare="Text">query-issue355</output-dir>
- <!-- <expected-error>org.apache.asterix.common.exceptions.AsterixException</expected-error> -->
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="acos">
- <output-dir compare="Text">acos</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="asin">
- <output-dir compare="Text">asin</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="atan">
- <output-dir compare="Text">atan</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="atan2">
- <output-dir compare="Text">atan2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="cos">
- <output-dir compare="Text">cos</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="cosh">
- <output-dir compare="Text">cosh</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="sin">
- <output-dir compare="Text">sin</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="sinh">
- <output-dir compare="Text">sinh</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="tan">
- <output-dir compare="Text">tan</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="tanh">
- <output-dir compare="Text">tanh</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="exp">
- <output-dir compare="Text">exp</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="ln">
- <output-dir compare="Text">ln</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="log">
- <output-dir compare="Text">log</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="sqrt">
- <output-dir compare="Text">sqrt</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="scientific">
- <output-dir compare="Text">scientific</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="scientific_error">
- <output-dir compare="Text">none</output-dir>
- <expected-error>ASX1073: Cannot resolve alias reference for undefined identifier e (in line 24, at column 10)</expected-error>
- <expected-error>ASX1073: Cannot resolve alias reference for undefined identifier e (in line 24, at column 8)</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="sign">
- <output-dir compare="Text">sign</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="non-finite">
- <output-dir compare="Clean-JSON">non-finite</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="add_int8">
- <output-dir compare="Text">add_int8</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="multiply_int8">
- <output-dir compare="Text">multiply_int8</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
- <compilation-unit name="unary-minus_double_02">
- <output-dir compare="Text">unary-minus_double_02</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="open-closed">
- <!--
- <test-case FilePath="open-closed">
- <compilation-unit name="c2c-w-optional">
- <output-dir compare="Text">c2c-w-optional</output-dir>
- </compilation-unit>
- </test-case>
- -->
- <!--
- <test-case FilePath="open-closed">
- <compilation-unit name="c2c-wo-optional">
- <output-dir compare="Text">c2c-wo-optional</output-dir>
- </compilation-unit>
- </test-case>
- -->
- <!--
- <test-case FilePath="open-closed">
- <compilation-unit name="c2c">
- <output-dir compare="Text">c2c</output-dir>
- </compilation-unit>
- </test-case>
- -->
- <test-case FilePath="open-closed">
- <compilation-unit name="heterog-list-ordered01">
- <output-dir compare="Text">heterog-list-ordered01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-closed">
- <compilation-unit name="heterog-list01">
- <output-dir compare="Text">heterog-list01</output-dir>
- </compilation-unit>
- </test-case>
- <!--
- <test-case FilePath="open-closed">
- <compilation-unit name="heterog-list02">
- <output-dir compare="Text">heterog-list02</output-dir>
- </compilation-unit>
- </test-case>
- -->
- <!--
- <test-case FilePath="open-closed">
- <compilation-unit name="heterog-list03">
- <output-dir compare="Text">heterog-list03</output-dir>
- </compilation-unit>
- </test-case>
- -->
- <test-case FilePath="open-closed">
- <compilation-unit name="open-closed-01">
- <output-dir compare="Text">open-closed-01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-closed">
- <compilation-unit name="open-closed-12">
- <output-dir compare="Text">open-closed-12</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-closed">
- <compilation-unit name="open-closed-14">
- <output-dir compare="Text">open-closed-14</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-closed">
- <compilation-unit name="query-issue134">
- <output-dir compare="Text">query-issue134</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-closed">
- <compilation-unit name="query-issue55">
- <output-dir compare="Text">query-issue55</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-closed">
- <compilation-unit name="query-issue55-1">
- <output-dir compare="Text">query-issue55-1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-closed">
- <compilation-unit name="query-issue166">
- <output-dir compare="Text">query-issue166</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-closed">
- <compilation-unit name="query-issue208">
- <output-dir compare="Text">query-issue208</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-closed">
- <compilation-unit name="query-issue236">
- <output-dir compare="Text">query-issue236</output-dir>
- </compilation-unit>
- </test-case>
- <!--
- <test-case FilePath="open-closed">
- <compilation-unit name="open-closed-15">
- <output-dir compare="Text">open-closed-15</output-dir>
- </compilation-unit>
- </test-case>
- -->
- <!--
- <test-case FilePath="open-closed">
- <compilation-unit name="open-closed-16">
- <output-dir compare="Text">open-closed-16</output-dir>
- </compilation-unit>
- </test-case>
- -->
- <!--
- <test-case FilePath="open-closed">
- <compilation-unit name="open-closed-17">
- <output-dir compare="Text">open-closed-17</output-dir>
- </compilation-unit>
- </test-case>
- -->
- <!--
- <test-case FilePath="open-closed">
- <compilation-unit name="open-closed-19">
- <output-dir compare="Text">open-closed-19</output-dir>
- </compilation-unit>
- </test-case>
- -->
- <!--
- <test-case FilePath="open-closed">
- <compilation-unit name="open-closed-20">
- <output-dir compare="Text">open-closed-20</output-dir>
- </compilation-unit>
- </test-case>
- -->
- <!--
- <test-case FilePath="open-closed">
- <compilation-unit name="open-closed-21">
- <output-dir compare="Text">open-closed-21</output-dir>
- </compilation-unit>
- </test-case>
- -->
- <!--
- <test-case FilePath="open-closed">
- <compilation-unit name="open-closed-22">
- <output-dir compare="Text">open-closed-22</output-dir>
- </compilation-unit>
- </test-case>
- -->
- <test-case FilePath="open-closed">
- <compilation-unit name="open-closed-24">
- <output-dir compare="Text">open-closed-24</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-closed">
- <compilation-unit name="open-closed-25">
- <output-dir compare="Text">open-closed-25</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-closed">
- <compilation-unit name="open-closed-26">
- <output-dir compare="Text">open-closed-26</output-dir>
- </compilation-unit>
- </test-case>
- <!--
- <test-case FilePath="open-closed">
- <compilation-unit name="open-closed-28">
- <output-dir compare="Text">open-closed-28</output-dir>
- </compilation-unit>
- </test-case>
- -->
- <test-case FilePath="open-closed">
- <compilation-unit name="open-closed-29">
- <output-dir compare="Text">open-closed-29</output-dir>
- </compilation-unit>
- </test-case>
- <!--
- <test-case FilePath="open-closed">
- <compilation-unit name="open-closed-30">
- <output-dir compare="Text">open-closed-30</output-dir>
- </compilation-unit>
- </test-case>
- -->
- <test-case FilePath="open-closed">
- <compilation-unit name="open-closed-31">
- <output-dir compare="Text">open-closed-31</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-closed">
- <compilation-unit name="open-closed-32">
- <output-dir compare="Text">open-closed-32</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-closed">
- <compilation-unit name="open-closed-33">
- <output-dir compare="Text">open-closed-33</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-closed">
- <compilation-unit name="query-proposal02">
- <output-dir compare="Text">query-proposal02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-closed">
- <compilation-unit name="query-proposal">
- <output-dir compare="Text">query-proposal</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-closed">
- <compilation-unit name="query-issue350">
- <output-dir compare="Text">query-issue350</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-closed">
- <compilation-unit name="query-issue350-2">
- <output-dir compare="Text">query-issue350-2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-closed">
- <compilation-unit name="query-issue343">
- <output-dir compare="Text">query-issue343</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-closed">
- <compilation-unit name="query-issue343-2">
- <output-dir compare="Text">query-issue343-2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-closed">
- <compilation-unit name="query-issue196">
- <output-dir compare="Text">query-issue196</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-closed">
- <compilation-unit name="query-issue377">
- <output-dir compare="Text">query-issue377</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-closed">
- <compilation-unit name="query-issue410">
- <output-dir compare="Text">query-issue410</output-dir>
- <expected-error>Field type double cannot be promoted to type string</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-closed">
- <compilation-unit name="query-issue453">
- <output-dir compare="Text">query-issue453</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-closed">
- <compilation-unit name="query-issue453-2">
- <output-dir compare="Text">query-issue453-2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-closed">
- <compilation-unit name="query-issue456">
- <output-dir compare="Text">query-issue456</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-closed">
- <compilation-unit name="query-issue465">
- <output-dir compare="Text">query-issue465</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-closed">
- <compilation-unit name="query-issue258">
- <output-dir compare="Text">query-issue258</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-closed">
- <compilation-unit name="query-issue423">
- <output-dir compare="Text">query-issue423</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-closed">
- <compilation-unit name="query-issue423-2">
- <output-dir compare="Text">query-issue423-2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-closed">
- <compilation-unit name="query-issue442"><!-- Exception is not thrown!! -->
- <output-dir compare="Text">query-issue442</output-dir>
- <!-- <expected-error>org.apache.asterix.common.exceptions.AsterixException</expected-error> -->
- </compilation-unit>
- </test-case>
- <!--<test-case FilePath="open-closed">
- <compilation-unit name="query-issue487">
- <output-dir compare="Text">query-issue487</output-dir>
- <expected-error>org.apache.asterix.common.exceptions.AsterixException</expected-error>
- </compilation-unit>
- </test-case> -->
- <test-case FilePath="open-closed">
- <compilation-unit name="query-issue592">
- <output-dir compare="Text">query-issue592</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-closed">
- <compilation-unit name="query-issue625">
- <output-dir compare="Text">query-issue625</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-closed">
- <compilation-unit name="query-issue208">
- <output-dir compare="Text">query-issue208</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="open-closed">
- <compilation-unit name="query-issue236">
- <output-dir compare="Text">query-issue236</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="quantifiers">
- <test-case FilePath="quantifiers">
- <compilation-unit name="any_and_every_01">
- <output-dir compare="Text">any_and_every_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="quantifiers">
- <compilation-unit name="anysat_01">
- <output-dir compare="Text">somesat_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="quantifiers">
- <compilation-unit name="everysat_01">
- <output-dir compare="Text">everysat_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="quantifiers">
- <compilation-unit name="query-ASTERIXDB-1005">
- <output-dir compare="Text">query-ASTERIXDB-1005</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="quantifiers">
- <compilation-unit name="query-ASTERIXDB-1674">
- <output-dir compare="Text">query-ASTERIXDB-1674</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="quantifiers">
- <compilation-unit name="query-ASTERIXDB-2307">
- <output-dir compare="Text">query-ASTERIXDB-2307</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="quantifiers">
- <compilation-unit name="query-ASTERIXDB-2696">
- <output-dir compare="Text">query-ASTERIXDB-2696</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="quantifiers">
- <compilation-unit name="query-ASTERIXDB-2947">
- <output-dir compare="Text">query-ASTERIXDB-2947</output-dir>
- </compilation-unit>
- </test-case>
- <!--
- <test-case FilePath="quantifiers">
- <compilation-unit name="everysat_02">
- <output-dir compare="Text">everysat_02</output-dir>
- </compilation-unit>
- </test-case>
- -->
- <!--
- <test-case FilePath="quantifiers">
- <compilation-unit name="everysat_03">
- <output-dir compare="Text">everysat_03</output-dir>
- </compilation-unit>
- </test-case>
- -->
- <test-case FilePath="quantifiers">
- <compilation-unit name="everysat_04">
- <output-dir compare="Text">everysat_04</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="quantifiers">
- <compilation-unit name="somesat_01">
- <output-dir compare="Text">somesat_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="quantifiers">
- <compilation-unit name="somesat_02">
- <output-dir compare="Text">somesat_02</output-dir>
- </compilation-unit>
- </test-case>
- <!--
- <test-case FilePath="quantifiers">
- <compilation-unit name="somesat_03">
- <output-dir compare="Text">somesat_03</output-dir>
- </compilation-unit>
- </test-case>
- -->
- <!--
- <test-case FilePath="quantifiers">
- <compilation-unit name="somesat_04">
- <output-dir compare="Text">somesat_04</output-dir>
- </compilation-unit>
- </test-case>
- -->
- <!--
- <test-case FilePath="quantifiers">
- <compilation-unit name="somesat_05">
- <output-dir compare="Text">somesat_05</output-dir>
- </compilation-unit>
- </test-case>
- -->
- <test-case FilePath="quantifiers">
- <compilation-unit name="somesat_06">
- <output-dir compare="Text">somesat_06</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="range-hints">
- <test-case FilePath="range-hints">
- <compilation-unit name="order-by">
- <output-dir compare="Text">order-by</output-dir>
- </compilation-unit>
- </test-case>
- <!-- Fail sporadically <test-case FilePath="range-hints">
- <compilation-unit name="order-by-exception_01">
- <output-dir compare="Text">order-by</output-dir>
- <expected-error>org.json.JSONException: JSONObject['summary'] not found</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="range-hints">
- <compilation-unit name="order-by-exception_02">
- <output-dir compare="Text">order-by</output-dir>
- <expected-error>org.json.JSONException: JSONObject['summary'] not found</expected-error>
- </compilation-unit>
- </test-case> -->
- </test-group>
- <test-group name="resolution">
- <test-case FilePath="resolution">
- <compilation-unit name="conflict-field-dataset">
- <output-dir compare="Text">conflict-field-dataset</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="resolution">
- <compilation-unit name="conflict-field-dataset-from">
- <output-dir compare="Text">conflict-field-dataset-from</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="resolution">
- <compilation-unit name="conflict-field-dataset-fromterm">
- <output-dir compare="Text">conflict-field-dataset-from</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="resolution">
- <compilation-unit name="conflict-field-dataset-join">
- <output-dir compare="Text">conflict-field-dataset-from</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="resolution">
- <compilation-unit name="conflict-field-dataset-quantifier">
- <output-dir compare="Text">conflict-field-dataset-from</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="resolution">
- <compilation-unit name="conflict-field-dataset-unnest">
- <output-dir compare="Text">conflict-field-dataset-from</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="resolution">
- <compilation-unit name="conflict-fields-dataset">
- <output-dir compare="Text">conflict-field-dataset</output-dir>
- <expected-error>Cannot resolve ambiguous alias reference for identifier samptable</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="resolution">
- <compilation-unit name="fullyqualified">
- <output-dir compare="Text">fullyqualified</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="resolution">
- <compilation-unit name="fullyqualified2">
- <output-dir compare="Text">fullyqualified2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="resolution">
- <compilation-unit name="order_1">
- <output-dir compare="Text">order_1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="resolution">
- <compilation-unit name="groupby_rename_with_sugar">
- <output-dir compare="Text">groupby_rename_with_sugar</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="resolution">
- <compilation-unit name="function_dataverse">
- <output-dir compare="Text">function_dataverse</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="resolution">
- <compilation-unit name="field_accessor_1">
- <output-dir compare="Text">field_accessor_1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="resolution">
- <compilation-unit name="field_accessor_2_negative">
- <output-dir compare="Text">field_accessor_1</output-dir>
- <expected-error>ASX1074: Cannot resolve ambiguous alias reference for identifier c2 (in line 25, at column 7)</expected-error>
- <expected-error>ASX1074: Cannot resolve ambiguous alias reference for identifier c2 (in line 25, at column 7)</expected-error>
- <expected-error>ASX1074: Cannot resolve ambiguous alias reference for identifier c2 (in line 25, at column 7)</expected-error>
- <expected-error>ASX1074: Cannot resolve ambiguous alias reference for identifier c10 (in line 25, at column 51)</expected-error>
- <expected-error>ASX1073: Cannot resolve alias reference for undefined identifier c20 (in line 26, at column 8)</expected-error>
- <expected-error>ASX1073: Cannot resolve alias reference for undefined identifier c1 (in line 25, at column 19)</expected-error>
- <expected-error>ASX1073: Cannot resolve alias reference for undefined identifier samptable (in line 25, at column 14)</expected-error>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="scan">
- <test-case FilePath="scan">
- <compilation-unit name="10">
- <output-dir compare="Text">10</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="scan">
- <compilation-unit name="20">
- <output-dir compare="Text">20</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="scan">
- <compilation-unit name="issue238_query_1">
- <output-dir compare="Text">issue238_query_1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="scan">
- <compilation-unit name="issue238_query_2">
- <output-dir compare="Text">issue238_query_2</output-dir>
- </compilation-unit>
- </test-case>
- <!-- The syntax this test case tets isn't really invalid
- <test-case FilePath="scan">
- <compilation-unit name="invalid-scan-syntax">
- <output-dir compare="Text">invalid-scan-syntax</output-dir>
- <expected-error>java.lang.IllegalStateException: no result file</expected-error>
- </compilation-unit>
- </test-case>-->
- <test-case FilePath="scan">
- <compilation-unit name="30">
- <output-dir compare="Text">30</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="scan">
- <compilation-unit name="alltypes_01">
- <output-dir compare="Text">alltypes_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="scan">
- <compilation-unit name="alltypes_01">
- <output-dir compare="Clean-JSON">alltypes_01-cleanjson</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="scan">
- <compilation-unit name="alltypes_01">
- <output-dir compare="Lossless-JSON">alltypes_01-losslessjson</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="scan">
- <compilation-unit name="alltypes_01">
- <output-dir compare="Lossless-ADM-JSON">alltypes_01-losslessadmjson</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="scan">
- <compilation-unit name="alltypes_02">
- <output-dir compare="Text">alltypes_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="scan">
- <compilation-unit name="numeric_types_01">
- <output-dir compare="Text">numeric_types_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="scan">
- <compilation-unit name="spatial_types_01">
- <output-dir compare="Text">spatial_types_01</output-dir>
- </compilation-unit>
- </test-case>
- <!--
- <test-case FilePath="scan">
- <compilation-unit name="spatial_types_02">
- <output-dir compare="Text">spatial_types_02</output-dir>
- </compilation-unit>
- </test-case>
- -->
- <test-case FilePath="scan">
- <compilation-unit name="temp_types_01">
- <output-dir compare="Text">temp_types_01</output-dir>
- </compilation-unit>
- </test-case>
- <!--
- <test-case FilePath="scan">
- <compilation-unit name="temp_types_02">
- <output-dir compare="Text">temp_types_02</output-dir>
- </compilation-unit>
- </test-case>
- -->
- </test-group>
- <test-group name="select-star">
- <test-case FilePath="select-star">
- <compilation-unit name="group_by">
- <output-dir compare="Text">group_by</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="select-star">
- <compilation-unit name="from">
- <output-dir compare="Text">from</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="select-star">
- <compilation-unit name="join">
- <output-dir compare="Text">join</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="select-star">
- <compilation-unit name="mixed">
- <output-dir compare="Text">mixed</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="select-star">
- <compilation-unit name="no_star">
- <output-dir compare="Text">no_star</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="select-star">
- <compilation-unit name="var_star">
- <output-dir compare="Text">var_star</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="select-star">
- <compilation-unit name="var_star_2">
- <output-dir compare="Text">var_star_2</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="semistructured">
- <test-case FilePath="semistructured">
- <compilation-unit name="count-nullable">
- <output-dir compare="Text">count-nullable</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="semistructured">
- <compilation-unit name="cust-filter">
- <output-dir compare="Text">cust-filter</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="semistructured">
- <compilation-unit name="has-param1">
- <output-dir compare="Text">has-param1</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="similarity">
- <test-case FilePath="similarity">
- <compilation-unit name="edit-distance-check_ints">
- <output-dir compare="Text">edit-distance-check_ints</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="similarity">
- <compilation-unit name="edit-distance-check_strings">
- <output-dir compare="Text">edit-distance-check_strings</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="similarity">
- <compilation-unit name="edit-distance-check_unicode">
- <output-dir compare="Text">edit-distance-check_unicode</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="similarity">
- <compilation-unit name="edit-distance-list-is-filterable">
- <output-dir compare="Text">edit-distance-list-is-filterable</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="similarity">
- <compilation-unit name="edit-distance-string-is-filterable">
- <output-dir compare="Text">edit-distance-string-is-filterable</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="similarity">
- <compilation-unit name="edit-distance_ints">
- <output-dir compare="Text">edit-distance_ints</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="similarity">
- <compilation-unit name="edit-distance_strings">
- <output-dir compare="Text">edit-distance_strings</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="similarity">
- <compilation-unit name="fuzzyeq-edit-distance">
- <output-dir compare="Text">fuzzyeq-edit-distance</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="similarity">
- <compilation-unit name="fuzzyeq-similarity-jaccard">
- <output-dir compare="Text">fuzzyeq-similarity-jaccard</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="similarity">
- <compilation-unit name="prefix-len-jaccard">
- <output-dir compare="Text">prefix-len-jaccard</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="similarity">
- <compilation-unit name="similarity-jaccard-check_ints">
- <output-dir compare="Text">similarity-jaccard-check_ints</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="similarity">
- <compilation-unit name="similarity-jaccard-check_query">
- <output-dir compare="Text">similarity-jaccard-check_query</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="similarity">
- <compilation-unit name="similarity-jaccard-check_strings">
- <output-dir compare="Text">similarity-jaccard-check_strings</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="similarity">
- <compilation-unit name="similarity-jaccard-prefix-check">
- <output-dir compare="Text">similarity-jaccard-prefix-check</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="similarity">
- <compilation-unit name="similarity-jaccard-prefix">
- <output-dir compare="Text">similarity-jaccard-prefix</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="similarity">
- <compilation-unit name="similarity-jaccard-sorted-check_ints">
- <output-dir compare="Text">similarity-jaccard-sorted-check_ints</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="similarity">
- <compilation-unit name="similarity-jaccard-sorted-check_query">
- <output-dir compare="Text">similarity-jaccard-sorted-check_query</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="similarity">
- <compilation-unit name="similarity-jaccard-sorted-check_strings">
- <output-dir compare="Text">similarity-jaccard-sorted-check_strings</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="similarity">
- <compilation-unit name="similarity-jaccard-sorted_ints">
- <output-dir compare="Text">similarity-jaccard-sorted_ints</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="similarity">
- <compilation-unit name="similarity-jaccard-sorted_query">
- <output-dir compare="Text">similarity-jaccard-sorted_query</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="similarity">
- <compilation-unit name="similarity-jaccard-sorted_strings">
- <output-dir compare="Text">similarity-jaccard-sorted_strings</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="similarity">
- <compilation-unit name="similarity-jaccard_ints">
- <output-dir compare="Text">similarity-jaccard_ints</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="similarity">
- <compilation-unit name="similarity-jaccard_query">
- <output-dir compare="Text">similarity-jaccard_query</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="similarity">
- <compilation-unit name="similarity-jaccard_strings">
- <output-dir compare="Text">similarity-jaccard_strings</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="similarity">
- <compilation-unit name="similarity-jaccard-check_strings_issue628">
- <output-dir compare="Text">similarity-jaccard-check_strings_issue628</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="spatial">
- <test-case FilePath="spatial">
- <compilation-unit name="cell-aggregation-with-filtering">
- <output-dir compare="Text">cell-aggregation-with-filtering</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="spatial">
- <compilation-unit name="cell-aggregation">
- <output-dir compare="Text">cell-aggregation</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="spatial">
- <compilation-unit name="circle_accessor">
- <output-dir compare="Text">circle_accessor</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="spatial">
- <compilation-unit name="circle-intersect-circle">
- <output-dir compare="Text">circle-intersect-circle</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="spatial">
- <compilation-unit name="create-rtree-index">
- <output-dir compare="Text">create-rtree-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="spatial">
- <compilation-unit name="distance-between-points">
- <output-dir compare="Text">distance-between-points</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="spatial">
- <compilation-unit name="hints_spatial_partitioning">
- <output-dir compare="Text">hints_spatial_partitioning</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="spatial">
- <compilation-unit name="line_accessor">
- <output-dir compare="Text">line_accessor</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="spatial">
- <compilation-unit name="line-intersect-circle">
- <output-dir compare="Text">line-intersect-circle</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="spatial">
- <compilation-unit name="line-intersect-line">
- <output-dir compare="Text">line-intersect-line</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="spatial">
- <compilation-unit name="line-intersect-polygon">
- <output-dir compare="Text">line-intersect-polygon</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="spatial">
- <compilation-unit name="line-intersect-rectangle">
- <output-dir compare="Text">line-intersect-rectangle</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="spatial">
- <compilation-unit name="point_accessor">
- <output-dir compare="Text">point_accessor</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="spatial">
- <compilation-unit name="point-equals-point">
- <output-dir compare="Text">point-equals-point</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="spatial">
- <compilation-unit name="point-in-circle">
- <output-dir compare="Text">point-in-circle</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="spatial">
- <compilation-unit name="point-in-polygon">
- <output-dir compare="Text">point-in-polygon</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="spatial">
- <compilation-unit name="point-in-rectangle">
- <output-dir compare="Text">point-in-rectangle</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="spatial">
- <compilation-unit name="point-on-line">
- <output-dir compare="Text">point-on-line</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="spatial">
- <compilation-unit name="polygon_accessor">
- <output-dir compare="Text">polygon_accessor</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="spatial">
- <compilation-unit name="polygon-intersect-circle">
- <output-dir compare="Text">polygon-intersect-circle</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="spatial">
- <compilation-unit name="polygon-intersect-polygon">
- <output-dir compare="Text">polygon-intersect-polygon</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="spatial">
- <compilation-unit name="polygon-intersect-rectangle">
- <output-dir compare="Text">polygon-intersect-rectangle</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="spatial">
- <compilation-unit name="rectangle_accessor">
- <output-dir compare="Text">rectangle_accessor</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="spatial">
- <compilation-unit name="rectangle-intersect-circle">
- <output-dir compare="Text">rectangle-intersect-circle</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="spatial">
- <compilation-unit name="rectangle-intersect-rectangle">
- <output-dir compare="Text">rectangle-intersect-rectangle</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="spatial">
- <compilation-unit name="spatial-area">
- <output-dir compare="Text">spatial-area</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="spatial">
- <compilation-unit name="spatial-large-data">
- <output-dir compare="Text">spatial-large-data</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="spatial">
- <compilation-unit name="spatial_join_dynamic_partitioning">
- <output-dir compare="Text">spatial_join_dynamic_partitioning</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="spatial">
- <compilation-unit name="spatial_join_static_partitioning">
- <output-dir compare="Text">spatial_join_static_partitioning</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="spatial">
- <compilation-unit name="spatial_left_outer_join_st_intersects">
- <output-dir compare="Text">spatial_left_outer_join_st_intersects</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="spatial">
- <compilation-unit name="spatial_join_projection_check">
- <output-dir compare="Text">spatial_join_projection_check</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="sql-compat">
- <test-case FilePath="sql-compat">
- <compilation-unit name="in_non_list_01">
- <output-dir compare="Text">in_non_list_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="sql-compat">
- <compilation-unit name="outer_join_01">
- <output-dir compare="Text">outer_join_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="sql-compat">
- <compilation-unit name="outer_unnest_01">
- <output-dir compare="Text">outer_unnest_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="sql-compat">
- <compilation-unit name="select_star_01">
- <output-dir compare="Text">select_star_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="sql-compat">
- <compilation-unit name="select_star_02_negative">
- <output-dir compare="Text">none</output-dir>
- <expected-error>ASX1168: Ambiguous projection in SELECT clause (in line 32, at column 8)</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="sql-compat">
- <compilation-unit name="subquery_coercion_01_scalar">
- <output-dir compare="Text">subquery_coercion_01_scalar</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="sql-compat">
- <compilation-unit name="subquery_coercion_02_scalar_negative">
- <output-dir compare="Text">none</output-dir>
- <expected-error>ASX1169: Unable to do subquery coercion. Subquery returns more than one field (in line 32, at column 29)</expected-error>
- <expected-error>ASX1169: Unable to do subquery coercion. Unsupported projection kind (in line 32, at column 20)</expected-error>
- <expected-error>ASX1169: Unable to do subquery coercion. Both SELECT and SELECT VALUE are present (in line 35, at column 4)</expected-error>
- <expected-error>ASX1169: Unable to do subquery coercion. (in line 34, at column 3)</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="sql-compat">
- <compilation-unit name="subquery_coercion_03_cmp">
- <output-dir compare="Text">subquery_coercion_03_cmp</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="sql-compat">
- <compilation-unit name="subquery_coercion_04_cmp_negative">
- <output-dir compare="Text">none</output-dir>
- <expected-error>ASX1169: Unable to do subquery coercion. Subquery returns more than one field (in line 35, at column 26)</expected-error>
- <expected-error>ASX1169: Unable to do subquery coercion. Subquery returns more than one field (in line 35, at column 21)</expected-error>
- <expected-error>ASX1169: Unable to do subquery coercion. Both SELECT and SELECT VALUE are present (in line 37, at column 3)</expected-error>
- <expected-error>ASX1169: Unable to do subquery coercion. Both SELECT and SELECT VALUE are present (in line 38, at column 3)</expected-error>
- <expected-error>ASX1169: Unable to do subquery coercion. (in line 36, at column 3)</expected-error>
- <expected-error>ASX1169: Unable to do subquery coercion. Unsupported projection kind (in line 35, at column 20)</expected-error>
- <expected-error>ASX1169: Unable to do subquery coercion. Unsupported projection kind (in line 36, at column 11)</expected-error>
- <expected-error>ASX1169: Unable to do subquery coercion. Unsupported projection kind (in line 37, at column 11)</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="sql-compat">
- <compilation-unit name="subquery_coercion_05_in">
- <output-dir compare="Text">subquery_coercion_05_in</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="sql-compat">
- <compilation-unit name="subquery_coercion_06_in_negative">
- <output-dir compare="Text">none</output-dir>
- <expected-error>ASX1169: Unable to do subquery coercion. Subquery returns more than one field (in line 36, at column 27)</expected-error>
- <expected-error>ASX1169: Unable to do subquery coercion. Subquery returns more than one field (in line 37, at column 17)</expected-error>
- <expected-error>ASX1169: Unable to do subquery coercion. Both SELECT and SELECT VALUE are present (in line 38, at column 3)</expected-error>
- <expected-error>ASX1169: Unable to do subquery coercion. Both SELECT and SELECT VALUE are present (in line 39, at column 3)</expected-error>
- <expected-error>ASX1169: Unable to do subquery coercion. (in line 37, at column 3)</expected-error>
- <expected-error>ASX1169: Unable to do subquery coercion. Unsupported projection kind (in line 36, at column 21)</expected-error>
- <expected-error>ASX1169: Unable to do subquery coercion. Unsupported projection kind (in line 37, at column 25)</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="sql-compat">
- <compilation-unit name="subquery_coercion_07_from">
- <output-dir compare="Text">subquery_coercion_07_from</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="sql-compat">
- <compilation-unit name="subquery_coercion_08_misc">
- <output-dir compare="Text">subquery_coercion_08_misc</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="sql-compat">
- <compilation-unit name="union_all_01">
- <output-dir compare="Text">union_all_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="sql-compat">
- <compilation-unit name="union_all_02_negative">
- <output-dir compare="Text">none</output-dir>
- <expected-error>ASX1171: Unable to process UNION clause. Both SELECT and SELECT VALUE are present (in line 29, at column 1)</expected-error>
- <expected-error>ASX1171: Unable to process UNION clause. Unequal number of input fields (in line 29, at column 1)</expected-error>
- <expected-error>ASX1171: Unable to process UNION clause. Unsupported projection kind (in line 29, at column 8)</expected-error>
- <expected-error>ASX1171: Unable to process UNION clause. Unsupported projection kind (in line 30, at column 8)</expected-error>
- <expected-error>ASX1171: Unable to process UNION clause. Unsupported projection kind (in line 31, at column 8)</expected-error>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="statement-params">
- <test-case FilePath="statement-params">
- <compilation-unit name="index_01">
- <output-dir compare="Text">index_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="statement-params">
- <compilation-unit name="mixed_01">
- <output-dir compare="Text">mixed_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="statement-params">
- <compilation-unit name="named_01">
- <output-dir compare="Text">named_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="statement-params">
- <compilation-unit name="named_02">
- <output-dir compare="Text">named_01</output-dir>
- <expected-error>ASX1086: No value for parameter: $p2</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="statement-params">
- <compilation-unit name="named_03">
- <output-dir compare="Text">named_03</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="statement-params">
- <compilation-unit name="named_04">
- <output-dir compare="Text">named_04</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="statement-params">
- <compilation-unit name="positional_01">
- <output-dir compare="Text">positional_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="statement-params">
- <compilation-unit name="positional_02">
- <output-dir compare="Text">positional_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="statement-params">
- <compilation-unit name="positional_03">
- <output-dir compare="Text">positional_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="statement-params">
- <compilation-unit name="positional_04">
- <output-dir compare="Text">positional_02</output-dir>
- <expected-error>ASX1086: No value for parameter: $2</expected-error>
- <expected-error>ASX1086: No value for parameter: $3</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="statement-params">
- <compilation-unit name="positional_05">
- <output-dir compare="Text">positional_05</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="statement-params">
- <compilation-unit name="query-ASTERIXDB-2413">
- <output-dir compare="Text">query-ASTERIXDB-2413</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="string">
- <test-case FilePath="string">
- <compilation-unit name="codepoint-to-string1">
- <output-dir compare="Text">codepoint-to-string1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="codepoint-to-string2">
- <output-dir compare="Text">codepoint-to-string2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="concat/001">
- <output-dir compare="Text">concat/001</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="concat/002">
- <output-dir compare="Text">concat/002</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string" check-warnings="true">
- <compilation-unit name="concat/003">
- <output-dir compare="Text">concat/003</output-dir>
- <expected-warn>Type mismatch: function string-concat expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 26, at column 1)</expected-warn>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="concat/004">
- <output-dir compare="Text">concat/004</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="concat/005">
- <output-dir compare="Text">concat/005</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="concat/006">
- <output-dir compare="Text">concat/006</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="concat/007">
- <output-dir compare="Text">concat/007</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="concat/008">
- <output-dir compare="Text">concat/002</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="concat/concat_pipe">
- <output-dir compare="Text">concat/002</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="concat/concat_pipe_multi">
- <output-dir compare="Text">concat/concat_pipe_multi</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="constructor">
- <output-dir compare="Text">constructor</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="contains_01">
- <output-dir compare="Text">contains_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="cpttostr01">
- <output-dir compare="Text">cpttostr01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="cpttostr02">
- <output-dir compare="Text">cpttostr02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="cpttostr04">
- <output-dir compare="Text">cpttostr04</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="ends-with1">
- <output-dir compare="Text">ends-with1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="ends-with2">
- <output-dir compare="Text">ends-with2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="ends-with3">
- <output-dir compare="Text">ends-with3</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="ends-with4">
- <output-dir compare="Text">ends-with4</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="ends-with5">
- <output-dir compare="Text">ends-with5</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="escapes01">
- <output-dir compare="Text">escapes01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="escapes02">
- <output-dir compare="Text">escapes02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="initcap">
- <output-dir compare="Text">initcap</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="join/000">
- <output-dir compare="Text">join/000</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string" check-warnings="true">
- <compilation-unit name="join/001">
- <output-dir compare="Text">join/001</output-dir>
- <expected-warn>Type mismatch: function string-join expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 24, at column 15)</expected-warn>
- <expected-warn>Type mismatch: function string-join expects its 1st input parameter to be of type array, but the actual input type is bigint (in line 24, at column 15)</expected-warn>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="join/002">
- <output-dir compare="Text">join/002</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string" check-warnings="true">
- <compilation-unit name="join/003">
- <output-dir compare="Text">join/003</output-dir>
- <expected-warn>Type mismatch: function string-join expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 23, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function string-join expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 30, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function string-join expects its 1st input parameter to be of type array, but the actual input type is bigint (in line 33, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function string-join expects its 1st input parameter to be of type array, but the actual input type is bigint (in line 26, at column 1)</expected-warn>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string" check-warnings="true">
- <compilation-unit name="join/004">
- <output-dir compare="Text">join/004</output-dir>
- <expected-warn>Type mismatch: function string-join expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 26, at column 1)</expected-warn>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string" check-warnings="true">
- <compilation-unit name="join/005">
- <output-dir compare="Text">join/005</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="length_01">
- <output-dir compare="Text">length_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="length_02">
- <output-dir compare="Text">length_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="length_multi_code_point_01">
- <output-dir compare="Text">length_multi_code_point_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="like_01">
- <output-dir compare="Text">like_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="like_02">
- <output-dir compare="Text">like_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="like_03_negative">
- <output-dir compare="Text">like_03_negative</output-dir>
- <expected-error>Invalid pattern '__\c' for LIKE (in line 21, at column 11)</expected-error>
- <expected-error>Invalid pattern '%\' for LIKE (in line 21, at column 18)</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="like_null">
- <output-dir compare="Text">like_null</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="lowercase">
- <output-dir compare="Text">lowercase</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="matches02">
- <output-dir compare="Text">matches02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="matches03">
- <output-dir compare="Text">matches03</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="matches04">
- <output-dir compare="Text">matches04</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="matches05">
- <output-dir compare="Text">matches05</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="matches06">
- <output-dir compare="Text">matches06</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="matches1">
- <output-dir compare="Text">matches1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="matches11">
- <output-dir compare="Text">matches11</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="matches2">
- <output-dir compare="Text">matches2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="matches21">
- <output-dir compare="Text">matches21</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="matches22">
- <output-dir compare="Text">matches22</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="matches23">
- <output-dir compare="Text">matches23</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="matches3">
- <output-dir compare="Text">matches3</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="matchesnull">
- <output-dir compare="Text">matchesnull</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="position/offset0/position">
- <output-dir compare="Text">position/offset0/position</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="position/offset0/pos">
- <output-dir compare="Text">position/offset0/pos</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="position/offset0/position0">
- <output-dir compare="Text">position/offset0/position0</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="position/offset0/pos0">
- <output-dir compare="Text">position/offset0/pos0</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="position/offset0/pos0_multi_code_point">
- <output-dir compare="Text">position/offset0/pos0_multi_code_point</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="position/offset1/position1">
- <output-dir compare="Text">position/offset1/position1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="position/offset1/pos1">
- <output-dir compare="Text">position/offset1/pos1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="position/offset1/pos1_multi_code_point">
- <output-dir compare="Text">position/offset1/pos1_multi_code_point</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="regexp_contains/regexp_contains">
- <output-dir compare="Text">regexp_contains/regexp_contains</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="regexp_contains/regex_contains">
- <output-dir compare="Text">regexp_contains/regex_contains</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="regexp_contains/contains_regexp">
- <output-dir compare="Text">regexp_contains/contains_regexp</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="regexp_contains/contains_regex">
- <output-dir compare="Text">regexp_contains/contains_regex</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="regexp_contains_with_flag/regexp_contains_with_flag">
- <output-dir compare="Text">regexp_contains_with_flag/regexp_contains_with_flag</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="regexp_contains_with_flag/regex_contains_with_flag">
- <output-dir compare="Text">regexp_contains_with_flag/regex_contains_with_flag</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="regexp_contains_with_flag/contains_regexp_with_flag">
- <output-dir compare="Text">regexp_contains_with_flag/contains_regexp_with_flag</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="regexp_contains_with_flag/contains_regex_with_flag">
- <output-dir compare="Text">regexp_contains_with_flag/contains_regex_with_flag</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="regexp_like/regexp_like">
- <output-dir compare="Text">regexp_like/regexp_like</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="regexp_like/regex_like">
- <output-dir compare="Text">regexp_like/regex_like</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="regexp_like_with_flag/regexp_like_with_flag">
- <output-dir compare="Text">regexp_like_with_flag/regexp_like_with_flag</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="regexp_like_with_flag/regex_like_with_flag">
- <output-dir compare="Text">regexp_like_with_flag/regex_like_with_flag</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="regexp_matches/001">
- <output-dir compare="Text">regexp_matches/001</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="regexp_matches/002">
- <output-dir compare="Text">regexp_matches/002</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="regexp_matches/003">
- <output-dir compare="Text">regexp_matches/003</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="regexp_matches/004">
- <output-dir compare="Text">regexp_matches/004</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="regexp_position/offset0/regexp_position">
- <output-dir compare="Text">regexp_position/offset0/regexp_position</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="regexp_position/offset0/regexp_pos">
- <output-dir compare="Text">regexp_position/offset0/regexp_pos</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="regexp_position/offset0/regexp_position0">
- <output-dir compare="Text">regexp_position/offset0/regexp_position0</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="regexp_position/offset0/regexp_pos0">
- <output-dir compare="Text">regexp_position/offset0/regexp_pos0</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="regexp_position/offset0/regex_position">
- <output-dir compare="Text">regexp_position/offset0/regex_position</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="regexp_position/offset0/regex_pos">
- <output-dir compare="Text">regexp_position/offset0/regex_pos</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="regexp_position/offset0/regex_position0">
- <output-dir compare="Text">regexp_position/offset0/regex_position0</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="regexp_position/offset0/regex_pos0">
- <output-dir compare="Text">regexp_position/offset0/regex_pos0</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="regexp_position/offset1/regexp_position1">
- <output-dir compare="Text">regexp_position/offset1/regexp_position1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="regexp_position/offset1/regexp_pos1">
- <output-dir compare="Text">regexp_position/offset1/regexp_pos1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="regexp_position/offset1/regex_position1">
- <output-dir compare="Text">regexp_position/offset1/regex_position1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="regexp_position/offset1/regex_pos1">
- <output-dir compare="Text">regexp_position/offset1/regex_pos1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="regexp_position_with_flag/offset0/regexp_position_with_flag">
- <output-dir compare="Text">regexp_position_with_flag/offset0/regexp_position_with_flag</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="regexp_position_with_flag/offset0/regexp_pos_with_flag">
- <output-dir compare="Text">regexp_position_with_flag/offset0/regexp_pos_with_flag</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="regexp_position_with_flag/offset0/regexp_position0_with_flag">
- <output-dir compare="Text">regexp_position_with_flag/offset0/regexp_position0_with_flag</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="regexp_position_with_flag/offset0/regexp_pos0_with_flag">
- <output-dir compare="Text">regexp_position_with_flag/offset0/regexp_pos0_with_flag</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="regexp_position_with_flag/offset0/regex_position_with_flag">
- <output-dir compare="Text">regexp_position_with_flag/offset0/regex_position_with_flag</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="regexp_position_with_flag/offset0/regex_pos_with_flag">
- <output-dir compare="Text">regexp_position_with_flag/offset0/regex_pos_with_flag</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="substr-ASTERIXDB-2949">
- <output-dir compare="Text">substr-ASTERIXDB-2949</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="regexp_position_with_flag/offset0/regex_position0_with_flag">
- <output-dir compare="Text">regexp_position_with_flag/offset0/regex_position0_with_flag</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="regexp_position_with_flag/offset0/regex_pos0_with_flag">
- <output-dir compare="Text">regexp_position_with_flag/offset0/regex_pos0_with_flag</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="regexp_position_with_flag/offset1/regexp_position1_with_flag">
- <output-dir compare="Text">regexp_position_with_flag/offset1/regexp_position1_with_flag</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="regexp_position_with_flag/offset1/regexp_pos1_with_flag">
- <output-dir compare="Text">regexp_position_with_flag/offset1/regexp_pos1_with_flag</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="regexp_position_with_flag/offset1/regex_position1_with_flag">
- <output-dir compare="Text">regexp_position_with_flag/offset1/regex_position1_with_flag</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="regexp_position_with_flag/offset1/regex_pos1_with_flag">
- <output-dir compare="Text">regexp_position_with_flag/offset1/regex_pos1_with_flag</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="regexp_split/001">
- <output-dir compare="Text">regexp_split/001</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="regexp_split/002">
- <output-dir compare="Text">regexp_split/002</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="regexp_split/003">
- <output-dir compare="Text">regexp_split/003</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="regexp_split/004">
- <output-dir compare="Text">regexp_split/004</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="repeat">
- <output-dir compare="Text">repeat</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="replace">
- <output-dir compare="Text">replace</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="replace_with_limit">
- <output-dir compare="Text">replace_with_limit</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="regexp_replace/regexp_replace">
- <output-dir compare="Text">regexp_replace/regexp_replace</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="regexp_replace/regex_replace">
- <output-dir compare="Text">regexp_replace/regex_replace</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="regexp_replace_with_flag/regexp_replace_with_flag">
- <output-dir compare="Text">regexp_replace_with_flag/regexp_replace_with_flag</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="regexp_replace_with_flag/regex_replace_with_flag">
- <output-dir compare="Text">regexp_replace_with_flag/regex_replace_with_flag</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="reverse">
- <output-dir compare="Text">reverse</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="split">
- <output-dir compare="Text">split</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="starts-with1">
- <output-dir compare="Text">starts-with1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="starts-with2">
- <output-dir compare="Text">starts-with2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="starts-with3">
- <output-dir compare="Text">starts-with3</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="starts-with4">
- <output-dir compare="Text">starts-with4</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="starts-with5">
- <output-dir compare="Text">starts-with5</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="string-equal1">
- <output-dir compare="Text">string-equal1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="string-equal2">
- <output-dir compare="Text">string-equal2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="string-equal3">
- <output-dir compare="Text">string-equal3</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="string-equal4">
- <output-dir compare="Text">string-equal4</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="string-literal1">
- <output-dir compare="Text">string-literal1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="string-to-codepoint">
- <output-dir compare="Text">string-to-codepoint</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="string-to-codepoint1">
- <output-dir compare="Text">string-to-codepoint1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="string-to-codepoint2">
- <output-dir compare="Text">string-to-codepoint2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="string_to_codepoint_multi_codepoints_01">
- <output-dir compare="Text">string_to_codepoint_multi_codepoints_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="strlen02">
- <output-dir compare="Text">strlen02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="strlen03">
- <output-dir compare="Text">strlen03</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="strtocpt01">
- <output-dir compare="Text">strtocpt01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="strtocpt02">
- <output-dir compare="Text">strtocpt02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="strtocpt03">
- <output-dir compare="Text">strtocpt03</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="substr01/offset0/substring">
- <output-dir compare="Text">substr01/offset0/substring</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="substr01/offset0/substr">
- <output-dir compare="Text">substr01/offset0/substr</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="substr01/offset0/substring0">
- <output-dir compare="Text">substr01/offset0/substring0</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="substr01/offset0/substr0">
- <output-dir compare="Text">substr01/offset0/substr0</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="substr01/offset1/substring1">
- <output-dir compare="Text">substr01/offset1/substring1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="substr01/offset1/substr1">
- <output-dir compare="Text">substr01/offset1/substr1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="substr04">
- <output-dir compare="Text">substr04</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="substr05">
- <output-dir compare="Text">substr05</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="substr06">
- <output-dir compare="Text">substr06</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="substring_multi_codepoint_01">
- <output-dir compare="Text">substring_multi_codepoint_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="substring-after-1">
- <output-dir compare="Text">substring-after-1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="substring-after-2">
- <output-dir compare="Text">substring-after-2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="substring-after-3">
- <output-dir compare="Text">substring-after-3</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="substring-after-4">
- <output-dir compare="Text">substring-after-4</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="substring-before-1">
- <output-dir compare="Text">substring-before-1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="substring-before-2">
- <output-dir compare="Text">substring-before-2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="substring-before-3">
- <output-dir compare="Text">substring-before-3</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="substring2-1/offset0/substring">
- <output-dir compare="Text">substring2-1/offset0/substring</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="substring2-1/offset0/substr">
- <output-dir compare="Text">substring2-1/offset0/substr</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="substring2-1/offset0/substring0">
- <output-dir compare="Text">substring2-1/offset0/substring0</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="substring2-1/offset0/substr0">
- <output-dir compare="Text">substring2-1/offset0/substr0</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="substring2-1/offset1/substring1">
- <output-dir compare="Text">substring2-1/offset1/substring1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="substring2-1/offset1/substr1">
- <output-dir compare="Text">substring2-1/offset1/substr1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="substring2-2">
- <output-dir compare="Text">substring2-2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="substring2-3">
- <output-dir compare="Text">substring2-3</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="substring2-4">
- <output-dir compare="Text">substring2-4</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="substring_01">
- <output-dir compare="Text">substring_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="title">
- <output-dir compare="Text">initcap</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="toLowerCase02">
- <output-dir compare="Text">toLowerCase02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="toLowerCase03">
- <output-dir compare="Text">toLowerCase03</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="toLowerCase04">
- <output-dir compare="Text">toLowerCase04</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="trim">
- <output-dir compare="Text">trim</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="ltrim">
- <output-dir compare="Text">ltrim</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="rtrim">
- <output-dir compare="Text">rtrim</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="uppercase">
- <output-dir compare="Text">uppercase</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="varlen-encoding">
- <output-dir compare="Text">varlen-encoding</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="utf8">
- <output-dir compare="Text">utf8</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="query-ASTERIXDB-1190">
- <output-dir compare="Text">query-ASTERIXDB-1190</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="string-equal-public">
- <output-dir compare="Text">string-equal-public</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="substring-after-5">
- <output-dir compare="Text">substring-after-5</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="string">
- <compilation-unit name="substring-after-6">
- <output-dir compare="Text">substring-after-5</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="subquery">
- <test-case FilePath="subquery">
- <compilation-unit name="aggregate_join">
- <output-dir compare="Text">aggregate_join</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="subquery">
- <compilation-unit name="aggregate_join_external">
- <output-dir compare="Text">aggregate_join_external</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="subquery">
- <compilation-unit name="gby_inline">
- <output-dir compare="Text">gby_inline</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="subquery">
- <compilation-unit name="query-ASTERIXDB-1168">
- <output-dir compare="Text">query-ASTERIXDB-1168</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="subquery">
- <compilation-unit name="decorrelate_with_unique_id">
- <output-dir compare="Text">decorrelate_with_unique_id</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="subquery">
- <compilation-unit name="exists">
- <output-dir compare="Text">exists</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="subquery">
- <compilation-unit name="in">
- <output-dir compare="Text">in</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="subquery">
- <compilation-unit name="in_as_or">
- <output-dir compare="Text">in_as_or</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="subquery">
- <compilation-unit name="in_correlated">
- <output-dir compare="Text">in</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="subquery">
- <compilation-unit name="in_let">
- <output-dir compare="Text">in_let</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="subquery">
- <compilation-unit name="not_exists">
- <output-dir compare="Text">not_exists</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="subquery">
- <compilation-unit name="not_in">
- <output-dir compare="Text">not_in</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="subquery">
- <compilation-unit name="alias_negative">
- <output-dir compare="Text">alias_negative</output-dir>
- <expected-error>Need an alias for the enclosed expression</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="subquery">
- <compilation-unit name="relational_division">
- <output-dir compare="Text">relational_division</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="subquery">
- <compilation-unit name="relational_division2">
- <output-dir compare="Text">relational_division</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="subquery">
- <compilation-unit name="relational_division3">
- <output-dir compare="Text">relational_division</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="subquery">
- <compilation-unit name="non_unary_subplan_01">
- <output-dir compare="Text">non_unary_subplan_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="subquery">
- <compilation-unit name="query-ASTERIXDB-1571">
- <output-dir compare="Text">query-ASTERIXDB-1571</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="subquery">
- <compilation-unit name="query-ASTERIXDB-1571-2">
- <output-dir compare="Text">query-ASTERIXDB-1571</output-dir>
- <expected-error>Need an alias for the enclosed expression</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="subquery">
- <compilation-unit name="query-ASTERIXDB-1571-3">
- <output-dir compare="Text">query-ASTERIXDB-1571-3</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="subquery">
- <compilation-unit name="query-ASTERIXDB-1571-4">
- <output-dir compare="Text">query-ASTERIXDB-1571-4</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="subquery">
- <compilation-unit name="query-ASTERIXDB-1572">
- <output-dir compare="Text">query-ASTERIXDB-1572</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="subquery">
- <compilation-unit name="query-ASTERIXDB-1574">
- <output-dir compare="Text">query-ASTERIXDB-1574</output-dir>
- <expected-error>ASX1091: Type mismatch: expected value of type multiset or array, but got the value of type object (in line 25, at column 41)</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="subquery">
- <compilation-unit name="query-ASTERIXDB-1574-2">
- <output-dir compare="Text">query-ASTERIXDB-1574</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="subquery">
- <compilation-unit name="query-ASTERIXDB-1574-3">
- <output-dir compare="Text">query-ASTERIXDB-1574</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="subquery">
- <compilation-unit name="query-ASTERIXDB-1597">
- <output-dir compare="Text">query-ASTERIXDB-1597</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="subquery">
- <compilation-unit name="query-ASTERIXDB-1674">
- <output-dir compare="Text">query-ASTERIXDB-1674</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="subquery">
- <compilation-unit name="query-ASTERIXDB-2815">
- <output-dir compare="Text">query-ASTERIXDB-2815</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="subquery">
- <compilation-unit name="query-ASTERIXDB-2845">
- <output-dir compare="Text">query-ASTERIXDB-2845</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="subquery">
- <compilation-unit name="query-ASTERIXDB-3006">
- <output-dir compare="Text">query-ASTERIXDB-3006</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="subset-collection">
- <test-case FilePath="subset-collection">
- <compilation-unit name="01">
- <output-dir compare="Text">01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="subset-collection">
- <compilation-unit name="02">
- <output-dir compare="Text">02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="subset-collection">
- <compilation-unit name="03">
- <output-dir compare="Text">03</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="subset-collection">
- <compilation-unit name="05">
- <output-dir compare="Text">05</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="subset-collection">
- <compilation-unit name="06">
- <output-dir compare="Text">06</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="subset-collection">
- <compilation-unit name="07">
- <output-dir compare="Text">07</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="synonym">
- <test-case FilePath="synonym">
- <compilation-unit name="synonym-01">
- <output-dir compare="Text">synonym-01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="synonym">
- <compilation-unit name="synonym-02-negative">
- <output-dir compare="Text">none</output-dir>
- <expected-error>ASX1063: Cannot find dataverse with name UNKNOWN_DATAVERSE</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="synonym">
- <compilation-unit name="synonym-03">
- <output-dir compare="Text">synonym-03</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="tokenizers">
- <test-case FilePath="tokenizers">
- <compilation-unit name="counthashed-gram-tokens_01">
- <output-dir compare="Text">counthashed-gram-tokens_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tokenizers">
- <compilation-unit name="counthashed-gram-tokens_02">
- <output-dir compare="Text">counthashed-gram-tokens_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tokenizers">
- <compilation-unit name="counthashed-word-tokens_01">
- <output-dir compare="Text">counthashed-word-tokens_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tokenizers">
- <compilation-unit name="gram-tokens_01">
- <output-dir compare="Text">gram-tokens_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tokenizers">
- <compilation-unit name="gram-tokens_02">
- <output-dir compare="Text">gram-tokens_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tokenizers">
- <compilation-unit name="hashed-gram-tokens_01">
- <output-dir compare="Text">hashed-gram-tokens_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tokenizers">
- <compilation-unit name="hashed-gram-tokens_02">
- <output-dir compare="Text">hashed-gram-tokens_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tokenizers">
- <compilation-unit name="hashed-word-tokens_01">
- <output-dir compare="Text">hashed-word-tokens_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tokenizers">
- <compilation-unit name="word-tokens_01">
- <output-dir compare="Text">word-tokens_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tokenizers">
- <compilation-unit name="word-tokens_02">
- <output-dir compare="Text">word-tokens_02</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="tpcds">
- <test-case FilePath="tpcds">
- <compilation-unit name="query-ASTERIXDB-1580">
- <output-dir compare="Text">query-ASTERIXDB-1580</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="query-ASTERIXDB-1581">
- <output-dir compare="Text">query-ASTERIXDB-1581</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="query-ASTERIXDB-1581-2">
- <output-dir compare="Text">query-ASTERIXDB-1581-2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="query-ASTERIXDB-1581-3">
- <output-dir compare="Text">query-ASTERIXDB-1581-2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="query-ASTERIXDB-1581-4">
- <output-dir compare="Text">query-ASTERIXDB-1581-4</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="query-ASTERIXDB-1581-5">
- <output-dir compare="Text">query-ASTERIXDB-1581-2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="query-ASTERIXDB-1581-6">
- <output-dir compare="Text">query-ASTERIXDB-1581-6</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="query-ASTERIXDB-1581-correlated">
- <output-dir compare="Text">query-ASTERIXDB-1581-correlated</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="query-ASTERIXDB-1581-correlated-2">
- <output-dir compare="Text">query-ASTERIXDB-1581-2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="query-ASTERIXDB-1591">
- <output-dir compare="Text">query-ASTERIXDB-1591</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="query-ASTERIXDB-1596">
- <output-dir compare="Text">query-ASTERIXDB-1596</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="query-ASTERIXDB-1602">
- <output-dir compare="Text">query-ASTERIXDB-1602</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q01">
- <output-dir compare="Text">q01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q03">
- <output-dir compare="Text">q03</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q05">
- <output-dir compare="Text">q05</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q07">
- <output-dir compare="Text">q07</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q09">
- <output-dir compare="Text">q09</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q12">
- <output-dir compare="Text">q12</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q15">
- <output-dir compare="Text">q15</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q18">
- <output-dir compare="Text">q18</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q19">
- <output-dir compare="Text">q19</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q20">
- <output-dir compare="Text">q20</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q21">
- <output-dir compare="Text">q21</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q22">
- <output-dir compare="Text">q22</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q24a">
- <output-dir compare="Text">q24a</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q24b">
- <output-dir compare="Text">q24b</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q25">
- <output-dir compare="Text">q25</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q26">
- <output-dir compare="Text">q26</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q27">
- <output-dir compare="Text">q27</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q29">
- <output-dir compare="Text">q29</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q30">
- <output-dir compare="Text">q30</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q31">
- <output-dir compare="Text">q31</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q34">
- <output-dir compare="Text">q34</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q36">
- <output-dir compare="Text">q36</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q37">
- <output-dir compare="Text">q37</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q41">
- <output-dir compare="Text">q41</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q42">
- <output-dir compare="Text">q42</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q43">
- <output-dir compare="Text">q43</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q45">
- <output-dir compare="Text">q45</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q46">
- <output-dir compare="Text">q46</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q49">
- <output-dir compare="Text">q49</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q50">
- <output-dir compare="Text">q50</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q52">
- <output-dir compare="Text">q52</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q55">
- <output-dir compare="Text">q55</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q57">
- <output-dir compare="Text">q57</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q59">
- <output-dir compare="Text">q59</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q62">
- <output-dir compare="Text">q62</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q63">
- <output-dir compare="Text">q63</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q67">
- <output-dir compare="Text">q67</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q68">
- <output-dir compare="Text">q68</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q70">
- <output-dir compare="Text">q70</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q73">
- <output-dir compare="Text">q73</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q77">
- <output-dir compare="Text">q77</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q79">
- <output-dir compare="Text">q79</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q80">
- <output-dir compare="Text">q80</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q81">
- <output-dir compare="Text">q81</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q82">
- <output-dir compare="Text">q82</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q85">
- <output-dir compare="Text">q85</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q86">
- <output-dir compare="Text">q86</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q88">
- <output-dir compare="Text">q88</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q89">
- <output-dir compare="Text">q89</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q90">
- <output-dir compare="Text">q90</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q91">
- <output-dir compare="Text">q91</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q92">
- <output-dir compare="Text">q92</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q94">
- <output-dir compare="Text">q94</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q95">
- <output-dir compare="Text">q95</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q96">
- <output-dir compare="Text">q96</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpcds">
- <compilation-unit name="q98">
- <output-dir compare="Text">q98</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="tpch">
- <test-case FilePath="tpch">
- <compilation-unit name="distinct_by">
- <output-dir compare="Text">distinct_by</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch">
- <compilation-unit name="group_no_agg">
- <output-dir compare="Text">group_no_agg</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch">
- <compilation-unit name="nest_aggregate">
- <output-dir compare="Text">nest_aggregate</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch">
- <compilation-unit name="nest_aggregate2">
- <output-dir compare="Text">nest_aggregate2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch">
- <compilation-unit name="query-issue638">
- <output-dir compare="Text">query-issue638</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch">
- <compilation-unit name="query-issue785">
- <output-dir compare="Text">query-issue785</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch">
- <compilation-unit name="query-issue785-2">
- <output-dir compare="Text">query-issue785-2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch">
- <compilation-unit name="query-issue786">
- <output-dir compare="Text">query-issue786</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch">
- <compilation-unit name="query-issue601">
- <output-dir compare="Text">query-issue601</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch">
- <compilation-unit name="q10_returned_item">
- <output-dir compare="Text">q10_returned_item</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch">
- <compilation-unit name="q10_returned_item_int64">
- <output-dir compare="Text">q10_returned_item_int64</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch">
- <compilation-unit name="q11_important_stock">
- <output-dir compare="Text">q11_important_stock</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch">
- <compilation-unit name="q12_shipping">
- <output-dir compare="Text">q12_shipping</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch">
- <compilation-unit name="q13_customer_distribution">
- <output-dir compare="Text">q13_customer_distribution</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch">
- <compilation-unit name="q14_promotion_effect">
- <output-dir compare="Text">q14_promotion_effect</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch">
- <compilation-unit name="q15_top_supplier">
- <output-dir compare="Text">q15_top_supplier</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch">
- <compilation-unit name="q16_parts_supplier_relationship">
- <output-dir compare="Text">q16_parts_supplier_relationship</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch">
- <compilation-unit name="q17_small_quantity_order_revenue">
- <output-dir compare="Text">q17_small_quantity_order_revenue</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch">
- <compilation-unit name="q17_large_gby_variant">
- <output-dir compare="Text">q17_large_gby_variant</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch">
- <compilation-unit name="q18_large_volume_customer">
- <output-dir compare="Text">q18_large_volume_customer</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch">
- <compilation-unit name="q19_discounted_revenue">
- <output-dir compare="Text">q19_discounted_revenue</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch">
- <compilation-unit name="q01_pricing_summary_report_nt">
- <output-dir compare="Text">q01_pricing_summary_report_nt</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch">
- <compilation-unit name="q20_potential_part_promotion">
- <output-dir compare="Text">q20_potential_part_promotion</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch">
- <compilation-unit name="q21_suppliers_who_kept_orders_waiting">
- <output-dir compare="Text">q21_suppliers_who_kept_orders_waiting</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch">
- <compilation-unit name="q22_global_sales_opportunity">
- <output-dir compare="Text">q22_global_sales_opportunity</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch">
- <compilation-unit name="q02_minimum_cost_supplier">
- <output-dir compare="Text">q02_minimum_cost_supplier</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch">
- <compilation-unit name="q03_shipping_priority_nt">
- <output-dir compare="Text">q03_shipping_priority_nt</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch">
- <compilation-unit name="q04_order_priority">
- <output-dir compare="Text">q04_order_priority</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch">
- <compilation-unit name="q05_local_supplier_volume">
- <output-dir compare="Text">q05_local_supplier_volume</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch">
- <compilation-unit name="q06_forecast_revenue_change">
- <output-dir compare="Text">q06_forecast_revenue_change</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch">
- <compilation-unit name="q07_volume_shipping">
- <output-dir compare="Text">q07_volume_shipping</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch">
- <compilation-unit name="q08_national_market_share">
- <output-dir compare="Text">q08_national_market_share</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch">
- <compilation-unit name="q09_product_type_profit_nt">
- <output-dir compare="Text">q09_product_type_profit_nt</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch">
- <compilation-unit name="query-issue562">
- <output-dir compare="Text">query-issue562</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch">
- <compilation-unit name="query-issue810">
- <output-dir compare="Text">query-issue810</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch">
- <compilation-unit name="query-issue810-2">
- <output-dir compare="Text">query-issue810-2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch">
- <compilation-unit name="query-issue810-3">
- <output-dir compare="Text">query-issue810-3</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch">
- <compilation-unit name="query-issue827">
- <output-dir compare="Text">query-issue827</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch">
- <compilation-unit name="query-issue827-2">
- <output-dir compare="Text">query-issue827-2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch">
- <compilation-unit name="q01-ASTERIXDB-830">
- <output-dir compare="Text">q01_pricing_summary_report_nt</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="tpch-sql">
- <test-case FilePath="tpch-sql">
- <compilation-unit name="distinct_by">
- <output-dir compare="Text">distinct_by</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql">
- <compilation-unit name="group_no_agg">
- <output-dir compare="Text">group_no_agg</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql">
- <compilation-unit name="nest_aggregate">
- <output-dir compare="Text">nest_aggregate</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql">
- <compilation-unit name="nest_aggregate2">
- <output-dir compare="Text">nest_aggregate2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql">
- <compilation-unit name="query-issue638">
- <output-dir compare="Text">query-issue638</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql">
- <compilation-unit name="query-issue785">
- <output-dir compare="Text">query-issue785</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql">
- <compilation-unit name="query-issue785-2">
- <output-dir compare="Text">query-issue785-2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql">
- <compilation-unit name="query-issue786">
- <output-dir compare="Text">query-issue786</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql">
- <compilation-unit name="query-issue601">
- <output-dir compare="Text">query-issue601</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql">
- <compilation-unit name="q10_returned_item">
- <output-dir compare="Text">q10_returned_item</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql">
- <compilation-unit name="q10_returned_item_int64">
- <output-dir compare="Text">q10_returned_item_int64</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql">
- <compilation-unit name="q11_important_stock">
- <output-dir compare="Text">q11_important_stock</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql">
- <compilation-unit name="q12_shipping">
- <output-dir compare="Text">q12_shipping</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql">
- <compilation-unit name="q13_customer_distribution">
- <output-dir compare="Text">q13_customer_distribution</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql">
- <compilation-unit name="q14_promotion_effect">
- <output-dir compare="Text">q14_promotion_effect</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql">
- <compilation-unit name="q15_top_supplier">
- <output-dir compare="Text">q15_top_supplier</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql">
- <compilation-unit name="q16_parts_supplier_relationship">
- <output-dir compare="Text">q16_parts_supplier_relationship</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql">
- <compilation-unit name="q17_small_quantity_order_revenue">
- <output-dir compare="Text">q17_small_quantity_order_revenue</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql">
- <compilation-unit name="q17_large_gby_variant">
- <output-dir compare="Text">q17_large_gby_variant</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql">
- <compilation-unit name="q18_large_volume_customer">
- <output-dir compare="Text">q18_large_volume_customer</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql">
- <compilation-unit name="q19_discounted_revenue">
- <output-dir compare="Text">q19_discounted_revenue</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql">
- <compilation-unit name="q01_pricing_summary_report_nt">
- <output-dir compare="Text">q01_pricing_summary_report_nt</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql">
- <compilation-unit name="q20_potential_part_promotion">
- <output-dir compare="Text">q20_potential_part_promotion</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql">
- <compilation-unit name="q21_suppliers_who_kept_orders_waiting">
- <output-dir compare="Text">q21_suppliers_who_kept_orders_waiting</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql">
- <compilation-unit name="q22_global_sales_opportunity">
- <output-dir compare="Text">q22_global_sales_opportunity</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql">
- <compilation-unit name="q02_minimum_cost_supplier">
- <output-dir compare="Text">q02_minimum_cost_supplier</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql">
- <compilation-unit name="q03_shipping_priority_nt">
- <output-dir compare="Text">q03_shipping_priority_nt</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql">
- <compilation-unit name="q04_order_priority">
- <output-dir compare="Text">q04_order_priority</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql">
- <compilation-unit name="q05_local_supplier_volume">
- <output-dir compare="Text">q05_local_supplier_volume</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql">
- <compilation-unit name="q06_forecast_revenue_change">
- <output-dir compare="Text">q06_forecast_revenue_change</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql">
- <compilation-unit name="q07_volume_shipping">
- <output-dir compare="Text">q07_volume_shipping</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql">
- <compilation-unit name="q08_national_market_share">
- <output-dir compare="Text">q08_national_market_share</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql">
- <compilation-unit name="q09_product_type_profit_nt">
- <output-dir compare="Text">q09_product_type_profit_nt</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql">
- <compilation-unit name="query-issue562">
- <output-dir compare="Text">query-issue562</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql">
- <compilation-unit name="query-issue810">
- <output-dir compare="Text">query-issue810</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql">
- <compilation-unit name="query-issue810-2">
- <output-dir compare="Text">query-issue810-2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql">
- <compilation-unit name="query-issue810-3">
- <output-dir compare="Text">query-issue810-3</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="tpch-sql-sugar">
- <test-case FilePath="tpch-sql-sugar">
- <compilation-unit name="distinct_by">
- <output-dir compare="Text">distinct_by</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-sugar">
- <compilation-unit name="group_no_agg">
- <output-dir compare="Text">group_no_agg</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-sugar">
- <compilation-unit name="nest_aggregate">
- <output-dir compare="Text">nest_aggregate</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-sugar">
- <compilation-unit name="nest_aggregate2">
- <output-dir compare="Text">nest_aggregate2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-sugar">
- <compilation-unit name="query-issue638">
- <output-dir compare="Text">query-issue638</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-sugar">
- <compilation-unit name="query-issue785">
- <output-dir compare="Text">query-issue785</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-sugar">
- <compilation-unit name="query-issue785-2">
- <output-dir compare="Text">query-issue785-2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-sugar">
- <compilation-unit name="query-issue786">
- <output-dir compare="Text">query-issue786</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-sugar">
- <compilation-unit name="query-issue601">
- <output-dir compare="Text">query-issue601</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-sugar">
- <compilation-unit name="q10_returned_item">
- <output-dir compare="Text">q10_returned_item</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-sugar">
- <compilation-unit name="q10_returned_item_int64">
- <output-dir compare="Text">q10_returned_item_int64</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-sugar">
- <compilation-unit name="q11_important_stock">
- <output-dir compare="Text">q11_important_stock</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-sugar">
- <compilation-unit name="q12_shipping">
- <output-dir compare="Text">q12_shipping</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-sugar">
- <compilation-unit name="q12_shipping_broadcast">
- <output-dir compare="Text">q12_shipping</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-sugar">
- <compilation-unit name="q13_customer_distribution">
- <output-dir compare="Text">q13_customer_distribution</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-sugar">
- <compilation-unit name="q14_promotion_effect">
- <output-dir compare="Text">q14_promotion_effect</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-sugar">
- <compilation-unit name="q15_top_supplier">
- <output-dir compare="Text">q15_top_supplier</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-sugar">
- <compilation-unit name="q16_parts_supplier_relationship">
- <output-dir compare="Text">q16_parts_supplier_relationship</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-sugar">
- <compilation-unit name="q17_small_quantity_order_revenue">
- <output-dir compare="Text">q17_small_quantity_order_revenue</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-sugar">
- <compilation-unit name="q17_large_gby_variant">
- <output-dir compare="Text">q17_large_gby_variant</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-sugar">
- <compilation-unit name="q17_large_gby_variant_parameter">
- <output-dir compare="Text">none</output-dir>
- <expected-error>Invalid query parameter compiler.groupmemory -- value has to be greater than or equal to</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-sugar">
- <compilation-unit name="q18_large_volume_customer">
- <output-dir compare="Text">q18_large_volume_customer</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-sugar">
- <compilation-unit name="q19_discounted_revenue">
- <output-dir compare="Text">q19_discounted_revenue</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-sugar">
- <compilation-unit name="q01_pricing_summary_report_nt">
- <output-dir compare="Text">q01_pricing_summary_report_nt</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-sugar">
- <compilation-unit name="q01_pricing_summary_report_2">
- <output-dir compare="Text">q01_pricing_summary_report_nt</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-sugar">
- <compilation-unit name="q01_pricing_summary_report_parallelism">
- <output-dir compare="Text">q01_pricing_summary_report_nt</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-sugar">
- <compilation-unit name="q01_pricing_summary_report_parameter">
- <output-dir compare="Text">none</output-dir>
- <expected-error>Invalid query parameter compiler.sortmemory -- value has to be greater than or equal to</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-sugar">
- <compilation-unit name="q20_potential_part_promotion">
- <output-dir compare="Text">q20_potential_part_promotion</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-sugar">
- <compilation-unit name="q21_suppliers_who_kept_orders_waiting">
- <output-dir compare="Text">q21_suppliers_who_kept_orders_waiting</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-sugar">
- <compilation-unit name="q22_global_sales_opportunity">
- <output-dir compare="Text">q22_global_sales_opportunity</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-sugar">
- <compilation-unit name="q02_minimum_cost_supplier">
- <output-dir compare="Text">q02_minimum_cost_supplier</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-sugar">
- <compilation-unit name="q03_shipping_priority_nt">
- <output-dir compare="Text">q03_shipping_priority_nt</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-sugar">
- <compilation-unit name="q04_order_priority">
- <output-dir compare="Text">q04_order_priority</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-sugar">
- <compilation-unit name="q05_local_supplier_volume">
- <output-dir compare="Text">q05_local_supplier_volume</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-sugar">
- <compilation-unit name="q06_forecast_revenue_change">
- <output-dir compare="Text">q06_forecast_revenue_change</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-sugar">
- <compilation-unit name="q07_volume_shipping">
- <output-dir compare="Text">q07_volume_shipping</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-sugar">
- <compilation-unit name="q08_national_market_share">
- <output-dir compare="Text">q08_national_market_share</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-sugar">
- <compilation-unit name="q09_product_type_profit_nt">
- <output-dir compare="Text">q09_product_type_profit_nt</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-sugar">
- <compilation-unit name="q09_product_type_profit_parameter">
- <output-dir compare="Text">none</output-dir>
- <expected-error>Invalid query parameter compiler.joinmemory -- value has to be greater than or equal to</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-sugar">
- <compilation-unit name="query-issue562">
- <output-dir compare="Text">query-issue562</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-sugar">
- <compilation-unit name="query-issue810">
- <output-dir compare="Text">query-issue810</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-sugar">
- <compilation-unit name="query-issue810-2">
- <output-dir compare="Text">query-issue810-2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-sugar">
- <compilation-unit name="query-issue810-3">
- <output-dir compare="Text">query-issue810-3</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="tpch-with-index">
- <test-case FilePath="tpch-with-index">
- <compilation-unit name="distinct_by">
- <output-dir compare="Text">distinct_by</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-with-index">
- <compilation-unit name="nest_aggregate">
- <output-dir compare="Text">nest_aggregate</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-with-index">
- <compilation-unit name="nest_aggregate2">
- <output-dir compare="Text">nest_aggregate2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-with-index">
- <compilation-unit name="query-issue785">
- <output-dir compare="Text">query-issue785</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-with-index">
- <compilation-unit name="query-issue785-2">
- <output-dir compare="Text">query-issue785-2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-with-index">
- <compilation-unit name="query-issue786">
- <output-dir compare="Text">query-issue786</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-with-index">
- <compilation-unit name="query-issue601">
- <output-dir compare="Text">query-issue601</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-with-index">
- <compilation-unit name="q10_returned_item">
- <output-dir compare="Text">q10_returned_item</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-with-index">
- <compilation-unit name="q10_returned_item_int64">
- <output-dir compare="Text">q10_returned_item_int64</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-with-index">
- <compilation-unit name="q11_important_stock">
- <output-dir compare="Text">q11_important_stock</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-with-index">
- <compilation-unit name="q12_shipping">
- <output-dir compare="Text">q12_shipping</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-with-index">
- <compilation-unit name="q13_customer_distribution">
- <output-dir compare="Text">q13_customer_distribution</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-with-index">
- <compilation-unit name="q14_promotion_effect">
- <output-dir compare="Text">q14_promotion_effect</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-with-index">
- <compilation-unit name="q15_top_supplier">
- <output-dir compare="Text">q15_top_supplier</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-with-index">
- <compilation-unit name="q16_parts_supplier_relationship">
- <output-dir compare="Text">q16_parts_supplier_relationship</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-with-index">
- <compilation-unit name="q17_small_quantity_order_revenue">
- <output-dir compare="Text">q17_small_quantity_order_revenue</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-with-index">
- <compilation-unit name="q17_large_gby_variant">
- <output-dir compare="Text">q17_large_gby_variant</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-with-index">
- <compilation-unit name="q18_large_volume_customer">
- <output-dir compare="Text">q18_large_volume_customer</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-with-index">
- <compilation-unit name="q19_discounted_revenue">
- <output-dir compare="Text">q19_discounted_revenue</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-with-index">
- <compilation-unit name="q01_pricing_summary_report_nt">
- <output-dir compare="Text">q01_pricing_summary_report_nt</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-with-index">
- <compilation-unit name="q20_potential_part_promotion">
- <output-dir compare="Text">q20_potential_part_promotion</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-with-index">
- <compilation-unit name="q21_suppliers_who_kept_orders_waiting">
- <output-dir compare="Text">q21_suppliers_who_kept_orders_waiting</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-with-index">
- <compilation-unit name="q22_global_sales_opportunity">
- <output-dir compare="Text">q22_global_sales_opportunity</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-with-index">
- <compilation-unit name="q02_minimum_cost_supplier">
- <output-dir compare="Text">q02_minimum_cost_supplier</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-with-index">
- <compilation-unit name="q03_shipping_priority_nt">
- <output-dir compare="Text">q03_shipping_priority_nt</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-with-index">
- <compilation-unit name="q04_order_priority">
- <output-dir compare="Text">q04_order_priority</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-with-index">
- <compilation-unit name="q04_order_priority_with_nodegroup">
- <output-dir compare="Text">q04_order_priority</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-with-index">
- <compilation-unit name="q05_local_supplier_volume">
- <output-dir compare="Text">q05_local_supplier_volume</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-with-index">
- <compilation-unit name="q06_forecast_revenue_change">
- <output-dir compare="Text">q06_forecast_revenue_change</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-with-index">
- <compilation-unit name="q07_volume_shipping">
- <output-dir compare="Text">q07_volume_shipping</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-with-index">
- <compilation-unit name="q08_national_market_share">
- <output-dir compare="Text">q08_national_market_share</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-with-index">
- <compilation-unit name="q09_product_type_profit_nt">
- <output-dir compare="Text">q09_product_type_profit_nt</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-with-index">
- <compilation-unit name="query-issue562">
- <output-dir compare="Text">query-issue562</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-with-index">
- <compilation-unit name="query-issue810">
- <output-dir compare="Text">query-issue810</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-with-index">
- <compilation-unit name="query-issue810-2">
- <output-dir compare="Text">query-issue810-2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-with-index">
- <compilation-unit name="query-issue810-3">
- <output-dir compare="Text">query-issue810-3</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-with-index">
- <compilation-unit name="query-issue827">
- <output-dir compare="Text">query-issue827</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-with-index">
- <compilation-unit name="query-issue827-2">
- <output-dir compare="Text">query-issue827-2</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="tpch-sql-like">
- <test-case FilePath="tpch-sql-like">
- <compilation-unit name="query-issue638">
- <output-dir compare="Text">query-issue638</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-like">
- <compilation-unit name="query-issue785">
- <output-dir compare="Text">query-issue785</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-like">
- <compilation-unit name="query-issue785-2">
- <output-dir compare="Text">query-issue785-2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-like">
- <compilation-unit name="query-issue786">
- <output-dir compare="Text">query-issue786</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-like">
- <compilation-unit name="query-issue601">
- <output-dir compare="Text">query-issue601</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-like">
- <compilation-unit name="q10_returned_item">
- <output-dir compare="Text">q10_returned_item</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-like">
- <compilation-unit name="q10_returned_item_int64">
- <output-dir compare="Text">q10_returned_item_int64</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-like">
- <compilation-unit name="q11_important_stock">
- <output-dir compare="Text">q11_important_stock</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-like">
- <compilation-unit name="q12_shipping">
- <output-dir compare="Text">q12_shipping</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-like">
- <compilation-unit name="q13_customer_distribution">
- <output-dir compare="Text">q13_customer_distribution</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-like">
- <compilation-unit name="q14_promotion_effect">
- <output-dir compare="Text">q14_promotion_effect</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-like">
- <compilation-unit name="q15_top_supplier">
- <output-dir compare="Text">q15_top_supplier</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-like">
- <compilation-unit name="q16_parts_supplier_relationship">
- <output-dir compare="Text">q16_parts_supplier_relationship</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-like">
- <compilation-unit name="q17_small_quantity_order_revenue">
- <output-dir compare="Text">q17_small_quantity_order_revenue</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-like">
- <compilation-unit name="q17_large_gby_variant">
- <output-dir compare="Text">q17_large_gby_variant</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-like">
- <compilation-unit name="q18_large_volume_customer">
- <output-dir compare="Text">q18_large_volume_customer</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-like">
- <compilation-unit name="q19_discounted_revenue">
- <output-dir compare="Text">q19_discounted_revenue</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-like">
- <compilation-unit name="q01_pricing_summary_report_nt">
- <output-dir compare="Text">q01_pricing_summary_report_nt</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-like">
- <compilation-unit name="q20_potential_part_promotion">
- <output-dir compare="Text">q20_potential_part_promotion</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-like">
- <compilation-unit name="q21_suppliers_who_kept_orders_waiting">
- <output-dir compare="Text">q21_suppliers_who_kept_orders_waiting</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-like">
- <compilation-unit name="q22_global_sales_opportunity">
- <output-dir compare="Text">q22_global_sales_opportunity</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-like">
- <compilation-unit name="q02_minimum_cost_supplier">
- <output-dir compare="Text">q02_minimum_cost_supplier</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-like">
- <compilation-unit name="q03_shipping_priority_nt">
- <output-dir compare="Text">q03_shipping_priority_nt</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-like">
- <compilation-unit name="q04_order_priority">
- <output-dir compare="Text">q04_order_priority</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-like">
- <compilation-unit name="q05_local_supplier_volume">
- <output-dir compare="Text">q05_local_supplier_volume</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-like">
- <compilation-unit name="q06_forecast_revenue_change">
- <output-dir compare="Text">q06_forecast_revenue_change</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-like">
- <compilation-unit name="q07_volume_shipping">
- <output-dir compare="Text">q07_volume_shipping</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-like">
- <compilation-unit name="q08_national_market_share">
- <output-dir compare="Text">q08_national_market_share</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tpch-sql-like">
- <compilation-unit name="q09_product_type_profit_nt">
- <output-dir compare="Text">q09_product_type_profit_nt</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="writers">
- <test-case FilePath="writers">
- <compilation-unit name="print_01">
- <output-dir compare="Text">print_01</output-dir>
- </compilation-unit>
- </test-case>
- <!-- TODO(madhusudancs): Enable this test when REST API supports serialized output support.
- <test-case FilePath="writers">
- <compilation-unit name="serialized_01">
- <output-dir compare="Text">serialized_01</output-dir>
- </compilation-unit>
- </test-case>
- -->
- </test-group>
- <test-group name="cross-dataverse">
- <test-case FilePath="cross-dataverse">
- <compilation-unit name="cross-dv01">
- <output-dir compare="Text">cross-dv01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="cross-dataverse">
- <compilation-unit name="cross-dv02">
- <output-dir compare="Text">cross-dv02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="cross-dataverse">
- <compilation-unit name="cross-dv03">
- <output-dir compare="Text">cross-dv03</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="cross-dataverse">
- <compilation-unit name="cross-dv04">
- <output-dir compare="Text">cross-dv04</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="cross-dataverse">
- <compilation-unit name="cross-dv07">
- <output-dir compare="Text">cross-dv07</output-dir>
- </compilation-unit>
- </test-case>
- <!--NotImplementedException: No binary comparator factory implemented for type OBJECT.
- <test-case FilePath="cross-dataverse">
- <compilation-unit name="cross-dv08">
- <output-dir compare="Text">cross-dv08</output-dir>
- </compilation-unit>
- </test-case>
- -->
- <test-case FilePath="cross-dataverse">
- <compilation-unit name="cross-dv09">
- <output-dir compare="Text">cross-dv09</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="cross-dataverse">
- <compilation-unit name="cross-dv11">
- <output-dir compare="Text">cross-dv11</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="cross-dataverse">
- <compilation-unit name="cross-dv12">
- <output-dir compare="Text">cross-dv12</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="cross-dataverse">
- <compilation-unit name="cross-dv14">
- <output-dir compare="Text">cross-dv14</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="cross-dataverse">
- <compilation-unit name="cross-dv15">
- <output-dir compare="Text">cross-dv15</output-dir>
- </compilation-unit>
- </test-case>
- <!--NotImplementedException: No binary comparator factory implemented for type OBJECT.
- <test-case FilePath="cross-dataverse">
- <compilation-unit name="cross-dv17">
- <output-dir compare="Text">cross-dv17</output-dir>
- </compilation-unit>
- </test-case>
- -->
- <!--NotImplementedException: No binary comparator factory implemented for type OBJECT.
- <test-case FilePath="cross-dataverse">
- <compilation-unit name="cross-dv18">
- <output-dir compare="Text">cross-dv18</output-dir>
- </compilation-unit>
- </test-case>
- -->
- <test-case FilePath="cross-dataverse">
- <compilation-unit name="cross-dv19">
- <output-dir compare="Text">cross-dv19</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="cross-dataverse">
- <compilation-unit name="cross-dv20">
- <output-dir compare="Text">cross-dv20</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="cross-dataverse">
- <compilation-unit name="insert_across_dataverses">
- <output-dir compare="Text">insert_across_dataverses</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="cross-dataverse">
- <compilation-unit name="join_across_dataverses">
- <output-dir compare="Text">join_across_dataverses</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="cross-dataverse">
- <compilation-unit name="drop-dataverse">
- <output-dir compare="Text">drop-dataverse</output-dir>
- <expected-error>ASX1147: Cannot drop dataverse: type a.a being used by dataset b.b1</expected-error>
- <expected-error>ASX1147: Cannot drop dataverse: synonym a.s1 being used by function b.f1()</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="cross-dataverse">
- <compilation-unit name="drop-type-used-elsewhere">
- <output-dir compare="Text">drop-type-used-elsewhere</output-dir>
- <expected-error>Cannot drop type a.a being used by dataset b.b1</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="cross-dataverse">
- <compilation-unit name="drop-type-used-here-dataset">
- <output-dir compare="Text">drop-type-used-here-dataset</output-dir>
- <expected-error>Cannot drop type c.a being used by dataset c.a1</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="cross-dataverse">
- <compilation-unit name="drop-type-used-here-type">
- <output-dir compare="Text">drop-type-used-here-type</output-dir>
- <expected-error>Cannot drop type c.a being used by type c.b</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="cross-dataverse">
- <compilation-unit name="query-dataset-with-foreign-type">
- <output-dir compare="Text">query-dataset-with-foreign-type</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="user-defined-functions">
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="bad-function-ddl-1">
- <output-dir compare="Text">bad-function-ddl-1</output-dir>
- <expected-error>Cannot find dataset TweetMessages in dataverse experiments nor an alias with name TweetMessages</expected-error>
- <expected-error>Cannot find dataset TweetMessages2 in dataverse experiments2 nor an alias with name TweetMessages2</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="bad-function-ddl-2">
- <output-dir compare="Text">bad-function-ddl-2</output-dir>
- <expected-error>Cannot find dataset TweetMessages in dataverse experiments2 nor an alias with name TweetMessages</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="bad-function-ddl-3">
- <output-dir compare="Text">bad-function-ddl-3</output-dir>
- <expected-error>Cannot find dataset TweetMessages in dataverse experiments nor an alias with name TweetMessages</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="bad-function-ddl-4">
- <output-dir compare="Text">bad-function-ddl-4</output-dir>
- <expected-error>Cannot find dataset TweetMessages in dataverse experients nor an alias with name TweetMessages</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="bad-function-ddl-5">
- <output-dir compare="Text">bad-function-ddl-5</output-dir>
- <expected-error>ASX1081: Cannot find function with signature experiments.function_that_does_not_exist()</expected-error>
- <expected-error>ASX1081: Cannot find function with signature experiments2.function_that_does_not_exist()</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="bad-function-ddl-6">
- <output-dir compare="Text">bad-function-ddl-6</output-dir>
- <expected-error>ASX1081: Cannot find function with signature experiments.function_that_does_not_exist(2)</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="bad-function-ddl-7">
- <output-dir compare="Text">bad-function-ddl-7</output-dir>
- <expected-error>Cannot find dataset TweetMessaes in dataverse experiments nor an alias with name TweetMessaes</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="bad-function-ddl-8">
- <output-dir compare="Text">bad-function-ddl-8</output-dir>
- <expected-error>Cannot find dataset TweetMessaes in dataverse experiments nor an alias with name TweetMessaes</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="bad-function-ddl-9">
- <output-dir compare="Text">bad-function-ddl-9</output-dir>
- <expected-error>ASX1081: Cannot find function with signature experiments.function_that_does_not_exist()</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="bad-function-ddl-10">
- <output-dir compare="Text">bad-function-ddl-10</output-dir>
- <expected-error>ASX1081: Cannot find function with signature experiments.f0(2)</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="bad-function-ddl-11">
- <output-dir compare="Text">bad-function-ddl-11</output-dir>
- <expected-error>ASX1001: Syntax error: Unexpected type declaration for parameter a in function myfn001</expected-error>
- <expected-error>ASX1001: Syntax error: Unexpected return type declaration for function myfn002</expected-error>
- <expected-error>ASX1001: Syntax error: Unexpected return type declaration for function myfn003</expected-error>
- <expected-error>ASX1001: Syntax error: Unexpected IF NOT EXISTS (in line 29, at column 45)</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="check-dependencies-1">
- <output-dir compare="Text">check-dependencies-1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="create-or-replace-function-1">
- <output-dir compare="Text">create-or-replace-function-1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="drop-dependency-1">
- <output-dir compare="Text">drop-dependency-1</output-dir>
- <expected-error>ASX1147: Cannot drop dataverse: function C.f1(2) being used by function B.f0(2)</expected-error>
- <expected-error>ASX1147: Cannot drop dataverse: function C.f2(...) being used by function B.f3(2)</expected-error>
- <expected-error>ASX1147: Cannot drop dataverse: function C.f4(2) being used by function B.f5(...)</expected-error>
- <expected-error>ASX1147: Cannot drop dataverse: function C.f6(...) being used by function B.f7(...)</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="drop-dependency-2">
- <output-dir compare="Text">drop-dependency-2</output-dir>
- <expected-error>ASX1147: Cannot drop dataverse: dataset (or view) C.TweetMessages being used by function B.f2(2)</expected-error>
- <expected-error>ASX1147: Cannot drop dataverse: dataset (or view) C.TweetMessages being used by function B.f3(...)</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="drop-dependency-3">
- <output-dir compare="Text">drop-dependency-3</output-dir>
- <expected-error>Cannot drop function C.f1(2) being used by function B.f0(2)</expected-error>
- <expected-error>Cannot drop function C.f3(...) being used by function B.f2(2)</expected-error>
- <expected-error>Cannot drop function C.f5(2) being used by function B.f4(...)</expected-error>
- <expected-error>Cannot drop function C.f7(...) being used by function B.f6(...)</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="drop-dependency-4">
- <output-dir compare="Text">drop-dependency-4</output-dir>
- <expected-error>ASX1148: Cannot drop dataset C.TweetMessages being used by function B.f2(2)</expected-error>
- <expected-error>ASX1148: Cannot drop dataset C.TweetMessages being used by function B.f2(...)</expected-error>
- <expected-error>ASX1148: Cannot drop synonym C.TweetMessagesSyn being used by function B.f2(2)</expected-error>
- <expected-error>ASX1148: Cannot drop synonym C.TweetMessagesSyn being used by function B.f2(...)</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="drop-dependency-5">
- <output-dir compare="Text">drop-dependency-5</output-dir>
- <expected-error>Cannot drop function C.f1(2) being used by function C.f0(2)</expected-error>
- <expected-error>Cannot drop function C.f1(2) being used by function C.f0(...)</expected-error>
- <expected-error>Cannot drop function C.f1(...) being used by function C.f0(2)</expected-error>
- <expected-error>Cannot drop function C.f1(...) being used by function C.f0(...)</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="drop-dependency-6">
- <output-dir compare="Text">drop-dependency-6</output-dir>
- <expected-error>ASX1148: Cannot drop dataset C.TweetMessages being used by function C.f2(2)</expected-error>
- <expected-error>ASX1148: Cannot drop dataset C.TweetMessages being used by function C.f2(...)</expected-error>
- <expected-error>ASX1148: Cannot drop synonym C.TweetMessagesSyn being used by function C.f2(2)</expected-error>
- <expected-error>ASX1148: Cannot drop synonym C.TweetMessagesSyn being used by function C.f2(...)</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="drop-function-1">
- <output-dir compare="Text">drop-function-1</output-dir>
- <expected-error>ASX1081: Cannot find function with signature experiments.my_sum(2)</expected-error>
- <expected-error>ASX1081: Cannot find function with signature experiments.my_sum_va(2)</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="single-line-definition">
- <output-dir compare="Text">single-line-definition</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="query-ASTERIXDB-1298">
- <output-dir compare="Text">query-ASTERIXDB-1298</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="query-ASTERIXDB-1652">
- <output-dir compare="Text">query-ASTERIXDB-1652-2</output-dir>
- <expected-error>ASX1063: Cannot find dataverse with name test</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="query-ASTERIXDB-1652-2">
- <output-dir compare="Text">query-ASTERIXDB-1652-2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="query-issue218-2">
- <output-dir compare="Text">query-issue218-2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="query-issue218">
- <output-dir compare="Text">query-issue218</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="query-issue201">
- <output-dir compare="Text">query-issue201</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="query-issue172">
- <output-dir compare="Text">query-issue172</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="query-issue455">
- <output-dir compare="Text">query-issue455</output-dir>
- <expected-error>ASX1081: Cannot find function with signature test.printName()</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="query-issue489">
- <output-dir compare="Text">query-issue489</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="udf01">
- <output-dir compare="Text">udf01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="udf02">
- <output-dir compare="Text">udf02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="udf03">
- <output-dir compare="Text">udf03</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="udf04">
- <output-dir compare="Text">udf04</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="udf05">
- <output-dir compare="Text">udf05</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="udf06">
- <output-dir compare="Text">udf06</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="udf07">
- <output-dir compare="Text">udf07</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="udf08">
- <output-dir compare="Text">udf08</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="udf09">
- <output-dir compare="Text">udf09</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="udf10">
- <output-dir compare="Text">udf10</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="udf11">
- <output-dir compare="Text">udf11</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="udf12">
- <output-dir compare="Text">udf12</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="udf13">
- <output-dir compare="Text">udf13</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="udf14">
- <output-dir compare="Text">udf14</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="udf15">
- <output-dir compare="Text">udf15</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="udf16">
- <output-dir compare="Text">udf16</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="udf17">
- <output-dir compare="Text">udf17</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="udf18">
- <output-dir compare="Text">udf18</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="udf19">
- <output-dir compare="Text">udf19</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="udf20">
- <output-dir compare="Text">udf20</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="udf21">
- <output-dir compare="Text">udf21</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="udf22">
- <output-dir compare="Text">udf22</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="udf23">
- <output-dir compare="Text">udf23</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="udf24">
- <output-dir compare="Text">udf24</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="udf25">
- <output-dir compare="Text">udf25</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="udf27">
- <output-dir compare="Text">udf27</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="udf28">
- <output-dir compare="Text">udf28</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="udf29">
- <output-dir compare="Text">udf29</output-dir>
- </compilation-unit>
- </test-case>
- <!-- Need to verify the expected exception -->
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="udf30">
- <output-dir compare="Text">udf30</output-dir>
- <expected-error>ASX1073: Cannot resolve alias reference for undefined identifier y (in line 30, at column 8)</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="udf31">
- <output-dir compare="Text">udf31</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="udf32_metadata">
- <output-dir compare="Text">udf32_metadata</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="udf33_overloading">
- <output-dir compare="Text">udf33_overloading</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="udf34">
- <output-dir compare="Text">udf34</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="udf35_varargs_misc">
- <output-dir compare="Text">udf35_varargs_misc</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="udf36_in_with">
- <output-dir compare="Text">udf36_in_with</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="udf37_recursion">
- <output-dir compare="Text">none</output-dir>
- <expected-error>ASX1149: Illegal function or view recursion (in line 24, at column 1)</expected-error>
- <expected-error>ASX1149: Illegal function or view recursion (in line 25, at column 1)</expected-error>
- <expected-error>ASX1149: Illegal function or view recursion (in line 26, at column 1)</expected-error>
- <expected-error>ASX1149: Illegal function or view recursion (in line 27, at column 1)</expected-error>
- <expected-error>ASX1149: Illegal function or view recursion (in line 28, at column 1)</expected-error>
- <expected-error>ASX1149: Illegal function or view recursion (in line 29, at column 1)</expected-error>
- <expected-error>ASX1149: Illegal function or view recursion (in line 30, at column 1)</expected-error>
- <expected-error>ASX1149: Illegal function or view recursion (in line 31, at column 1)</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="udf38_no_recursion">
- <output-dir compare="Text">udf38_no_recursion</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="udf39_illegal_call">
- <output-dir compare="Text">none</output-dir>
- <expected-error>ASX1150: Illegal use of function test.f1a(1) (in line 32, at column 26)</expected-error>
- <expected-error>ASX1150: Illegal use of function test.f2a(...) (in line 29, at column 28)</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="f01">
- <output-dir compare="Text">f01</output-dir>
- <expected-error>ASX1081: Cannot find function with signature test.tinyint()</expected-error>
- </compilation-unit>
- </test-case>
- <!-- This test case is not valid anymore since we do not required "IMPORT_PRIVATE_FUNCTIONS" flag anymore -->
- <!-- <test-case FilePath="user-defined-functions">
- <compilation-unit name="invoke-private-function">
- <output-dir compare="Text">invoke-private-function</output-dir>
- </compilation-unit>
- </test-case>-->
- <!--
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="query-ASTERIXDB-1308-1">
- <output-dir compare="Text">query-ASTERIXDB-1308-1</output-dir>
- </compilation-unit>
- </test-case>
- -->
- <!-- <test-case FilePath="user-defined-functions">
- <compilation-unit name="query-ASTERIXDB-1308-2">
- <output-dir compare="Text">query-ASTERIXDB-1308-2</output-dir>
- </compilation-unit>
- </test-case> -->
- <!-- This test case is not valid anymore since we do not required "IMPORT_PRIVATE_FUNCTIONS" flag anymore -->
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="query-ASTERIXDB-1317">
- <output-dir compare="Text">query-ASTERIXDB-1317</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="view">
- <test-case FilePath="view">
- <compilation-unit name="create-view-1">
- <output-dir compare="Text">create-view-1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="view">
- <compilation-unit name="create-view-2-negative">
- <output-dir compare="Text">none</output-dir>
- <expected-error>ASX1063: Cannot find dataverse with name test (in line 24, at column 1)</expected-error>
- <expected-error><![CDATA[ASX1001: Syntax error: In line 25 >>create view test.v1 as select * from range(1,2) r order by;<< Encountered ";" at column 59]]></expected-error>
- <expected-error>ASX1081: Cannot find function with signature test.undefined_range(2) (in line 25, at column 38)</expected-error>
- <expected-error>ASX1160: A view with this name test.v1 already exists (in line 26, at column 1)</expected-error>
- <expected-error>ASX1072: A dataset with name ds1 already exists in dataverse test (in line 30, at column 1)</expected-error>
- <expected-error>ASX1072: A dataset with name ds2 already exists in dataverse test (in line 30, at column 1)</expected-error>
- <expected-error>ASX1160: A view with this name test.ds1 already exists (in line 27, at column 1)</expected-error>
- <expected-error><![CDATA[ASX1001: Syntax error: Unexpected IF NOT EXISTS (in line 25, at column 1)]]></expected-error>
- <expected-error>ASX1149: Illegal function or view recursion (in line 31, at column 1)</expected-error>
- <expected-error>ASX1149: Illegal function or view recursion (in line 32, at column 1)</expected-error>
- <expected-error>ASX1149: Illegal function or view recursion (in line 33, at column 1)</expected-error>
- <expected-error><![CDATA[ASX1001: Syntax error: In line 25 >>create view test.v1 primary key (r) not enforced as<< Encountered "primary" at column 21]]></expected-error>
- <expected-error><![CDATA[ASX1001: Syntax error: In line 28 >>create view test.v2 foreign key (r) references v1 not enforced as<< Encountered <IDENTIFIER> "foreign" at column 21]]></expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="view">
- <compilation-unit name="create-view-3-typed">
- <output-dir compare="Text">create-view-3-typed</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="view" check-warnings="true">
- <compilation-unit name="create-view-4-typed-warn">
- <output-dir compare="Text">create-view-4-typed-warn</output-dir>
- <expected-warn>ASX0006: Invalid format for tinyint in a (in line 30, at column 6)</expected-warn>
- <expected-warn>ASX0006: Invalid format for smallint in b (in line 30, at column 6)</expected-warn>
- <expected-warn>ASX0006: Invalid format for integer in c (in line 30, at column 6)</expected-warn>
- <expected-warn>ASX0006: Invalid format for bigint in d (in line 30, at column 6)</expected-warn>
- <expected-warn>ASX0006: Invalid format for float in e (in line 30, at column 6)</expected-warn>
- <expected-warn>ASX0006: Invalid format for double in f (in line 30, at column 6)</expected-warn>
- <expected-warn>ASX0006: Invalid format for datetime in g (in line 30, at column 6)</expected-warn>
- <expected-warn>ASX0006: Invalid format for date in h (in line 30, at column 6)</expected-warn>
- <expected-warn>ASX0006: Invalid format for time in j (in line 30, at column 6)</expected-warn>
- <expected-warn>ASX0006: Invalid format for duration in k (in line 30, at column 6)</expected-warn>
- <expected-warn>ASX0006: Invalid format for yearmonthduration in m (in line 30, at column 6)</expected-warn>
- <expected-warn>ASX0006: Invalid format for daytimeduration in n (in line 30, at column 6)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: boolean-default-null() cannot process input type duration (in line 30, at column 6)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: int8-default-null() cannot process input type duration (in line 30, at column 6)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: int16-default-null() cannot process input type duration (in line 30, at column 6)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: int32-default-null() cannot process input type duration (in line 30, at column 6)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: int64-default-null() cannot process input type duration (in line 30, at column 6)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: float-default-null() cannot process input type duration (in line 30, at column 6)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: double-default-null() cannot process input type duration (in line 30, at column 6)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: datetime-default-null() cannot process input type duration (in line 30, at column 6)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: date-default-null() cannot process input type duration (in line 30, at column 6)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: time-default-null() cannot process input type duration (in line 30, at column 6)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: duration-default-null() cannot process input type date (in line 30, at column 6)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: year-month-duration-default-null() cannot process input type date (in line 30, at column 6)</expected-warn>
- <expected-warn>ASX0004: Unsupported type: day-time-duration-default-null() cannot process input type date (in line 30, at column 6)</expected-warn>
- </compilation-unit>
- </test-case>
- <test-case FilePath="view" check-warnings="true">
- <compilation-unit name="create-view-5-typed-warn">
- <output-dir compare="Text">create-view-5-typed-warn</output-dir>
- <expected-warn>ASX0006: Invalid format for datetime in a (in line 27, at column 6)</expected-warn>
- <expected-warn>ASX0006: Invalid format for date in b (in line 27, at column 6)</expected-warn>
- <expected-warn>ASX0006: Invalid format for time in c (in line 27, at column 6)</expected-warn>
- </compilation-unit>
- </test-case>
- <test-case FilePath="view">
- <compilation-unit name="create-view-6-typed-negative">
- <output-dir compare="Text">none</output-dir>
- <expected-error>ASX1082: Cannot find datatype with name unknown_dv.t1</expected-error>
- <expected-error>ASX1082: Cannot find datatype with name test.t1_unknown</expected-error>
- <expected-error>ASX1079: Compilation error: view type cannot have open fields (in line 29, at column 1)</expected-error>
- <expected-error>ASX1004: Unsupported type: view cannot process input type t1_a (in line 30, at column 1)</expected-error>
- <expected-error><![CDATA[ASX1001: Syntax error: In line 25 >>create view test.v1(r bigint, a [bigint]) default null as<< Encountered "[" at column 33]]></expected-error>
- <expected-error>ASX1092: Parameter date_illegal_property_name cannot be set (in line 25, at column 1)</expected-error>
- <expected-error><![CDATA[ASX1001: Syntax error: In line 25 >>create view test.v1(r bigint) as<< Encountered "as" at column 31]]></expected-error>
- <expected-error><![CDATA[ASX1014: Field 'unknown_field' is not found (in line 25, at column 1)]]></expected-error>
- <expected-error><![CDATA[ASX1014: Field 'unknown_field_2' is not found (in line 25, at column 1)]]></expected-error>
- <expected-error><![CDATA[ASX1001: Syntax error: In line 28 >> as select r from range(1,2) r;<< Encountered "as" at column 3]]></expected-error>
- <expected-error><![CDATA[ASX0013: Duplicate field name 'r' (in line 25, at column 20)]]></expected-error>
- <expected-error><![CDATA[ASX1167: Cannot change primary key of view test1.employee_v1 (in line 38, at column 1)]]></expected-error>
- <expected-error><![CDATA[ASX1162: Invalid primary key definition (in line 25, at column 1)]]></expected-error>
- <expected-error><![CDATA[ASX1162: Invalid primary key definition (in line 26, at column 1)]]></expected-error>
- <expected-error><![CDATA[ASX1001: Syntax error: In line 36 >> as employee;<< Encountered "as" at column 3]]></expected-error>
- <expected-error><![CDATA[ASX1165: Invalid foreign key definition: view test1.employee_v1 does not have a primary key (in line 32, at column 1)]]></expected-error>
- <expected-error><![CDATA[ASX1166: Invalid foreign key definition: foreign key does not match primary key of view test1.employee_v1 (in line 34, at column 1)]]></expected-error>
- <expected-error><![CDATA[ASX1166: Invalid foreign key definition: foreign key does not match primary key of view test1.employee_v2 (in line 34, at column 1)]]></expected-error>
- <expected-error><![CDATA[ASX1164: Invalid foreign key definition (in line 34, at column 1)]]></expected-error>
- <expected-error><![CDATA[ASX1063: Cannot find dataverse with name test3 (in line 42, at column 1)]]></expected-error>
- <expected-error><![CDATA[ASX1159: Cannot find view with name test1.employee_v3 (in line 42, at column 1)]]></expected-error>
- <expected-error><![CDATA[ASX1159: Cannot find view with name test1.employee (in line 43, at column 1)]]></expected-error>
- <expected-error><![CDATA[ASX1164: Invalid foreign key definition (in line 43, at column 1)]]></expected-error>
- <expected-error><![CDATA[ASX1164: Invalid foreign key definition (in line 39, at column 1)]]></expected-error>
- <expected-error><![CDATA[ASX1166: Invalid foreign key definition: foreign key does not match primary key of view test1.employee_v1 (in line 39, at column 1)]]></expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="view">
- <compilation-unit name="create-view-7-foreign-key">
- <output-dir compare="Text">create-view-7-foreign-key</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="view">
- <compilation-unit name="drop-dataverse-1">
- <output-dir compare="Text">drop-dataverse-1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="view">
- <compilation-unit name="drop-dataverse-2-negative">
- <output-dir compare="Text">none</output-dir>
- <expected-error>ASX1147: Cannot drop dataverse: dataset (or view) test2.v2 being used by view test1.v1</expected-error>
- <expected-error>ASX1147: Cannot drop dataverse: dataset (or view) test2.v2 being used by function test1.f1()</expected-error>
- <expected-error>ASX1147: Cannot drop dataverse: dataset (or view) test2.ds2 being used by view test1.v1</expected-error>
- <expected-error>ASX1147: Cannot drop dataverse: function test2.f2() being used by view test1.v1</expected-error>
- <expected-error>ASX1147: Cannot drop dataverse: synonym test2.s3 being used by view test1.v1</expected-error>
- <expected-error>ASX1147: Cannot drop dataverse: type test2.t1 being used by dataset test1.v1</expected-error>
- <expected-error>ASX1147: Cannot drop dataverse: dataset (or view) test2.employee_v2 being used by view test1.employee_v1</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="view">
- <compilation-unit name="drop-view-1">
- <output-dir compare="Text">drop-view-1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="view">
- <compilation-unit name="drop-view-2-negative">
- <output-dir compare="Text">drop-view-2-negative</output-dir>
- <expected-error>ASX1063: Cannot find dataverse with name test (in line 24, at column 1)</expected-error>
- <expected-error>ASX1159: Cannot find view with name test.v1 (in line 25, at column 1)</expected-error>
- <expected-error>ASX1050: Cannot find dataset with name v1 in dataverse test (in line 27, at column 1)</expected-error>
- <expected-error>ASX1159: Cannot find view with name test.ds1 (in line 30, at column 1)</expected-error>
- <expected-error>ASX1148: Cannot drop dataset test2.ds2 being used by view test1.v1</expected-error>
- <expected-error>ASX1148: Cannot drop function test2.f2() being used by view test1.v1</expected-error>
- <expected-error>ASX1148: Cannot drop synonym test2.s2 being used by view test1.v1</expected-error>
- <expected-error>ASX1148: Cannot drop view test2.v2 being used by view test1.v1</expected-error>
- <expected-error>ASX1148: Cannot drop view test2.v2 being used by function test1.f1()</expected-error>
- <expected-error>ASX1148: Cannot drop type test2.t1 being used by dataset test1.v1</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="view">
- <compilation-unit name="view-1">
- <output-dir compare="Text">view-1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="view">
- <compilation-unit name="view-2-negative">
- <output-dir compare="Text">none</output-dir>
- <expected-error>ASX1050: Cannot find dataset with name v1 in dataverse test1 (in line 24, at column 17)</expected-error>
- <expected-error>ASX1050: Cannot find dataset with name v2 in dataverse test1 (in line 24, at column 17)</expected-error>
- <expected-error>ASX1050: Cannot find dataset with name v3 in dataverse test1 (in line 24, at column 1)</expected-error>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="load">
- <test-case FilePath="load">
- <compilation-unit name="load_non-empty_index">
- <output-dir compare="Text">load_non-empty_index</output-dir>
- <expected-error>HYR0034: Cannot load an index that is not empty</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="load">
- <compilation-unit name="csv_01">
- <output-dir compare="Text">csv_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="load">
- <compilation-unit name="csv_02">
- <output-dir compare="Text">csv_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="load">
- <compilation-unit name="csv_03">
- <output-dir compare="Text">csv_03</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="load">
- <compilation-unit name="csv_04">
- <output-dir compare="Text">csv_04</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="load">
- <compilation-unit name="csv_05">
- <output-dir compare="Text">csv_05</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="load">
- <compilation-unit name="csv_06">
- <output-dir compare="Text">csv_06</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="load">
- <compilation-unit name="csv_07">
- <output-dir compare="Text">csv_07</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="load">
- <compilation-unit name="csv_08_header_cr">
- <output-dir compare="Text">csv_08</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="load">
- <compilation-unit name="csv_08_header_lf">
- <output-dir compare="Text">csv_08</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="load">
- <compilation-unit name="csv_08_header_crlf">
- <output-dir compare="Text">csv_08</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="load">
- <compilation-unit name="issue14_query">
- <output-dir compare="Text">issue14_query</output-dir>
- <expected-error>Unspecified parameter: format</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="load">
- <compilation-unit name="issue315_query">
- <output-dir compare="Text">none</output-dir>
- <expected-error>Invalid path</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="load">
- <compilation-unit name="issue289_query">
- <output-dir compare="Text">issue289_query</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="load">
- <compilation-unit name="issue650_query">
- <output-dir compare="Text">none</output-dir>
- <expected-error>Cannot find dataset with name Users in dataverse fuzzyjoin</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="load">
- <compilation-unit name="type_promotion_0">
- <output-dir compare="Text">type_promotion_0</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="load">
- <compilation-unit name="escapes01">
- <output-dir compare="Text">escapes01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="load">
- <compilation-unit name="escapes02">
- <output-dir compare="Text">escapes02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="load">
- <compilation-unit name="escapes-err-1"><!-- Exception is never thrown!!!. needs to be investigated -->
- <output-dir compare="Text">none</output-dir>
- <!-- <expected-error>org.apache.hyracks.api.exceptions.HyracksException</expected-error> -->
- </compilation-unit>
- </test-case>
- <test-case FilePath="user-defined-functions">
- <compilation-unit name="query-issue244">
- <output-dir compare="Text">query-issue244</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="load">
- <compilation-unit name="duplicate-key-error">
- <output-dir compare="Text">none</output-dir>
- <expected-error>Loading duplicate keys into the primary storage</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="load">
- <compilation-unit name="issue610_adm_token_end_collection">
- <output-dir compare="Text">issue610_adm_token_end_collection</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="load">
- <compilation-unit name="adm_binary">
- <output-dir compare="Text">adm_binary</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="load">
- <compilation-unit name="dataset-with-meta">
- <output-dir compare="Text">dataset-with-meta</output-dir>
- <expected-error>ASX1079: Compilation error: DatasetWithMeta: load dataset is not supported on datasets with meta records (in line 27, at column 1)</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="load">
- <compilation-unit name="escapes-err-1">
- <output-dir compare="Text">escapes-err-1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="load">
- <compilation-unit name="escapes01">
- <output-dir compare="Text">escapes01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="load">
- <compilation-unit name="escapes02">
- <output-dir compare="Text">escapes02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="load">
- <compilation-unit name="file-not-found">
- <output-dir compare="Text">file-not-found</output-dir>
- <expected-error>ASX3077: bla: path not found</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="load">
- <compilation-unit name="type_promotion_0">
- <output-dir compare="Text">type_promotion_0</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="load">
- <compilation-unit name="utf8">
- <output-dir compare="Text">utf8</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="hints">
- <test-case FilePath="hints">
- <compilation-unit name="issue_251_dataset_hint_5">
- <output-dir compare="Text">issue_251_dataset_hint_5</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="hints">
- <compilation-unit name="issue_251_dataset_hint_7">
- <output-dir compare="Text">issue_251_dataset_hint_7</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="function">
- <test-case FilePath="function">
- <compilation-unit name="issue-2394">
- <output-dir compare="Text">issue-2394</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="function">
- <compilation-unit name="drop_if_exists">
- <output-dir compare="Text">drop_if_exists</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="feeds">
- <test-case FilePath="feeds">
- <compilation-unit name="feeds_07">
- <output-dir compare="Text">feeds_07</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="feeds">
- <compilation-unit name="feeds_08">
- <output-dir compare="Text">feeds_08</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="feeds">
- <compilation-unit name="feeds_09">
- <output-dir compare="Text">feeds_09</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="feeds">
- <compilation-unit name="create-policy-from-file">
- <output-dir compare="Text">create-policy-from-file</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="feeds">
- <compilation-unit name="feeds_01">
- <output-dir compare="Text">feeds_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="feeds">
- <compilation-unit name="feeds_02">
- <output-dir compare="Text">feeds_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="feeds">
- <compilation-unit name="feeds_03">
- <output-dir compare="Text">feeds_03</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="feeds">
- <compilation-unit name="feeds_10">
- <output-dir compare="Text">feeds_10</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="feeds">
- <compilation-unit name="feeds_11">
- <output-dir compare="Text">feeds_11</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="feeds">
- <compilation-unit name="feeds_12">
- <output-dir compare="Text">feeds_12</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="feeds">
- <compilation-unit name="feeds_13">
- <output-dir compare="Text">feeds_13</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="feeds">
- <compilation-unit name="issue_230_feeds">
- <output-dir compare="Text">issue_230_feeds</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="feeds">
- <compilation-unit name="insert-feed">
- <output-dir compare="Text">insert-feed</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="feeds">
- <compilation-unit name="insert-feed-with-pk-index">
- <output-dir compare="Text">insert-feed</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="feeds">
- <compilation-unit name="connect-feed-with-function">
- <output-dir compare="Text">connect-feed-with-function</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="feeds">
- <compilation-unit name="change-feed-filter-on-meta-dataset">
- <output-dir compare="Text">change-feed-filter-on-meta-dataset</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="feeds">
- <compilation-unit name="change-feed-with-meta-pk-index">
- <output-dir compare="Text">change-feed-with-meta-pk-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="feeds">
- <compilation-unit name="change-feed-with-meta-with-mixed-index">
- <output-dir compare="Text">change-feed-with-meta-with-mixed-index</output-dir>
- <expected-error>Compilation error: Cannot create index on meta fields (in line 61, at column 35)</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="feeds">
- <compilation-unit name="change-feed-with-meta-pk-in-meta-index-with-missing-after-ingest">
- <output-dir compare="Text">change-feed-with-meta-pk-in-meta-index-with-missing-after-ingest</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="feeds">
- <compilation-unit name="change-feed-with-meta-pk-in-meta-open-index-with-missing">
- <output-dir compare="Text">change-feed-with-meta-pk-in-meta-open-index-with-missing</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="feeds">
- <compilation-unit name="change-feed-with-meta-open-index-in-meta">
- <output-dir compare="Text">change-feed-with-meta-open-index-in-meta</output-dir>
- <expected-error>Compilation error: Cannot create index on meta fields (in line 60, at column 34)</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="feeds">
- <compilation-unit name="change-feed-with-meta-pk-in-meta-open-index-in-value">
- <output-dir compare="Text">change-feed-with-meta-pk-in-meta-open-index-in-value</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="feeds">
- <compilation-unit name="change-feed-with-meta-pk-in-meta-index-after-ingest">
- <output-dir compare="Text">change-feed-with-meta-pk-in-meta-index-after-ingest</output-dir>
- <expected-error>Compilation error: Cannot create index on meta fields (in line 27, at column 37)</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="feeds">
- <compilation-unit name="change-feed-with-meta-pk-in-meta-index-in-meta">
- <output-dir compare="Text">change-feed-with-meta-pk-in-meta-index-in-meta</output-dir>
- <expected-error>Compilation error: Cannot create index on meta fields (in line 60, at column 37)</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="feeds">
- <compilation-unit name="change-feed-with-meta-csv">
- <output-dir compare="Text">change-feed-with-meta-csv</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="feeds">
- <compilation-unit name="change-feed-with-meta-pk-in-meta">
- <output-dir compare="Text">change-feed-with-meta-pk-in-meta</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="feeds">
- <compilation-unit name="change-feed">
- <output-dir compare="Text">change-feed</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="feeds">
- <compilation-unit name="drop-nonexistent-feed">
- <output-dir compare="Text">drop-nonexistent-feed</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="feeds">
- <compilation-unit name="twitter-feed">
- <output-dir compare="Text">twitter-feed</output-dir>
- <expected-error>Twitter4J library not found!</expected-error>
- <expected-error>Unknown source feed: TwitterFeed</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="feeds">
- <compilation-unit name="revised-tweet-parser">
- <output-dir compare="Text">revised-tweet-parser</output-dir>
- <expected-error>Twitter4J library not found!</expected-error>
- <expected-error>Unknown source feed: TwitterFeed</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="feeds">
- <compilation-unit name="connect-feed">
- <output-dir compare="Text">connect-feed</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="feeds">
- <compilation-unit name="change-feed-with-meta-pk-in-meta">
- <output-dir compare="Text">change-feed-with-meta-pk-in-meta</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="feeds">
- <compilation-unit name="feed-with-filtered-dataset">
- <output-dir compare="Text">feed-with-filtered-dataset</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="feeds">
- <compilation-unit name="change-feed">
- <output-dir compare="Text">change-feed</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="feeds">
- <compilation-unit name="drop-dataverse-with-disconnected-feed">
- <output-dir compare="Text">drop-dataverse-with-disconnected-feed</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="feeds">
- <compilation-unit name="feed-push-socket">
- <output-dir compare="Text">feed-push-socket</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="feeds">
- <compilation-unit name="insert-feed">
- <output-dir compare="Text">insert-feed</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="feeds">
- <compilation-unit name="start-feed">
- <output-dir compare="Text">start-feed</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="feeds">
- <compilation-unit name="start-started-feed">
- <output-dir compare="Text">start-started-feed</output-dir>
- <expected-error>experiments.TweetFeed(Feed) is already started</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="feeds">
- <compilation-unit name="stop-stopped-feed">
- <output-dir compare="Text">stop-stopped-feed</output-dir>
- <expected-error>new_experiments.TweetFeed(Feed) cannot be stopped because its state is STOPPED</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="feeds">
- <compilation-unit name="push-socket-with-auuid">
- <output-dir compare="Text">push-socket-with-auuid</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="feeds">
- <compilation-unit name="disconnect-live-feed">
- <output-dir compare="Text">disconnect-live-feed</output-dir>
- <expected-error>This operation cannot be done when Feed</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="feeds">
- <compilation-unit name="connect-live-feed">
- <output-dir compare="Text">connect-live-feed</output-dir>
- <expected-error>This operation cannot be done when Feed</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="feeds">
- <compilation-unit name="record-reader-with-malformed-input-stream">
- <output-dir compare="Text">record-reader-with-malformed-input-stream</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="feeds">
- <compilation-unit name="feed-with-undefined-function">
- <output-dir compare="Text">feed-with-undefined-function</output-dir>
- <expected-error>Cannot find function</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="feeds">
- <compilation-unit name="drop-function-used-by-feed">
- <output-dir compare="Text">drop-function-used-by-feed</output-dir>
- <expected-error>ASX1148: Cannot drop function experiments.test_func0(1) being used by feed connection experiments.UserFeed</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="feeds">
- <compilation-unit name="drop-function-no-longer-used-by-feed">
- <output-dir compare="Text">drop-function-used-by-feed</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="feeds">
- <compilation-unit name="drop-dataverse-with-function-used-by-feed">
- <output-dir compare="Text">drop-dataverse-with-function-used-by-feed</output-dir>
- <expected-error>ASX1147: Cannot drop dataverse: function fundv.test_func0(1) being used by feed connection feeddv.UserFeed</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="feeds">
- <compilation-unit name="http_feed">
- <output-dir compare="Text">http_feed</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="feeds">
- <compilation-unit name="http_feed_json">
- <output-dir compare="Text">http_feed_json</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="feeds">
- <compilation-unit name="change-feed-with-where-on-meta">
- <output-dir compare="Text">change-feed-with-where-on-meta</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="meta">
- <test-case FilePath="meta">
- <compilation-unit name="meta_in_with_clause">
- <output-dir compare="Text">meta_in_with_clause</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="meta">
- <compilation-unit name="resolving_pk_with_meta">
- <output-dir compare="Text">resolving_pk_with_meta</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="meta">
- <compilation-unit name="meta_after_gby">
- <output-dir compare="Text">meta_after_gby</output-dir>
- <expected-error>Compilation error: Inappropriate use of function 'meta'. For example, after GROUP BY (in line 29, at column 21)</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="meta">
- <compilation-unit name="query_dataset_with_meta-1">
- <output-dir compare="Text">query_dataset_with_meta-1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="meta">
- <compilation-unit name="query_dataset_with_meta-2">
- <output-dir compare="Text">query_dataset_with_meta-2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="meta">
- <compilation-unit name="query_dataset_with_meta_failure">
- <output-dir compare="Text">query_dataset_with_meta_failure</output-dir>
- <expected-error>ASX1079: Compilation error: Cannot resolve ambiguous meta function call. There are more than one dataset choice (in line 24, at column 7)</expected-error>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="big-object">
- <test-case FilePath="big-object">
- <compilation-unit name="big_object_sort">
- <output-dir compare="Text">big_object_sort</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="big-object">
- <compilation-unit name="big_object_groupby">
- <output-dir compare="Text">big_object_groupby</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="big-object">
- <compilation-unit name="big_object_groupby-2">
- <output-dir compare="Text">big_object_groupby-2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="big-object">
- <compilation-unit name="big_object_join">
- <output-dir compare="Text">big_object_join</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="big-object">
- <compilation-unit name="big_object_join_low_memory_err">
- <output-dir compare="Text">big_object_join</output-dir>
- <expected-error>HYR0123: Insufficient memory is provided for the join operators, please increase the join memory budget.</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="big-object">
- <compilation-unit name="big_object_load_20M">
- <output-dir compare="Text">big_object_load_20M</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="big-object">
- <compilation-unit name="big_object_bulkload">
- <output-dir compare="Text">big_object_bulkload</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="big-object">
- <compilation-unit name="big_object_feed_20M">
- <output-dir compare="Text">big_object_feed_20M</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="big-object">
- <compilation-unit name="big_object_insert">
- <output-dir compare="Text">big_object_insert</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="big-object">
- <compilation-unit name="big_object_load_only_20M">
- <output-dir compare="Text">big_object_load_only_20M</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="leftouterjoin">
- <test-case FilePath="leftouterjoin">
- <compilation-unit name="loj-01-core">
- <output-dir compare="Text">loj-01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="leftouterjoin">
- <compilation-unit name="loj-01-sugar">
- <output-dir compare="Text">loj-01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="leftouterjoin">
- <compilation-unit name="loj-02-push-select">
- <output-dir compare="Text">loj-02-push-select</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="leftouterjoin">
- <compilation-unit name="loj-03-no-listify">
- <output-dir compare="Text">loj-03-no-listify</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="leftouterjoin">
- <compilation-unit name="query_issue658">
- <output-dir compare="Text">query_issue658</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="leftouterjoin">
- <compilation-unit name="query_issue285">
- <output-dir compare="Text">query_issue285</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="leftouterjoin">
- <compilation-unit name="query_issue285-2">
- <output-dir compare="Text">query_issue285-2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="leftouterjoin">
- <compilation-unit name="query_issue849">
- <output-dir compare="Text">query_issue849</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="leftouterjoin">
- <compilation-unit name="query_issue849-2">
- <output-dir compare="Text">query_issue849-2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="leftouterjoin">
- <compilation-unit name="empty-dataset">
- <output-dir compare="Text">empty-dataset</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="leftouterjoin">
- <compilation-unit name="query-ASTERIXDB-769">
- <output-dir compare="Text">query-ASTERIXDB-769</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="leftouterjoin">
- <compilation-unit name="query-ASTERIXDB-2857">
- <output-dir compare="Text">query-ASTERIXDB-2857</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="leftouterjoin">
- <compilation-unit name="right_branch_opt_1">
- <output-dir compare="Text">right_branch_opt_1</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="index-leftouterjoin">
- <test-case FilePath="index-leftouterjoin">
- <compilation-unit name="probe-pidx-with-join-btree-sidx1">
- <output-dir compare="Text">probe-pidx-with-join-btree-sidx1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-leftouterjoin">
- <compilation-unit name="probe-pidx-with-join-btree-sidx2">
- <output-dir compare="Text">probe-pidx-with-join-btree-sidx2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-leftouterjoin">
- <compilation-unit name="probe-pidx-join-btree-sidx3-idxonly">
- <output-dir compare="Text">probe-pidx-join-btree-sidx3-idxonly</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-leftouterjoin">
- <compilation-unit name="probe-pidx-with-join-rtree-sidx1">
- <output-dir compare="Text">probe-pidx-with-join-rtree-sidx1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-leftouterjoin">
- <compilation-unit name="probe-pidx-with-join-rtree-sidx2">
- <output-dir compare="Text">probe-pidx-with-join-rtree-sidx2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-leftouterjoin">
- <compilation-unit name="probe-pidx-with-join-invidx-sidx1">
- <output-dir compare="Text">probe-pidx-with-join-invidx-sidx1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-leftouterjoin">
- <compilation-unit name="probe-pidx-with-join-invidx-sidx2">
- <output-dir compare="Text">probe-pidx-with-join-invidx-sidx2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-leftouterjoin">
- <compilation-unit name="probe-sidx-btree-idxonly-join-btree-pidx1">
- <output-dir compare="Text">probe-sidx-btree-idxonly-join-btree-pidx1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-leftouterjoin">
- <compilation-unit name="probe-sidx-btree-idxonly-join-btree-sidx1-idxonly">
- <output-dir compare="Text">probe-sidx-btree-idxonly-join-btree-sidx1-idxonly</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-leftouterjoin">
- <compilation-unit name="probe-sidx-btree-non-idxonly-join-btree-pidx1">
- <output-dir compare="Text">probe-sidx-btree-non-idxonly-join-btree-pidx1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-leftouterjoin">
- <compilation-unit name="probe-sidx-btree-non-idxonly-join-btree-sidx1-idxonly">
- <output-dir compare="Text">probe-sidx-btree-non-idxonly-join-btree-sidx1-idxonly</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="index-leftouterjoin">
- <compilation-unit name="probe-pidx-with-join-btree-pidx1">
- <output-dir compare="Text">probe-pidx-with-join-btree-pidx1</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="distinct">
- <test-case FilePath="distinct">
- <compilation-unit name="array">
- <output-dir compare="Text">array</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="distinct">
- <compilation-unit name="record">
- <output-dir compare="Text">record</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="distinct">
- <compilation-unit name="query-issue443">
- <output-dir compare="Text">query-issue443</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="distinct">
- <compilation-unit name="query-issue443-2">
- <output-dir compare="Text">query-issue443-2</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="rightouterjoin">
- <test-case FilePath="rightouterjoin">
- <compilation-unit name="roj-01-core">
- <output-dir compare="Text">roj-01-core</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="rightouterjoin">
- <compilation-unit name="roj-02-core">
- <output-dir compare="Text">roj-02-core</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="rightouterjoin">
- <compilation-unit name="roj-03-negative">
- <output-dir compare="Text">none</output-dir>
- <expected-error>ASX1130: Illegal use of RIGHT OUTER JOIN</expected-error>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="tinysocial">
- <test-case FilePath="tinysocial">
- <compilation-unit name="tinysocial-suite">
- <output-dir compare="Text">tinysocial-suite</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="tinysocial">
- <compilation-unit name="tinysocial-suite-open">
- <output-dir compare="Text">tinysocial-suite-open</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="types">
- <test-case FilePath="types">
- <compilation-unit name="any-object">
- <output-dir compare="Text">any-object</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="gettype">
- <output-dir compare="Text">gettype</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="isarray">
- <output-dir compare="Text">isarray</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="isatomic">
- <output-dir compare="Text">isatomic</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="isboolean">
- <output-dir compare="Text">isboolean</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="isnumber">
- <output-dir compare="Text">isnumber</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="isobject">
- <output-dir compare="Text">isobject</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="isstring">
- <output-dir compare="Text">isstring</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="isbinary">
- <output-dir compare="Text">isbinary</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="iscircle">
- <output-dir compare="Text">iscircle</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="isdate">
- <output-dir compare="Text">isdate</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="isdatetime">
- <output-dir compare="Text">isdatetime</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="isduration">
- <output-dir compare="Text">isduration</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="isinterval">
- <output-dir compare="Text">isinterval</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="isline">
- <output-dir compare="Text">isline</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="ismultiset">
- <output-dir compare="Text">ismultiset</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="ispoint">
- <output-dir compare="Text">ispoint</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="ispolygon">
- <output-dir compare="Text">ispolygon</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="isrectangle">
- <output-dir compare="Text">isrectangle</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="isspatial">
- <output-dir compare="Text">isspatial</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="istemporal">
- <output-dir compare="Text">istemporal</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="istime">
- <output-dir compare="Text">istime</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="isuuid">
- <output-dir compare="Text">isuuid</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="is_all_types">
- <output-dir compare="Text">is_all_types</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="record01">
- <output-dir compare="Text">record01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="type_promotion_0">
- <output-dir compare="Text">type_promotion_0</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="type_promotion_1">
- <output-dir compare="Text">type_promotion_1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="opentype_orderby_01">
- <output-dir compare="Text">opentype_orderby_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="promotion_closedtype_field_vs_closedtype_field_01">
- <output-dir compare="Text">promotion_closedtype_field_vs_closedtype_field_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="promotion_closedtype_field_vs_closedtype_field_02">
- <output-dir compare="Text">promotion_closedtype_field_vs_closedtype_field_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="promotion_closedtype_field_vs_closedtype_field_03">
- <output-dir compare="Text">promotion_closedtype_field_vs_closedtype_field_03</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="promotion_closedtype_field_vs_closedtype_field_04">
- <output-dir compare="Text">promotion_closedtype_field_vs_closedtype_field_04</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="promotion_closedtype_field_vs_closedtype_field_05">
- <output-dir compare="Text">promotion_closedtype_field_vs_closedtype_field_05</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="promotion_closedtype_field_vs_constant_01">
- <output-dir compare="Text">promotion_closedtype_field_vs_constant_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="promotion_closedtype_field_vs_constant_02">
- <output-dir compare="Text">promotion_closedtype_field_vs_constant_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="promotion_closedtype_field_vs_constant_03">
- <output-dir compare="Text">promotion_closedtype_field_vs_constant_03</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="promotion_closedtype_field_vs_constant_04">
- <output-dir compare="Text">promotion_closedtype_field_vs_constant_04</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="promotion_closedtype_field_vs_constant_05">
- <output-dir compare="Text">promotion_closedtype_field_vs_constant_05</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="promotion_closedtype_field_vs_constant_06">
- <output-dir compare="Text">promotion_closedtype_field_vs_constant_06</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="promotion_closedtype_field_vs_constant_07">
- <output-dir compare="Text">promotion_closedtype_field_vs_constant_07</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="promotion_closedtype_field_vs_constant_08">
- <output-dir compare="Text">promotion_closedtype_field_vs_constant_08</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="promotion_closedtype_field_vs_constant_09">
- <output-dir compare="Text">promotion_closedtype_field_vs_constant_09</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="promotion_closedtype_field_vs_constant_10">
- <output-dir compare="Text">promotion_closedtype_field_vs_constant_10</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="promotion_closedtype_field_vs_constant_11">
- <output-dir compare="Text">promotion_closedtype_field_vs_constant_11</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="promotion_closedtype_field_vs_opentype_field_01">
- <output-dir compare="Text">promotion_closedtype_field_vs_opentype_field_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="promotion_closedtype_field_vs_opentype_field_02">
- <output-dir compare="Text">promotion_closedtype_field_vs_opentype_field_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="promotion_closedtype_field_vs_opentype_field_03">
- <output-dir compare="Text">promotion_closedtype_field_vs_opentype_field_03</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="promotion_closedtype_field_vs_opentype_field_04">
- <output-dir compare="Text">promotion_closedtype_field_vs_opentype_field_04</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="promotion_opentype_field_vs_constant_01">
- <output-dir compare="Text">promotion_opentype_field_vs_constant_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="promotion_opentype_field_vs_constant_02">
- <output-dir compare="Text">promotion_opentype_field_vs_constant_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="promotion_opentype_field_vs_constant_03">
- <output-dir compare="Text">promotion_opentype_field_vs_constant_03</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="promotion_opentype_field_vs_constant_04">
- <output-dir compare="Text">promotion_opentype_field_vs_constant_04</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="promotion_opentype_field_vs_constant_05">
- <output-dir compare="Text">promotion_opentype_field_vs_constant_05</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="promotion_opentype_field_vs_constant_06">
- <output-dir compare="Text">promotion_opentype_field_vs_constant_06</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="promotion_opentype_field_vs_constant_07">
- <output-dir compare="Text">promotion_opentype_field_vs_constant_07</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="promotion_opentype_field_vs_constant_08">
- <output-dir compare="Text">promotion_opentype_field_vs_constant_08</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="promotion_opentype_field_vs_opentype_field_01">
- <output-dir compare="Text">promotion_opentype_field_vs_opentype_field_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="promotion_opentype_field_vs_opentype_field_02">
- <output-dir compare="Text">promotion_opentype_field_vs_opentype_field_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="to_array">
- <output-dir compare="Text">to_array</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="to_atomic">
- <output-dir compare="Text">to_atomic</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="to_boolean_01">
- <output-dir compare="Text">to_boolean_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types" check-warnings="true">
- <compilation-unit name="to_boolean_02">
- <output-dir compare="Text">to_boolean_02</output-dir>
- <expected-warn>ASX0004: Unsupported type: to-boolean() cannot process input type date (in line 24, at column 8)</expected-warn>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="to_bigint_01">
- <output-dir compare="Text">to_bigint_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types" check-warnings="true">
- <compilation-unit name="to_bigint_02">
- <output-dir compare="Text">to_bigint_02</output-dir>
- <expected-warn>ASX0004: Unsupported type: to-bigint() cannot process input type date (in line 24, at column 7)</expected-warn>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="to_double_01">
- <output-dir compare="Text">to_double_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types" check-warnings="true">
- <compilation-unit name="to_double_02">
- <output-dir compare="Text">to_double_02</output-dir>
- <expected-warn>ASX0004: Unsupported type: to-double() cannot process input type date (in line 24, at column 7)</expected-warn>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="to_number_01">
- <output-dir compare="Text">to_number_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="to_number_02">
- <output-dir compare="Text">to_number_02</output-dir>
- <expected-error>ASX0002: Type mismatch</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="to_object">
- <output-dir compare="Text">to_object</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="to_string_01">
- <output-dir compare="Text">to_string_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="to_string_02">
- <output-dir compare="Text">to_string_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="domain_boundaries">
- <output-dir compare="Text">domain_boundaries</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="domain_boundaries_error">
- <output-dir compare="Text">domain_boundaries_error</output-dir>
- <expected-error>ASX0001: Field type null cannot be promoted to type tinyint</expected-error>
- <expected-error>ASX0001: Field type null cannot be promoted to type smallint</expected-error>
- <expected-error>ASX0001: Field type null cannot be promoted to type integer</expected-error>
- <expected-error>ASX0001: Field type null cannot be promoted to type bigint</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="types">
- <compilation-unit name="query-ASTERIXDB-2950">
- <output-dir compare="Text">none</output-dir>
- <expected-error>ASX1002: Type mismatch: function get-item expects its 1st input parameter to be of type multiset or array, but the actual input type is bigint (in line 24, at column 21)</expected-error>
- <expected-error>ASX1002: Type mismatch: function get-item expects its 1st input parameter to be of type multiset or array, but the actual input type is bigint (in line 25, at column 31)</expected-error>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="cleanjson">
- <test-case FilePath="json">
- <compilation-unit name="issue-ASTERIXDB-1165">
- <output-dir compare="Clean-JSON">issue-ASTERIXDB-1165</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="json">
- <compilation-unit name="int01">
- <output-dir compare="Clean-JSON">int01-cleanjson</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="materialization">
- <test-case FilePath="materialization">
- <compilation-unit name="assign-reuse">
- <output-dir compare="Text">assign-reuse</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="filters">
- <test-case FilePath="filters">
- <compilation-unit name="equality-predicate">
- <output-dir compare="Text">equality-predicate</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="filters">
- <compilation-unit name="filter-auto-key">
- <output-dir compare="Text">filter-auto-key</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="filters">
- <compilation-unit name="load">
- <output-dir compare="Text">load</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="filters">
- <compilation-unit name="load-with-secondary-btree">
- <output-dir compare="Text">load-with-secondary-btree</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="filters">
- <compilation-unit name="load-with-secondary-btree-index-only">
- <output-dir compare="Text">load-with-secondary-btree-index-only</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="filters">
- <compilation-unit name="load-with-secondary-inverted-ngram">
- <output-dir compare="Text">load-with-secondary-inverted-ngram</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="filters">
- <compilation-unit name="load-with-secondary-inverted-word">
- <output-dir compare="Text">load-with-secondary-inverted-word</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="filters">
- <compilation-unit name="load-with-secondary-rtree">
- <output-dir compare="Text">load-with-secondary-rtree</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="filters">
- <compilation-unit name="insert">
- <output-dir compare="Text">insert</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="filters">
- <compilation-unit name="insert-with-secondary-btree">
- <output-dir compare="Text">insert-with-secondary-btree</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="filters">
- <compilation-unit name="insert-with-correlated-secondary-btree">
- <output-dir compare="Text">insert-with-secondary-btree</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="filters">
- <compilation-unit name="insert-with-secondary-inverted-ngram">
- <output-dir compare="Text">insert-with-secondary-inverted-ngram</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="filters">
- <compilation-unit name="insert-with-correlated-secondary-inverted-ngram">
- <output-dir compare="Text">insert-with-secondary-inverted-ngram</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="filters">
- <compilation-unit name="insert-with-secondary-inverted-word">
- <output-dir compare="Text">insert-with-secondary-inverted-word</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="filters">
- <compilation-unit name="insert-with-correlated-secondary-inverted-word">
- <output-dir compare="Text">insert-with-secondary-inverted-word</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="filters">
- <compilation-unit name="insert-with-secondary-rtree">
- <output-dir compare="Text">insert-with-secondary-rtree</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="filters">
- <compilation-unit name="insert-with-correlated-secondary-rtree">
- <output-dir compare="Text">insert-with-secondary-rtree</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="filters">
- <compilation-unit name="nested-filter-equality-predicate">
- <output-dir compare="Text">nested-filter-equality-predicate</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="filters">
- <compilation-unit name="upsert">
- <output-dir compare="Text">upsert</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="filters">
- <compilation-unit name="delete">
- <output-dir compare="Text">delete</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="json">
- <test-case FilePath="json">
- <compilation-unit name="int01">
- <output-dir compare="Lossless-JSON">int01-losslessjson</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="csv">
- <test-case FilePath="csv">
- <compilation-unit name="basic-types">
- <output-dir compare="CSV">basic-types</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="csv">
- <compilation-unit name="basic-types">
- <output-dir compare="CSV_Header">basic-types-header</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="csv-tsv-parser">
- <test-case FilePath="csv-tsv-parser">
- <compilation-unit name="csv-parser-001">
- <output-dir compare="Text">csv-parser-001</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="csv-tsv-parser">
- <compilation-unit name="tsv-parser-001">
- <output-dir compare="Text">tsv-parser-001</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="binary">
- <test-case FilePath="binary">
- <compilation-unit name="parse">
- <output-dir compare="Text">parse</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="binary">
- <compilation-unit name="print">
- <output-dir compare="Text">print</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="binary">
- <compilation-unit name="concat">
- <output-dir compare="Text">concat</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="binary">
- <compilation-unit name="concat2">
- <output-dir compare="Text">concat2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="binary">
- <compilation-unit name="subbinary">
- <output-dir compare="Text">subbinary</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="binary">
- <compilation-unit name="find">
- <output-dir compare="Text">find</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="binary">
- <compilation-unit name="insert">
- <output-dir compare="Text">insert</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="binary">
- <compilation-unit name="equal_join">
- <output-dir compare="Text">equal_join</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="binary">
- <compilation-unit name="index_join">
- <output-dir compare="Text">index_join</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="binary">
- <compilation-unit name="length">
- <output-dir compare="Text">length</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="unnest">
- <test-case FilePath="unnest">
- <compilation-unit name="ASTERIXDB-2750_unnest_join">
- <output-dir compare="Text">ASTERIXDB-2750_unnest_join</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="unnest">
- <compilation-unit name="ASTERIXDB-2844_unnest_syntax">
- <output-dir compare="Text">ASTERIXDB-2844_unnest_syntax</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="unnest">
- <compilation-unit name="left-outer-unnest">
- <output-dir compare="Text">left-outer-unnest</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="unnest">
- <compilation-unit name="left-outer-unnest-with-pos">
- <output-dir compare="Text">left-outer-unnest-with-pos</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="union">
- <test-case FilePath="union">
- <compilation-unit name="union">
- <output-dir compare="Text">union</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="union">
- <compilation-unit name="union_heterogeneous_scalar">
- <output-dir compare="Text">union_heterogeneous_scalar</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="union">
- <compilation-unit name="union_negative">
- <output-dir compare="Text">union</output-dir>
- <expected-error>Cannot find dataset t in dataverse TinySocial nor an alias with name t</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="union">
- <compilation-unit name="union_negative_3">
- <output-dir compare="Text">union</output-dir>
- <expected-error>Operation UNION with set semantics is not supported.</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="union">
- <compilation-unit name="union_nested">
- <output-dir compare="Text">union_nested</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="union">
- <compilation-unit name="union_opt_1">
- <output-dir compare="Text">union_opt_1</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="union">
- <compilation-unit name="union_orderby">
- <output-dir compare="Text">union_orderby</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="union">
- <compilation-unit name="union_orderby_2">
- <output-dir compare="Text">union_orderby_2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="union">
- <compilation-unit name="union_orderby_3">
- <output-dir compare="Text">union_orderby_3</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="union">
- <compilation-unit name="union_orderby_4">
- <output-dir compare="Text">union_orderby_3</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="union">
- <compilation-unit name="union_orderby_5">
- <output-dir compare="Text">union_orderby_5</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="union">
- <compilation-unit name="query-ASTERIXDB-1354-2">
- <output-dir compare="Text">query-ASTERIXDB-1354-2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="union">
- <compilation-unit name="query-ASTERIXDB-1629">
- <output-dir compare="Text">query-ASTERIXDB-1629</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="union">
- <compilation-unit name="query-ASTERIXDB-1047">
- <output-dir compare="Text">query-ASTERIXDB-1047</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="union">
- <compilation-unit name="query-ASTERIXDB-1205-2">
- <output-dir compare="Text">query-ASTERIXDB-1205-2</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="union">
- <compilation-unit name="query-ASTERIXDB-1205-3">
- <output-dir compare="Text">query-ASTERIXDB-1205-3</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="union">
- <compilation-unit name="query-ASTERIXDB-1205">
- <output-dir compare="Text">query-ASTERIXDB-1205</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="union">
- <compilation-unit name="query-ASTERIXDB-1354">
- <output-dir compare="Text">query-ASTERIXDB-1354</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="upsert">
- <test-case FilePath="upsert">
- <compilation-unit name="filtered-dataset">
- <output-dir compare="Text">filtered-dataset</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="upsert">
- <compilation-unit name="issue1587-foreignDataType">
- <output-dir compare="Text">issue1587-foreignDataType</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="upsert">
- <compilation-unit name="nested-index">
- <output-dir compare="Text">nested-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="upsert">
- <compilation-unit name="primary-secondary-rtree">
- <output-dir compare="Text">primary-secondary-rtree</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="upsert">
- <compilation-unit name="primary-correlated-secondary-rtree">
- <output-dir compare="Text">primary-secondary-rtree</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="upsert">
- <compilation-unit name="upsert-with-self-read">
- <output-dir compare="Text">upsert-with-self-read</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="upsert">
- <compilation-unit name="nullable-index">
- <output-dir compare="Text">nullable-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="upsert">
- <compilation-unit name="open-index">
- <output-dir compare="Text">open-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="upsert">
- <compilation-unit name="primary-index">
- <output-dir compare="Text">primary-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="upsert">
- <compilation-unit name="primary-secondary-btree">
- <output-dir compare="Text">primary-secondary-btree</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="upsert">
- <compilation-unit name="primary-correlated-secondary-btree">
- <output-dir compare="Text">primary-secondary-btree</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="upsert">
- <compilation-unit name="primary-secondary-inverted">
- <output-dir compare="Text">primary-secondary-inverted</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="upsert">
- <compilation-unit name="primary-correlated-secondary-inverted">
- <output-dir compare="Text">primary-secondary-inverted</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="upsert">
- <compilation-unit name="multiple-secondaries">
- <output-dir compare="Text">multiple-secondaries</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="upsert">
- <compilation-unit name="multiple-correlated-secondaries">
- <output-dir compare="Text">multiple-secondaries</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="upsert">
- <compilation-unit name="upsert-case-returning">
- <output-dir compare="Text">upsert-case-returning</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="json-parser">
- <test-case FilePath="json-parser">
- <compilation-unit name="numeric-tinyint">
- <output-dir compare="Text">numeric-tinyint</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="json-parser">
- <compilation-unit name="numeric-tinyint-overflow">
- <output-dir compare="Text">numeric-tinyint</output-dir>
- <expected-error>Numeric value (1000) out of range of Java byte</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="json-parser">
- <compilation-unit name="numeric-float">
- <output-dir compare="Text">numeric-float</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="json-parser">
- <compilation-unit name="numeric-demote-double-bigint">
- <output-dir compare="Text">numeric-demote-double-bigint</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="json-parser">
- <compilation-unit name="numeric-promote-bigint-double">
- <output-dir compare="Text">numeric-promote-bigint-double</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="json-parser">
- <compilation-unit name="duplicate-fields">
- <output-dir compare="Text">duplicate-fields</output-dir>
- <expected-error>Duplicate field 'field'</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="json-parser">
- <compilation-unit name="malformed-json">
- <output-dir compare="Text">malformed-json</output-dir>
- <expected-error>Unexpected character ('}' (code 125)): was expecting double-quote to start field name</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="json-parser">
- <compilation-unit name="null-missing">
- <output-dir compare="Text">null-missing</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="json-parser">
- <compilation-unit name="nonoptional-missing">
- <output-dir compare="Text">nonoptional-missing</output-dir>
- <expected-error>ASX3075: Closed field missing_value has null value</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="json-parser">
- <compilation-unit name="nonoptional-null">
- <output-dir compare="Text">nonoptional-null</output-dir>
- <expected-error>ASX3075: Closed field null_value has null value</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="json-parser">
- <compilation-unit name="spatial">
- <output-dir compare="Text">spatial</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="json-parser">
- <compilation-unit name="spatial-line-3-points">
- <output-dir compare="Text">spatial-line-3-points</output-dir>
- <expected-error>Line must have 4 coordinates</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="json-parser">
- <compilation-unit name="spatial-polygon-unclosed">
- <output-dir compare="Text">spatial-polygon-unclosed</output-dir>
- <expected-error>Unclosed polygon is not supported</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="json-parser">
- <compilation-unit name="spatial-polygon-with-hole">
- <output-dir compare="Text">spatial-polygon-with-hole</output-dir>
- <expected-error>Only simple geometries are supported (Point, LineString and Polygon without holes)</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="json-parser">
- <compilation-unit name="temporal">
- <output-dir compare="Text">temporal</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="json-parser">
- <compilation-unit name="type-mismatch">
- <output-dir compare="Text">type-mismatch</output-dir>
- <expected-error>ASX3054: Mismatch Type, expecting a value of type string</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="json-parser">
- <compilation-unit name="unsupported-type-circle">
- <output-dir compare="Text">unsupported-type-circle</output-dir>
- <expected-error>ASX0004: Unsupported type: org.apache.asterix.external.parser.factory.JSONDataParserFactory cannot process input type circle</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="json-parser">
- <compilation-unit name="unsupported-type-daytimeduration">
- <output-dir compare="Text">unsupported-type-daytimeduration</output-dir>
- <expected-error>ASX0004: Unsupported type: org.apache.asterix.external.parser.factory.JSONDataParserFactory cannot process input type daytimeduration</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="json-parser">
- <compilation-unit name="unsupported-type-duration">
- <output-dir compare="Text">unsupported-type-duration</output-dir>
- <expected-error>ASX0004: Unsupported type: org.apache.asterix.external.parser.factory.JSONDataParserFactory cannot process input type duration</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="json-parser">
- <compilation-unit name="unsupported-type-interval">
- <output-dir compare="Text">unsupported-type-interval</output-dir>
- <expected-error>ASX0004: Unsupported type: org.apache.asterix.external.parser.factory.JSONDataParserFactory cannot process input type interval</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="json-parser">
- <compilation-unit name="unsupported-type-multiset">
- <output-dir compare="Text">unsupported-type-multiset</output-dir>
- <expected-error>ASX0004: Unsupported type: org.apache.asterix.external.parser.factory.JSONDataParserFactory cannot process input type multiset</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="json-parser">
- <compilation-unit name="unsupported-type-point3d">
- <output-dir compare="Text">unsupported-type-point3d</output-dir>
- <expected-error>ASX0004: Unsupported type: org.apache.asterix.external.parser.factory.JSONDataParserFactory cannot process input type point3d</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="json-parser">
- <compilation-unit name="unsupported-type-rectangle">
- <output-dir compare="Text">unsupported-type-rectangle</output-dir>
- <expected-error>ASX0004: Unsupported type: org.apache.asterix.external.parser.factory.JSONDataParserFactory cannot process input type rectangle</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="json-parser" check-warnings="true">
- <compilation-unit name="parse-json-function">
- <output-dir compare="Text">parse-json-function</output-dir>
- <source-location>false</source-location>
- <expected-warn>Type mismatch: function parse-json expects its 1st input parameter to be of type string, but the actual input type is bigint</expected-warn>
- <expected-warn>Malformed input stream</expected-warn>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="composite-key">
- <test-case FilePath="composite-key">
- <compilation-unit name="query-ASTERIXDB-920">
- <output-dir compare="Text">query-ASTERIXDB-920</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="composite-key">
- <compilation-unit name="query-ASTERIXDB-2334">
- <output-dir compare="Text">query-ASTERIXDB-2334</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="composite-key">
- <compilation-unit name="composite-low-high">
- <output-dir compare="Text">composite-low-high</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="composite-key">
- <compilation-unit name="composite-prefix">
- <output-dir compare="Text">composite-prefix</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="composite-key">
- <compilation-unit name="composite-prefix-low-high">
- <output-dir compare="Text">composite-prefix-low-high</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="limit">
- <test-case FilePath="limit">
- <compilation-unit name="limit_negative_value">
- <output-dir compare="Text">limit_negative_value</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="limit">
- <compilation-unit name="limit_type_01">
- <output-dir compare="Text">limit_type_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="limit">
- <compilation-unit name="limit_type_02">
- <output-dir compare="Text">limit_type_01</output-dir>
- <expected-error>ASX0039: Expected integer value, got 2.75 (in line 28, at column 10)</expected-error>
- <expected-error>ASX0039: Expected integer value, got 1.75 (in line 28, at column 19)</expected-error>
- <expected-error>ASX1091: Type mismatch: expected value of type integer, but got the value of type string (in line 28, at column 7)</expected-error>
- <expected-error>ASX1091: Type mismatch: expected value of type integer, but got the value of type boolean (in line 28, at column 16)</expected-error>
- <expected-error>ASX0021: Source value 9999999999 is out of range that integer can hold - integer.MAX_VALUE: 2147483647, integer.MIN_VALUE: -2147483648</expected-error>
- <expected-error>ASX0021: Source value 8888888888 is out of range that integer can hold - integer.MAX_VALUE: 2147483647, integer.MIN_VALUE: -2147483648</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="limit">
- <compilation-unit name="offset_without_limit">
- <output-dir compare="Text">offset_without_limit</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="limit">
- <compilation-unit name="push-limit-to-external-scan">
- <output-dir compare="Text">push-limit-to-external-scan</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="limit">
- <compilation-unit name="push-limit-to-external-scan-select">
- <output-dir compare="Text">push-limit-to-external-scan-select</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="limit">
- <compilation-unit name="push-limit-to-primary-scan">
- <output-dir compare="Text">push-limit-to-primary-scan</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="limit">
- <compilation-unit name="push-limit-to-primary-scan-select">
- <output-dir compare="Text">push-limit-to-primary-scan-select</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="limit">
- <compilation-unit name="push-limit-to-primary-lookup">
- <output-dir compare="Text">push-limit-to-primary-lookup</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="limit">
- <compilation-unit name="push-limit-to-primary-lookup-select">
- <output-dir compare="Text">push-limit-to-primary-lookup-select</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="limit">
- <compilation-unit name="query-ASTERIXDB-2420">
- <output-dir compare="Text">query-ASTERIXDB-2420</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="compression">
- <test-case FilePath="compression">
- <compilation-unit name="incompressible-pages/large-page">
- <output-dir compare="Text">incompressible-pages/large-page</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="compression">
- <compilation-unit name="incompressible-pages/small-page">
- <output-dir compare="Text">incompressible-pages/small-page</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="compression">
- <compilation-unit name="invalid-compression-scheme">
- <output-dir compare="Text">invalid-compression-scheme</output-dir>
- <expected-error>ASX1096: Unknown compression scheme zip. Supported schemes are [snappy,none]</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="compression">
- <compilation-unit name="scheme-none">
- <output-dir compare="Text">scheme-none</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="compression">
- <compilation-unit name="scheme-snappy">
- <output-dir compare="Text">scheme-snappy</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="ddl-with-clause">
- <test-case FilePath="ddl-with-clause">
- <compilation-unit name="missing-non-optional">
- <output-dir compare="Text">missing-non-optional</output-dir>
- <expected-error>ASX1061: Field 'merge-policy.name' in the with clause cannot be null or missing</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="ddl-with-clause">
- <compilation-unit name="type-mismatch">
- <output-dir compare="Text">type-mismatch</output-dir>
- <expected-error>ASX1060: Field 'merge-policy.parameters.max-mergable-component-size' in the with clause must be of type bigint, but found string</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="ddl-with-clause">
- <compilation-unit name="unsupported-field">
- <output-dir compare="Text">unsupported-field</output-dir>
- <expected-error>ASX1059: Field(s) [unknown-field] unsupported in the with clause</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="ddl-with-clause">
- <compilation-unit name="unsupported-subfield">
- <output-dir compare="Text">unsupported-subfield</output-dir>
- <expected-error>ASX1097: Subfield(s) [unknown-subfield] in 'merge-policy' unsupported in the with clause</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="metrics">
- <test-case FilePath="metrics">
- <compilation-unit name="full-scan">
- <output-dir compare="Text">full-scan</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="warnings">
- <test-case FilePath="warnings" check-warnings="true">
- <compilation-unit name="inapplicable-hint-warning">
- <output-dir compare="Text">inapplicable-hint-warning</output-dir>
- <expected-warn>HYR10006: Could not apply Group By hint: hash</expected-warn>
- <expected-warn>ASX1107: Unexpected hint: indexnl. "hash" expected at this location</expected-warn>
- <expected-warn>ASX1107: Unexpected hint: hash. "hash-bcast", "indexnl", "skip-index", "use-index" expected at this location</expected-warn>
- <expected-warn>ASX1107: Unexpected hint: auto. "indexnl", "skip-index", "use-index" expected at this location</expected-warn>
- <expected-warn>ASX1107: Unexpected hint: hash. "indexnl", "range", "skip-index", "spatial-partitioning", "use-index" expected at this location</expected-warn>
- <expected-warn>ASX1107: Unexpected hint: hash. None expected at this location</expected-warn>
- </compilation-unit>
- </test-case>
- <test-case FilePath="warnings" check-warnings="true">
- <compilation-unit name="min-max-incompatible-types">
- <output-dir compare="Text">min-max-incompatible-types</output-dir>
- <expected-warn>ASX0003: Type incompatibility: function min/max gets incompatible input values: bigint and string</expected-warn>
- <expected-warn>ASX0004: Unsupported type: min/max cannot process input type object</expected-warn>
- </compilation-unit>
- </test-case>
- <test-case FilePath="warnings" check-warnings="true">
- <compilation-unit name="plan-warning">
- <output-dir compare="Text">plan-warning</output-dir>
- <expected-warn>HYR10007: Encountered a cross product join</expected-warn>
- </compilation-unit>
- </test-case>
- <test-case FilePath="warnings" check-warnings="true">
- <compilation-unit name="unknown-hint-warning">
- <output-dir compare="Text">unknown-hint-warning</output-dir>
- <expected-warn>ASX1107: Unexpected hint: unknown_hint_groupby. "hash" expected at this location</expected-warn>
- <expected-warn>ASX1107: Unexpected hint: unknown_hint_relexpr. "hash-bcast", "indexnl", "skip-index", "use-index" expected at this location</expected-warn>
- <expected-warn>ASX1107: Unexpected hint: unknown_hint_between. "indexnl", "skip-index", "use-index" expected at this location</expected-warn>
- <expected-warn>ASX1107: Unexpected hint: unknown_hint_funcall. "indexnl", "range", "skip-index", "spatial-partitioning", "use-index" expected at this location</expected-warn>
- <expected-warn>ASX1107: Unexpected hint: unknown_hint_elsewhere. None expected at this location</expected-warn>
- <expected-warn>ASX1107: Unexpected hint: unknown_hint_relexpr_6. "hash-bcast", "indexnl", "skip-index", "use-index" expected at this location</expected-warn>
- <expected-warn>ASX1107: Unexpected hint: unknown_hint_relexpr_6. "hash-bcast", "indexnl", "skip-index", "use-index" expected at this location</expected-warn>
- </compilation-unit>
- </test-case>
- <test-case FilePath="warnings">
- <compilation-unit name="warnings-limit">
- <output-dir compare="Clean-JSON">warnings-limit</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="nonpure">
- <test-case FilePath="nonpure">
- <compilation-unit name="global-datetime-use-index">
- <output-dir compare="Text">global-datetime-use-index</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="nonpure">
- <compilation-unit name="local-datetime-ignore-index">
- <output-dir compare="Text">local-datetime-ignore-index</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="orderby_limit">
- <test-case FilePath="orderby_limit">
- <compilation-unit name="limit_on_variable_01">
- <output-dir compare="Text">limit_on_variable_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="orderby_limit">
- <compilation-unit name="orderby_limit_01">
- <output-dir compare="Text">orderby_limit_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="orderby_limit">
- <compilation-unit name="orderby_limit_02">
- <output-dir compare="Text">orderby_limit_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="orderby_limit">
- <compilation-unit name="orderby_limit_offset_01">
- <output-dir compare="Text">orderby_limit_offset_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="orderby_limit">
- <compilation-unit name="orderby_limit_primary_index_01">
- <output-dir compare="Text">orderby_limit_primary_index_01</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="fun_return_null_missing/string_fun">
- <test-case FilePath="fun_return_null_missing/string_fun" check-warnings="true">
- <compilation-unit name="string_fun_001">
- <output-dir compare="Text">string_fun_001</output-dir>
- <expected-warn>Type mismatch: function trim expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 41, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function like expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 32, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function uppercase expects its 1st input parameter to be of type string, but the actual input type is smallint (in line 38, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function ltrim expects its 1st input parameter to be of type string, but the actual input type is smallint (in line 42, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function like expects its 1st input parameter to be of type string, but the actual input type is tinyint (in line 33, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function string-length expects its 1st input parameter to be of type string, but the actual input type is integer (in line 36, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function trim expects its 1st input parameter to be of type string, but the actual input type is integer (in line 40, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function lowercase expects its 1st input parameter to be of type string, but the actual input type is smallint (in line 37, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function rtrim expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 45, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function contains expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 34, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function ltrim expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 43, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function position expects its 1st input parameter to be of type string, but the actual input type is tinyint (in line 46, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function string-to-codepoint expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 35, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function initcap expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 39, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function rtrim expects its 1st input parameter to be of type string, but the actual input type is integer (in line 44, at column 1)</expected-warn>
-
- <expected-warn>Type mismatch: function trim expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 41, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function like expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 32, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function uppercase expects its 1st input parameter to be of type string, but the actual input type is smallint (in line 38, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function ltrim expects its 1st input parameter to be of type string, but the actual input type is smallint (in line 42, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function like expects its 1st input parameter to be of type string, but the actual input type is tinyint (in line 33, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function string-length expects its 1st input parameter to be of type string, but the actual input type is integer (in line 36, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function trim expects its 1st input parameter to be of type string, but the actual input type is integer (in line 40, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function lowercase expects its 1st input parameter to be of type string, but the actual input type is smallint (in line 37, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function rtrim expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 45, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function contains expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 34, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function ltrim expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 43, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function position expects its 1st input parameter to be of type string, but the actual input type is tinyint (in line 46, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function string-to-codepoint expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 35, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function initcap expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 39, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function rtrim expects its 1st input parameter to be of type string, but the actual input type is integer (in line 44, at column 1)</expected-warn>
-
- <expected-warn>Type mismatch: function trim expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 41, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function like expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 32, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function uppercase expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 38, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function ltrim expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 42, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function like expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 33, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function string-length expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 36, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function trim expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 40, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function lowercase expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 37, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function rtrim expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 45, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function contains expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 34, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function ltrim expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 43, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function position expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 46, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function string-to-codepoint expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 35, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function initcap expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 39, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function rtrim expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 44, at column 1)</expected-warn>
-
- <expected-warn>Type mismatch: function regexp-position expects its 1st input parameter to be of type string, but the actual input type is integer (in line 38, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function replace expects its 1st input parameter to be of type string, but the actual input type is smallint (in line 42, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function starts-with expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 32, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function reverse expects its 1st input parameter to be of type string, but the actual input type is integer (in line 43, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function regexp-like expects its 3rd input parameter to be of type string, but the actual input type is tinyint (in line 37, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function string-equal expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 41, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function substring-after expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 44, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function matches expects its 2nd input parameter to be of type string, but the actual input type is integer (in line 34, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function regexp-like expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 36, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function regexp-position expects its 3rd input parameter to be of type string, but the actual input type is tinyint (in line 39, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function substring-before expects its 1st input parameter to be of type string, but the actual input type is tinyint (in line 45, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function regexp-replace expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 40, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function matches expects its 3rd input parameter to be of type string, but the actual input type is tinyint (in line 35, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function split expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 46, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function ends-with expects its 1st input parameter to be of type string, but the actual input type is smallint (in line 33, at column 1)</expected-warn>
-
- <expected-warn>Type mismatch: function regexp-position expects its 1st input parameter to be of type string, but the actual input type is integer (in line 38, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function replace expects its 1st input parameter to be of type string, but the actual input type is smallint (in line 42, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function starts-with expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 32, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function reverse expects its 1st input parameter to be of type string, but the actual input type is integer (in line 43, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function regexp-like expects its 3rd input parameter to be of type string, but the actual input type is tinyint (in line 37, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function string-equal expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 41, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function substring-after expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 44, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function matches expects its 2nd input parameter to be of type string, but the actual input type is integer (in line 34, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function regexp-like expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 36, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function regexp-position expects its 3rd input parameter to be of type string, but the actual input type is tinyint (in line 39, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function substring-before expects its 1st input parameter to be of type string, but the actual input type is tinyint (in line 45, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function regexp-replace expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 40, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function matches expects its 3rd input parameter to be of type string, but the actual input type is tinyint (in line 35, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function split expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 46, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function ends-with expects its 1st input parameter to be of type string, but the actual input type is smallint (in line 33, at column 1)</expected-warn>
-
- <expected-warn>Type mismatch: function regexp-position expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 38, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function replace expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 42, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function starts-with expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 32, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function reverse expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 43, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function regexp-like expects its 3rd input parameter to be of type string, but the actual input type is bigint (in line 37, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function string-equal expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 41, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function substring-after expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 44, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function matches expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 34, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function regexp-like expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 36, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function regexp-position expects its 3rd input parameter to be of type string, but the actual input type is bigint (in line 39, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function substring-before expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 45, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function regexp-replace expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 40, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function matches expects its 3rd input parameter to be of type string, but the actual input type is bigint (in line 35, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function split expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 46, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function ends-with expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 33, at column 1)</expected-warn>
- </compilation-unit>
- </test-case>
- <test-case FilePath="fun_return_null_missing/string_fun" check-warnings="true">
- <compilation-unit name="string_fun_002">
- <output-dir compare="Text">string_fun_002</output-dir>
- <expected-warn>Type mismatch: function rtrim expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 42, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function regexp-replace expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 52, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function regexp-like expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 48, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function position expects its 1st input parameter to be of type string, but the actual input type is tinyint (in line 43, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function matches expects its 2nd input parameter to be of type string, but the actual input type is integer (in line 46, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function replace expects its 1st input parameter to be of type string, but the actual input type is smallint (in line 54, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function starts-with expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 44, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function regexp-position expects its 3rd input parameter to be of type string, but the actual input type is tinyint (in line 51, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function rtrim expects its 1st input parameter to be of type string, but the actual input type is integer (in line 41, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function substring-after expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 56, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function substring-before expects its 1st input parameter to be of type string, but the actual input type is tinyint (in line 57, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function lowercase expects its 1st input parameter to be of type string, but the actual input type is smallint (in line 34, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function ends-with expects its 1st input parameter to be of type string, but the actual input type is smallint (in line 45, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function trim expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 38, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function uppercase expects its 1st input parameter to be of type string, but the actual input type is smallint (in line 35, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function regexp-like expects its 3rd input parameter to be of type string, but the actual input type is tinyint (in line 49, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function matches expects its 3rd input parameter to be of type string, but the actual input type is tinyint (in line 47, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function string-equal expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 53, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function split expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 58, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function string-to-codepoint expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 32, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function trim expects its 1st input parameter to be of type string, but the actual input type is integer (in line 37, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function reverse expects its 1st input parameter to be of type string, but the actual input type is integer (in line 55, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function like expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 29, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function contains expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 31, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function ltrim expects its 1st input parameter to be of type string, but the actual input type is smallint (in line 39, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function initcap expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 36, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function ltrim expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 40, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function string-length expects its 1st input parameter to be of type string, but the actual input type is integer (in line 33, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function like expects its 1st input parameter to be of type string, but the actual input type is tinyint (in line 30, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function regexp-position expects its 1st input parameter to be of type string, but the actual input type is integer (in line 50, at column 1)</expected-warn>
- </compilation-unit>
- </test-case>
- <test-case FilePath="fun_return_null_missing/string_fun" check-warnings="true">
- <compilation-unit name="string_fun_003">
- <output-dir compare="Text">string_fun_003</output-dir>
- <expected-warn>Invalid value: function repeat expects its 2nd input parameter to be an integer value, got 5.3 (in line 31, at column 1)</expected-warn>
- <expected-warn>Invalid value: function regexp-replace expects its 4th input parameter to be an integer value, got 5.3 (in line 36, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function substring expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 37, at column 1)</expected-warn>
- <expected-warn>Invalid value: function replace expects its 4th input parameter to be an integer value, got 5.3 (in line 34, at column 1)</expected-warn>
- <expected-warn>Invalid value: function repeat expects its 2nd input parameter to be a non-negative value, got -2.0 (in line 30, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function string-join expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 32, at column 1)</expected-warn>
-
- <expected-warn>Invalid value: function repeat expects its 2nd input parameter to be an integer value, got 5.3 (in line 31, at column 1)</expected-warn>
- <expected-warn>Invalid value: function regexp-replace expects its 4th input parameter to be an integer value, got 5.3 (in line 36, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function substring expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 37, at column 1)</expected-warn>
- <expected-warn>Invalid value: function replace expects its 4th input parameter to be an integer value, got 5.3 (in line 34, at column 1)</expected-warn>
- <expected-warn>Invalid value: function repeat expects its 2nd input parameter to be a non-negative value, got -2.0 (in line 30, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function string-join expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 32, at column 1)</expected-warn>
-
- <expected-warn>Invalid value: function repeat expects its 2nd input parameter to be an integer value, got Infinity (in line 29, at column 1)</expected-warn>
- <expected-warn>Invalid value: function replace expects its 4th input parameter to be an integer value, got -Infinity (in line 30, at column 1)</expected-warn>
- <expected-warn>Invalid value: function replace expects its 4th input parameter to be an integer value, got NaN (in line 31, at column 1)</expected-warn>
- <expected-warn>Invalid value: function regexp-replace expects its 4th input parameter to be an integer value, got NaN (in line 32, at column 1)</expected-warn>
- <expected-warn>Invalid value: function regexp-replace expects its 4th input parameter to be an integer value, got -Infinity (in line 33, at column 1)</expected-warn>
- <expected-warn>Invalid value: function substring expects its 2nd input parameter to be an integer value, got Infinity (in line 34, at column 1)</expected-warn>
- </compilation-unit>
- </test-case>
- <test-case FilePath="fun_return_null_missing/string_fun" check-warnings="true">
- <compilation-unit name="string_fun_004">
- <output-dir compare="Text">string_fun_004</output-dir>
- <expected-warn>Type mismatch: function string-concat expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 30, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function substring expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string (in line 32, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function substring expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 33, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function string-concat expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 31, at column 7)</expected-warn>
- <expected-warn>Type mismatch: function codepoint-to-string expects its 1st input parameter to be of type array, but the actual input type is string (in line 34, at column 1)</expected-warn>
- <expected-warn>Unsupported type: codepoint-to-string cannot process input type string (in line 35, at column 1)</expected-warn>
-
- <expected-warn>Type mismatch: function string-concat expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 30, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function substring expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string (in line 32, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function substring expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 33, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function string-concat expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 31, at column 7)</expected-warn>
- <expected-warn>Type mismatch: function codepoint-to-string expects its 1st input parameter to be of type array, but the actual input type is string (in line 34, at column 1)</expected-warn>
- <expected-warn>Unsupported type: codepoint-to-string cannot process input type string (in line 35, at column 1)</expected-warn>
-
- <expected-warn>Type mismatch: function string-concat expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 30, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function string-concat expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 31, at column 7)</expected-warn>
- <expected-warn>Type mismatch: function substring expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string (in line 32, at column 1)</expected-warn>
- <expected-warn>Type mismatch: function codepoint-to-string expects its 1st input parameter to be of type array, but the actual input type is string (in line 33, at column 1)</expected-warn>
- <expected-warn>Unsupported type: codepoint-to-string cannot process input type string (in line 34, at column 1)</expected-warn>
- <expected-warn>Unsupported type: codepoint-to-string cannot process input type double (in line 35, at column 1)</expected-warn>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="fun_return_null_missing/numeric_fun" >
- <test-case FilePath="fun_return_null_missing/numeric_fun" check-warnings="true">
- <compilation-unit name="numeric_fun_001">
- <output-dir compare="Text">numeric_fun_001</output-dir>
- <expected-warn>Type mismatch: function abs expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function acos expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string </expected-warn>
- <expected-warn>Type mismatch: function asin expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function atan expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function atan2 expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function degrees expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function radians expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function cos expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function cosh expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function sin expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function sinh expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function tan expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function tanh expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function exp expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function ln expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function log expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function sqrt expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function sign expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function ceiling expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function floor expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function round-half-to-even expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function numeric-unary-minus expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function random expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
-
- <expected-warn>Type mismatch: function abs expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function acos expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string </expected-warn>
- <expected-warn>Type mismatch: function asin expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function atan expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function atan2 expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function degrees expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function radians expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function cos expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function cosh expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function sin expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function sinh expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function tan expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function tanh expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function exp expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function ln expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function log expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function sqrt expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function sign expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function ceiling expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function floor expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function round-half-to-even expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function numeric-unary-minus expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function random expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
-
- <expected-warn>Type mismatch: function abs expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function acos expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string </expected-warn>
- <expected-warn>Type mismatch: function asin expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function atan expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function atan2 expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function degrees expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function radians expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function cos expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function cosh expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function sin expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function sinh expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function tan expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function tanh expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function exp expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function ln expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function log expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function sqrt expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function sign expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function ceiling expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function floor expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function round-half-to-even expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function numeric-unary-minus expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function random expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="fun_return_null_missing/numeric_fun" check-warnings="true">
- <compilation-unit name="numeric_fun_002">
- <output-dir compare="Text">numeric_fun_002</output-dir>
- <expected-warn>Type mismatch: function round expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function round-half-to-even expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function trunc expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function round-half-to-even expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function trunc expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function round expects its 2nd input parameter to be of type tinyint, smallint, integer or bigint, but the actual input type is string</expected-warn>
-
- <expected-warn>Type mismatch: function round expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function round-half-to-even expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function trunc expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function round expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function round-half-to-even expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function trunc expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
-
- <expected-warn>Type mismatch: function round expects its 2nd input parameter to be of type tinyint, smallint, integer or bigint, but the actual input type is string (in line 33, at column 24)</expected-warn>
- <expected-warn>Type mismatch: function round expects its 2nd input parameter to be of type tinyint, smallint, integer or bigint, but the actual input type is string (in line 33, at column 47)</expected-warn>
- <expected-warn>Type mismatch: function trunc expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string (in line 34, at column 24)</expected-warn>
- <expected-warn>Type mismatch: function trunc expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string (in line 34, at column 47)</expected-warn>
-
- <expected-warn>Type mismatch: function round-half-to-even expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string (in line 36, at column 40)</expected-warn>
- <expected-warn>Type mismatch: function round-half-to-even expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string (in line 36, at column 79)</expected-warn>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="fun_return_null_missing/numeric_fun" check-warnings="true">
- <compilation-unit name="numeric_fun_003">
- <output-dir compare="Text">numeric_fun_003</output-dir>
- <expected-warn>Type mismatch: function numeric-add expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function numeric-divide expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
- <expected-warn>Type incompatibility: function numeric-multiply gets incompatible input values: daytimeduration and string</expected-warn>
- <expected-warn>Type mismatch: function numeric-add expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function numeric-subtract expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
- <expected-warn>Type incompatibility: function numeric-divide gets incompatible input values: time and string</expected-warn>
- <expected-warn>Type incompatibility: function numeric-add gets incompatible input values: yearmonthduration and string</expected-warn>
- <expected-warn>Type mismatch: function numeric-subtract expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function numeric-multiply expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
- <expected-warn>Type incompatibility: function numeric-add gets incompatible input values: daytimeduration and string</expected-warn>
- <expected-warn>Type mismatch: function numeric-div expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function numeric-subtract expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function power expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function numeric-mod expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
- <expected-warn>Type incompatibility: function numeric-add gets incompatible input values: time and string</expected-warn>
- <expected-warn>Type incompatibility: function power gets incompatible input values: yearmonthduration and string</expected-warn>
- <expected-warn>Type mismatch: function numeric-multiply expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function numeric-multiply expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function numeric-mod expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function numeric-divide expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function numeric-div expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
- <expected-warn>Type incompatibility: function numeric-multiply gets incompatible input values: date and string</expected-warn>
- <expected-warn>Type mismatch: function numeric-add expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
- <expected-warn>Type incompatibility: function numeric-add gets incompatible input values: duration and string</expected-warn>
- <expected-warn>Type incompatibility: function numeric-div gets incompatible input values: datetime and string</expected-warn>
- <expected-warn>Type mismatch: function numeric-mod expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function power expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
- <expected-warn>Type incompatibility: function numeric-add gets incompatible input values: date and string</expected-warn>
- <expected-warn>Type incompatibility: function numeric-mod gets incompatible input values: duration and string</expected-warn>
- <expected-warn>Type mismatch: function numeric-subtract expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function numeric-subtract expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function numeric-divide expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function numeric-subtract expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function numeric-subtract expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function numeric-div expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function numeric-subtract expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
- <expected-warn>Type incompatibility: function numeric-add gets incompatible input values: datetime and string</expected-warn>
- <expected-warn>Type mismatch: function power expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="fun_return_null_missing" check-warnings="true">
- <compilation-unit name="field-access">
- <output-dir compare="Text">field-access</output-dir>
- <expected-warn>Type mismatch: function field-access-by-name expects its 1st input parameter to be of type object, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function field-access-by-name expects its 1st input parameter to be of type object, but the actual input type is string</expected-warn>
- <expected-warn>Type mismatch: function field-access-by-name expects its 1st input parameter to be of type object, but the actual input type is string</expected-warn>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- </test-group>
- <test-group name="window">
- <test-case FilePath="window">
- <compilation-unit name="cume_dist_01">
- <output-dir compare="Text">cume_dist_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="window">
- <compilation-unit name="dense_rank_01">
- <output-dir compare="Text">dense_rank_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="window">
- <compilation-unit name="first_value_01">
- <output-dir compare="Text">first_value_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="window">
- <compilation-unit name="lag_01">
- <output-dir compare="Text">lag_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="window">
- <compilation-unit name="last_value_01">
- <output-dir compare="Text">last_value_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="window">
- <compilation-unit name="lead_01">
- <output-dir compare="Text">lead_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="window">
- <compilation-unit name="misc_01">
- <output-dir compare="Text">misc_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="window">
- <compilation-unit name="nth_value_01">
- <output-dir compare="Text">nth_value_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="window">
- <compilation-unit name="ntile_01">
- <output-dir compare="Text">ntile_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="window">
- <compilation-unit name="percent_rank_01">
- <output-dir compare="Text">percent_rank_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="window">
- <compilation-unit name="pg_win">
- <output-dir compare="Text">pg_win</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="window">
- <compilation-unit name="rank_01">
- <output-dir compare="Text">rank_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="window">
- <compilation-unit name="ratio_to_report_01">
- <output-dir compare="Text">ratio_to_report_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="window">
- <compilation-unit name="row_number_01">
- <output-dir compare="Text">row_number_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="window">
- <compilation-unit name="win_negative">
- <output-dir compare="Text">misc_01</output-dir>
- <expected-error>ASX0002: Type mismatch</expected-error>
- <expected-error>ASX1104: Invalid modifier FROM FIRST/LAST for function</expected-error>
- <expected-error>ASX1037: Invalid query parameter compiler.windowmemory</expected-error>
- <expected-error>ASX1102: Expected window or aggregate function, got: lowercase</expected-error>
- <expected-error>ASX1079: Compilation error: count is a SQL-92 aggregate function</expected-error>
- <expected-error>ASX1104: Invalid modifier RESPECT/IGNORE NULLS for function</expected-error>
- <expected-error>ASX1104: Invalid modifier RESPECT/IGNORE NULLS for function</expected-error>
- <expected-error>ASX1104: Invalid modifier FROM FIRST/LAST for function</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="window">
- <compilation-unit name="win_null_missing">
- <output-dir compare="Text">win_null_missing</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="window">
- <compilation-unit name="win_opt_01">
- <output-dir compare="Text">win_opt_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="window">
- <compilation-unit name="win_opt_02">
- <output-dir compare="Text">win_opt_02</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
+ &SqlppQueries;
</test-suite>
diff --git a/asterixdb/asterix-benchmark/pom.xml b/asterixdb/asterix-benchmark/pom.xml
index cffee2e..4573d13 100644
--- a/asterixdb/asterix-benchmark/pom.xml
+++ b/asterixdb/asterix-benchmark/pom.xml
@@ -21,7 +21,7 @@
<parent>
<artifactId>apache-asterixdb</artifactId>
<groupId>org.apache.asterix</groupId>
- <version>0.9.8-SNAPSHOT</version>
+ <version>0.9.8.2-SNAPSHOT</version>
</parent>
<artifactId>asterix-benchmark</artifactId>
diff --git a/asterixdb/asterix-client-helper/pom.xml b/asterixdb/asterix-client-helper/pom.xml
index 32f0722..131a2ee 100644
--- a/asterixdb/asterix-client-helper/pom.xml
+++ b/asterixdb/asterix-client-helper/pom.xml
@@ -22,7 +22,7 @@
<parent>
<groupId>org.apache.asterix</groupId>
<artifactId>apache-asterixdb</artifactId>
- <version>0.9.8-SNAPSHOT</version>
+ <version>0.9.8.2-SNAPSHOT</version>
</parent>
<properties>
<root.dir>${basedir}/..</root.dir>
diff --git a/asterixdb/asterix-common/pom.xml b/asterixdb/asterix-common/pom.xml
index e76fb5b..f2c7c16 100644
--- a/asterixdb/asterix-common/pom.xml
+++ b/asterixdb/asterix-common/pom.xml
@@ -21,7 +21,7 @@
<parent>
<artifactId>apache-asterixdb</artifactId>
<groupId>org.apache.asterix</groupId>
- <version>0.9.8-SNAPSHOT</version>
+ <version>0.9.8.2-SNAPSHOT</version>
</parent>
<artifactId>asterix-common</artifactId>
<licenses>
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/annotations/ExternalSubpathAnnotation.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/annotations/ExternalSubpathAnnotation.java
new file mode 100644
index 0000000..e3e5ea3
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/annotations/ExternalSubpathAnnotation.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.common.annotations;
+
+import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionAnnotation;
+
+public final class ExternalSubpathAnnotation implements IExpressionAnnotation {
+
+ private final String subPath;
+
+ public ExternalSubpathAnnotation(String subPath) {
+ this.subPath = subPath == null ? "" : subPath.trim();
+ }
+
+ public String getSubPath() {
+ return subPath;
+ }
+}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IApplicationContext.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IApplicationContext.java
index 67f8253..f4e241c 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IApplicationContext.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IApplicationContext.java
@@ -31,6 +31,7 @@
import org.apache.hyracks.api.application.IServiceContext;
import org.apache.hyracks.api.client.IHyracksClientConnection;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.result.IResultSet;
public interface IApplicationContext {
@@ -67,6 +68,11 @@
IHyracksClientConnection getHcc() throws HyracksDataException;
/**
+ * @return a result set provider associated with {@link IHyracksClientConnection}
+ */
+ IResultSet getResultSet() throws HyracksDataException;
+
+ /**
* @return the cluster coordination service.
*/
ICoordinationService getCoordinationService();
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IClientRequest.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IClientRequest.java
index 921fb64..ee518dd 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IClientRequest.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IClientRequest.java
@@ -18,8 +18,16 @@
*/
package org.apache.asterix.common.api;
+import java.util.List;
+
import org.apache.asterix.common.dataflow.ICcApplicationContext;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.api.job.JobStatus;
+import org.apache.hyracks.api.job.resource.IJobCapacityController;
+import org.apache.hyracks.api.job.resource.IReadOnlyClusterCapacity;
+
+import com.fasterxml.jackson.databind.node.ObjectNode;
public interface IClientRequest {
@@ -86,7 +94,44 @@
void cancel(ICcApplicationContext appCtx) throws HyracksDataException;
/**
- * @return A json representation of this request
+ * @return A json string representation of this request
*/
String toJson();
+
+ /**
+ * @return A json node representation of this request
+ */
+ ObjectNode asJson();
+
+ /**
+ * @return A redacted json node representation of this request
+ */
+ ObjectNode asRedactedJson();
+
+ /**
+ * Called when the job is created.
+ *
+ * @param jobId the job id
+ * @param requiredClusterCapacity the required resources by the job
+ * @param status the status of the job; whether it will be executed or queued
+ */
+ void jobCreated(JobId jobId, IReadOnlyClusterCapacity requiredClusterCapacity,
+ IJobCapacityController.JobSubmissionStatus status);
+
+ /**
+ * Called when the job starts running.
+ *
+ * @param jobId the job id
+ */
+ void jobStarted(JobId jobId);
+
+ /**
+ * Called when the job finishes.
+ *
+ * @param jobId the job id
+ * @param jobStatus the final job status
+ * @param exceptions exceptions encountered if any
+ */
+ void jobFinished(JobId jobId, JobStatus jobStatus, List<Exception> exceptions);
+
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IDatasetLifecycleManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IDatasetLifecycleManager.java
index 1c2a047..c7eee21 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IDatasetLifecycleManager.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IDatasetLifecycleManager.java
@@ -161,12 +161,14 @@
void flushDataset(IReplicationStrategy replicationStrategy, IntPredicate partitions) throws HyracksDataException;
/**
- * Waits for all ongoing IO operations on all open datasets that are matching {@code replicationStrategy}.
+ * Waits for all ongoing IO operations on all open datasets that are matching {@code replicationStrategy} and
+ * {@code partition}.
*
* @param replicationStrategy
+ * @param partition
* @throws HyracksDataException
*/
- void waitForIO(IReplicationStrategy replicationStrategy) throws HyracksDataException;
+ void waitForIO(IReplicationStrategy replicationStrategy, int partition) throws HyracksDataException;
/**
* @return the current datasets io stats
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/INcApplicationContext.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/INcApplicationContext.java
index 5475b97..a5ae099 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/INcApplicationContext.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/INcApplicationContext.java
@@ -20,6 +20,7 @@
import java.io.IOException;
import java.rmi.RemoteException;
+import java.util.OptionalInt;
import java.util.concurrent.Executor;
import org.apache.asterix.common.context.IStorageComponentProvider;
@@ -147,4 +148,6 @@
* @return the disk write rate limiter provider
*/
IDiskWriteRateLimiterProvider getDiskWriteRateLimiterProvider();
+
+ OptionalInt getMetadataPartitionId();
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IReceptionist.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IReceptionist.java
index 95ed22e..153eea9 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IReceptionist.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IReceptionist.java
@@ -34,9 +34,9 @@
/**
* Generates a {@link IClientRequest} based on the requests parameters
*
- * @param requestParameters
+ * @param requestParameters the request parameters
* @return the client request
- * @throws HyracksDataException
+ * @throws HyracksDataException HyracksDataException
*/
IClientRequest requestReceived(ICommonRequestParameters requestParameters) throws HyracksDataException;
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IRequestTracker.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IRequestTracker.java
index a3ddb30..5a99f09 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IRequestTracker.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IRequestTracker.java
@@ -21,8 +21,9 @@
import java.util.Collection;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.job.IJobLifecycleListener;
-public interface IRequestTracker {
+public interface IRequestTracker extends IJobLifecycleListener {
/**
* Starts tracking {@code request}
@@ -75,4 +76,10 @@
* @return the recently completed requests
*/
Collection<IClientRequest> getCompletedRequests();
+
+ /**
+ *
+ * @return the total number of requests since cluster start/restart
+ */
+ long getTotalNumberOfRequests();
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/cluster/IClusterStateManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/cluster/IClusterStateManager.java
index 76802d9..fa62392 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/cluster/IClusterStateManager.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/cluster/IClusterStateManager.java
@@ -172,6 +172,11 @@
AlgebricksAbsolutePartitionConstraint getClusterLocations();
/**
+ * @return the constraint representing all the partitions of the cluster sorted by node name
+ */
+ AlgebricksAbsolutePartitionConstraint getSortedClusterLocations();
+
+ /**
* @param excludePendingRemoval
* true, if the desired set shouldn't have pending removal nodes
* @return the set of participant nodes
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
index 5538a0a..5ed4621 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
@@ -23,6 +23,7 @@
import static org.apache.hyracks.control.common.config.OptionTypes.INTEGER_BYTE_UNIT;
import static org.apache.hyracks.control.common.config.OptionTypes.LONG_BYTE_UNIT;
import static org.apache.hyracks.control.common.config.OptionTypes.POSITIVE_INTEGER;
+import static org.apache.hyracks.control.common.config.OptionTypes.STRING;
import static org.apache.hyracks.util.StorageUtil.StorageUnit.KILOBYTE;
import static org.apache.hyracks.util.StorageUtil.StorageUnit.MEGABYTE;
@@ -103,7 +104,17 @@
COMPILER_ARRAYINDEX(
BOOLEAN,
AlgebricksConfig.ARRAY_INDEX_DEFAULT,
- "Enable/disable using array-indexes in queries");
+ "Enable/disable using array-indexes in queries"),
+ COMPILER_CBO(BOOLEAN, AlgebricksConfig.CBO_DEFAULT, "Set the mode for cost based optimization"),
+ COMPILER_CBOTEST(BOOLEAN, AlgebricksConfig.CBO_TEST_DEFAULT, "Set the mode for cost based optimization"),
+ COMPILER_FORCEJOINORDER(
+ BOOLEAN,
+ AlgebricksConfig.FORCE_JOIN_ORDER_DEFAULT,
+ "Set the mode for forcing the join order in a query plan"),
+ COMPILER_QUERYPLANSHAPE(
+ STRING,
+ AlgebricksConfig.QUERY_PLAN_SHAPE_DEFAULT,
+ "Set the mode for forcing the shape of the query plan");
private final IOptionType type;
private final Object defaultValue;
@@ -137,7 +148,7 @@
@Override
public boolean hidden() {
- return this == COMPILER_EXTERNALSCANMEMORY;
+ return this == COMPILER_EXTERNALSCANMEMORY || this == COMPILER_CBOTEST;
}
}
@@ -173,6 +184,14 @@
public static final String COMPILER_EXTERNALSCANMEMORY_KEY = Option.COMPILER_EXTERNALSCANMEMORY.ini();
+ public static final String COMPILER_CBO_KEY = Option.COMPILER_CBO.ini();
+
+ public static final String COMPILER_CBO_TEST_KEY = Option.COMPILER_CBOTEST.ini();
+
+ public static final String COMPILER_FORCE_JOIN_ORDER_KEY = Option.COMPILER_FORCEJOINORDER.ini();
+
+ public static final String COMPILER_QUERY_PLAN_SHAPE_KEY = Option.COMPILER_QUERYPLANSHAPE.ini();
+
public static final int COMPILER_PARALLELISM_AS_STORAGE = 0;
public CompilerProperties(PropertiesAccessor accessor) {
@@ -246,4 +265,30 @@
public int getExternalScanMemorySize() {
return accessor.getInt(Option.COMPILER_EXTERNALSCANMEMORY);
}
+
+ public boolean getCBOMode() {
+ return accessor.getBoolean(Option.COMPILER_CBO);
+ }
+
+ public boolean getCBOTestMode() {
+ return accessor.getBoolean(Option.COMPILER_CBOTEST);
+ }
+
+ public boolean getForceJoinOrderMode() {
+ return accessor.getBoolean(Option.COMPILER_FORCEJOINORDER);
+ }
+
+ public String getQueryPlanShapeMode() {
+ String queryPlanShapeMode = accessor.getString(Option.COMPILER_QUERYPLANSHAPE);
+ if (!(queryPlanShapeMode.equals(AlgebricksConfig.QUERY_PLAN_SHAPE_ZIGZAG)
+ || queryPlanShapeMode.equals(AlgebricksConfig.QUERY_PLAN_SHAPE_LEFTDEEP)
+ || queryPlanShapeMode.equals(AlgebricksConfig.QUERY_PLAN_SHAPE_RIGHTDEEP)))
+ return AlgebricksConfig.QUERY_PLAN_SHAPE_DEFAULT;
+ return queryPlanShapeMode;
+ }
+
+ public int getSortMemoryFrames() {
+ int numFrames = (int) getSortMemorySize() / getFrameSize();
+ return Math.max(numFrames, OptimizationConfUtil.MIN_FRAME_LIMIT_FOR_SORT);
+ }
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/DatasetConfig.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/DatasetConfig.java
index b85c0be..478bd46 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/DatasetConfig.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/DatasetConfig.java
@@ -41,7 +41,8 @@
SINGLE_PARTITION_NGRAM_INVIX,
LENGTH_PARTITIONED_WORD_INVIX,
LENGTH_PARTITIONED_NGRAM_INVIX,
- ARRAY;
+ ARRAY,
+ SAMPLE;
}
public enum TransactionState {
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/OptimizationConfUtil.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/OptimizationConfUtil.java
index d498d69..9307469 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/OptimizationConfUtil.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/OptimizationConfUtil.java
@@ -34,7 +34,7 @@
public class OptimizationConfUtil {
- private static final int MIN_FRAME_LIMIT_FOR_SORT = AbstractStableSortPOperator.MIN_FRAME_LIMIT_FOR_SORT;
+ public static final int MIN_FRAME_LIMIT_FOR_SORT = AbstractStableSortPOperator.MIN_FRAME_LIMIT_FOR_SORT;
private static final int MIN_FRAME_LIMIT_FOR_GROUP_BY = AbstractGroupByPOperator.MIN_FRAME_LIMIT_FOR_GROUP_BY;
private static final int MIN_FRAME_LIMIT_FOR_JOIN = AbstractJoinPOperator.MIN_FRAME_LIMIT_FOR_JOIN;
private static final int MIN_FRAME_LIMIT_FOR_WINDOW = WindowPOperator.MIN_FRAME_LIMIT_FOR_WINDOW;
@@ -77,6 +77,14 @@
int externalScanBufferSize = getExternalScanBufferSize(
(String) querySpecificConfig.get(CompilerProperties.COMPILER_EXTERNALSCANMEMORY_KEY),
compilerProperties.getExternalScanMemorySize(), sourceLoc);
+ boolean cbo =
+ getBoolean(querySpecificConfig, CompilerProperties.COMPILER_CBO_KEY, compilerProperties.getCBOMode());
+ boolean cboTest = getBoolean(querySpecificConfig, CompilerProperties.COMPILER_CBO_TEST_KEY,
+ compilerProperties.getCBOTestMode());
+ boolean forceJoinOrder = getBoolean(querySpecificConfig, CompilerProperties.COMPILER_FORCE_JOIN_ORDER_KEY,
+ compilerProperties.getForceJoinOrderMode());
+ String queryPlanShape = getString(querySpecificConfig, CompilerProperties.COMPILER_QUERY_PLAN_SHAPE_KEY,
+ compilerProperties.getQueryPlanShapeMode());
PhysicalOptimizationConfig physOptConf = new PhysicalOptimizationConfig();
physOptConf.setFrameSize(frameSize);
@@ -95,6 +103,10 @@
physOptConf.setMinMemoryAllocation(minMemoryAllocation);
physOptConf.setArrayIndexEnabled(arrayIndex);
physOptConf.setExternalScanBufferSize(externalScanBufferSize);
+ physOptConf.setCBOMode(cbo);
+ physOptConf.setCBOTestMode(cboTest);
+ physOptConf.setForceJoinOrderMode(forceJoinOrder);
+ physOptConf.setQueryPlanShapeMode(queryPlanShape);
return physOptConf;
}
@@ -117,6 +129,14 @@
sourceLoc);
}
+ public static int getGroupByNumFrames(CompilerProperties compilerProperties,
+ Map<String, Object> querySpecificConfig, SourceLocation sourceLoc) throws AlgebricksException {
+ return getFrameLimit(CompilerProperties.COMPILER_GROUPMEMORY_KEY,
+ (String) querySpecificConfig.get(CompilerProperties.COMPILER_GROUPMEMORY_KEY),
+ compilerProperties.getGroupMemorySize(), compilerProperties.getFrameSize(),
+ MIN_FRAME_LIMIT_FOR_GROUP_BY, sourceLoc);
+ }
+
public static int getTextSearchNumFrames(CompilerProperties compilerProperties,
Map<String, Object> querySpecificConfig, SourceLocation sourceLoc) throws AlgebricksException {
return getFrameLimit(CompilerProperties.COMPILER_TEXTSEARCHMEMORY_KEY,
@@ -164,4 +184,12 @@
}
return defaultValue;
}
+
+ private static String getString(Map<String, Object> queryConfig, String queryConfigKey, String defaultValue) {
+ String valueInQuery = (String) queryConfig.get(queryConfigKey);
+ if (valueInQuery != null) {
+ return valueInQuery;
+ }
+ return defaultValue;
+ }
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/StorageProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/StorageProperties.java
index 12c9c68..6264f64 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/StorageProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/StorageProperties.java
@@ -74,6 +74,7 @@
@Override
public Section section() {
switch (this) {
+ case STORAGE_BUFFERCACHE_PAGESIZE:
case STORAGE_COMPRESSION_BLOCK:
case STORAGE_LSM_BLOOMFILTER_FALSEPOSITIVERATE:
case STORAGE_GLOBAL_CLEANUP:
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/BaseOperationTracker.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/BaseOperationTracker.java
index 5964bb4..7d3dba4 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/BaseOperationTracker.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/BaseOperationTracker.java
@@ -33,17 +33,19 @@
private static final Logger LOGGER = LogManager.getLogger();
protected final int datasetID;
protected final DatasetInfo dsInfo;
+ protected final int partition;
- public BaseOperationTracker(int datasetID, DatasetInfo dsInfo) {
+ public BaseOperationTracker(int datasetID, DatasetInfo dsInfo, int partition) {
this.datasetID = datasetID;
this.dsInfo = dsInfo;
+ this.partition = partition;
}
@Override
public void beforeOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
IModificationOperationCallback modificationCallback) throws HyracksDataException {
if (opType == LSMOperationType.REPLICATE) {
- dsInfo.declareActiveIOOperation(REPLICATE);
+ dsInfo.declareActiveIOOperation(REPLICATE, partition);
}
}
@@ -59,7 +61,7 @@
public void completeOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
IModificationOperationCallback modificationCallback) throws HyracksDataException {
if (opType == LSMOperationType.REPLICATE) {
- dsInfo.undeclareActiveIOOperation(REPLICATE);
+ dsInfo.undeclareActiveIOOperation(REPLICATE, partition);
}
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetInfo.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetInfo.java
index d15d9be..87a3c2f 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetInfo.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetInfo.java
@@ -33,12 +33,16 @@
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
+import it.unimi.dsi.fastutil.ints.Int2IntMap;
+import it.unimi.dsi.fastutil.ints.Int2IntOpenHashMap;
+
public class DatasetInfo extends Info implements Comparable<DatasetInfo> {
private static final Logger LOGGER = LogManager.getLogger();
// partition -> index
private final Map<Integer, Set<IndexInfo>> partitionIndexes;
// resourceID -> index
private final Map<Long, IndexInfo> indexes;
+ private final Int2IntMap partitionPendingIO;
private final int datasetID;
private final ILogManager logManager;
private final LogRecord waitLog = new LogRecord();
@@ -54,6 +58,7 @@
public DatasetInfo(int datasetID, ILogManager logManager) {
this.partitionIndexes = new HashMap<>();
this.indexes = new HashMap<>();
+ this.partitionPendingIO = new Int2IntOpenHashMap();
this.setLastAccess(-1);
this.datasetID = datasetID;
this.setRegistered(false);
@@ -74,7 +79,8 @@
setLastAccess(System.currentTimeMillis());
}
- public synchronized void declareActiveIOOperation(ILSMIOOperation.LSMIOOperationType opType) {
+ public synchronized void declareActiveIOOperation(ILSMIOOperation.LSMIOOperationType opType, int partition) {
+ partitionPendingIO.put(partition, partitionPendingIO.getOrDefault(partition, 0) + 1);
numActiveIOOps++;
switch (opType) {
case FLUSH:
@@ -91,7 +97,8 @@
}
}
- public synchronized void undeclareActiveIOOperation(ILSMIOOperation.LSMIOOperationType opType) {
+ public synchronized void undeclareActiveIOOperation(ILSMIOOperation.LSMIOOperationType opType, int partition) {
+ partitionPendingIO.put(partition, partitionPendingIO.getOrDefault(partition, 0) - 1);
numActiveIOOps--;
switch (opType) {
case FLUSH:
@@ -253,6 +260,26 @@
}
}
+ public void waitForIO(int partition) throws HyracksDataException {
+ logManager.log(waitLog);
+ synchronized (this) {
+ while (partitionPendingIO.getOrDefault(partition, 0) > 0) {
+ try {
+ wait();
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ throw HyracksDataException.create(e);
+ }
+ }
+ if (partitionPendingIO.getOrDefault(partition, 0) < 0) {
+ LOGGER.error("number of IO operations cannot be negative for dataset {}, partition {}", this,
+ partition);
+ throw new IllegalStateException(
+ "Number of IO operations cannot be negative: " + this + ", partition " + partition);
+ }
+ }
+ }
+
public synchronized int getPendingFlushes() {
return pendingFlushes;
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java
index 117b4fc..4fc9dd6 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java
@@ -552,10 +552,10 @@
}
@Override
- public void waitForIO(IReplicationStrategy replicationStrategy) throws HyracksDataException {
+ public void waitForIO(IReplicationStrategy replicationStrategy, int partition) throws HyracksDataException {
for (DatasetResource dsr : datasets.values()) {
if (dsr.isOpen() && replicationStrategy.isMatch(dsr.getDatasetID())) {
- dsr.getDatasetInfo().waitForIO();
+ dsr.getDatasetInfo().waitForIO(partition);
}
}
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/GlobalVirtualBufferCache.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/GlobalVirtualBufferCache.java
index 0bb9bd5..1565730 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/GlobalVirtualBufferCache.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/GlobalVirtualBufferCache.java
@@ -173,13 +173,16 @@
// 2. there are still some active readers and memory cannot be reclaimed.
// But for both cases, we will notify all primary index op trackers to let their writers retry,
// if they have been blocked. Moreover, we will check whether more flushes are needed.
+ List<ILSMOperationTracker> opTrackers = new ArrayList<>();
synchronized (this) {
final int size = primaryIndexes.size();
for (int i = 0; i < size; i++) {
- ILSMOperationTracker opTracker = primaryIndexes.get(i).getOperationTracker();
- synchronized (opTracker) {
- opTracker.notifyAll();
- }
+ opTrackers.add(primaryIndexes.get(i).getOperationTracker());
+ }
+ }
+ for (ILSMOperationTracker opTracker : opTrackers) {
+ synchronized (opTracker) {
+ opTracker.notifyAll();
}
}
checkAndNotifyFlushThread();
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/PrimaryIndexOperationTracker.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/PrimaryIndexOperationTracker.java
index 8491f6c..9c2b656 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/PrimaryIndexOperationTracker.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/PrimaryIndexOperationTracker.java
@@ -58,7 +58,6 @@
@NotThreadSafe
public class PrimaryIndexOperationTracker extends BaseOperationTracker implements IoOperationCompleteListener {
private static final Logger LOGGER = LogManager.getLogger();
- private final int partition;
// Number of active operations on an ILSMIndex instance.
private final AtomicInteger numActiveOperations;
private final ILogManager logManager;
@@ -70,8 +69,7 @@
public PrimaryIndexOperationTracker(int datasetID, int partition, ILogManager logManager, DatasetInfo dsInfo,
ILSMComponentIdGenerator idGenerator) {
- super(datasetID, dsInfo);
- this.partition = partition;
+ super(datasetID, dsInfo, partition);
this.logManager = logManager;
this.numActiveOperations = new AtomicInteger();
this.idGenerator = idGenerator;
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/DatasetLocalResource.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/DatasetLocalResource.java
index 249505b..58d5f9b 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/DatasetLocalResource.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/DatasetLocalResource.java
@@ -72,6 +72,10 @@
this.partition = partition;
}
+ public IResource getResource() {
+ return resource;
+ }
+
@Override
public IIndex createInstance(INCServiceContext ncServiceCtx) throws HyracksDataException {
return resource.createInstance(ncServiceCtx);
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/NoOpFrameOperationCallbackFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/NoOpFrameOperationCallbackFactory.java
index 915f3b3..3ab86c0 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/NoOpFrameOperationCallbackFactory.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/NoOpFrameOperationCallbackFactory.java
@@ -47,6 +47,11 @@
}
@Override
+ public void beforeExit(boolean success) throws HyracksDataException {
+ // No Op
+ }
+
+ @Override
public void close() throws IOException {
// No Op
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
index fc234ef..4910343 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
@@ -87,6 +87,7 @@
PARQUET_CONTAINS_OVERFLOWED_BIGINT(57),
UNEXPECTED_ERROR_ENCOUNTERED(58),
INVALID_PARQUET_FILE(59),
+ FUNCTION_EVALUATION_FAILED(60),
UNSUPPORTED_JRE(100),
@@ -265,6 +266,11 @@
S3_REGION_NOT_SUPPORTED(1170),
COMPILATION_SET_OPERATION_ERROR(1171),
INVALID_TIMEZONE(1172),
+ INVALID_PARAM_VALUE_ALLOWED_VALUE(1173),
+ SAMPLE_HAS_ZERO_ROWS(1174),
+ INVALID_SAMPLE_SIZE(1175),
+ OUT_OF_RANGE_SAMPLE_SIZE(1176),
+ INVALID_SAMPLE_SEED(1177),
// Feed errors
DATAFLOW_ILLEGAL_STATE(3001),
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/NoOpWarningCollector.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/NoOpWarningCollector.java
index a036b7e..6ac805b 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/NoOpWarningCollector.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/NoOpWarningCollector.java
@@ -20,27 +20,10 @@
package org.apache.asterix.common.exceptions;
import org.apache.hyracks.api.exceptions.IWarningCollector;
-import org.apache.hyracks.api.exceptions.Warning;
-public final class NoOpWarningCollector implements IWarningCollector {
-
- public static final IWarningCollector INSTANCE = new NoOpWarningCollector();
+public final class NoOpWarningCollector {
+ public static final IWarningCollector INSTANCE = org.apache.hyracks.api.exceptions.NoOpWarningCollector.INSTANCE;
private NoOpWarningCollector() {
}
-
- @Override
- public void warn(Warning warning) {
- // no-op
- }
-
- @Override
- public boolean shouldWarn() {
- return false;
- }
-
- @Override
- public long getTotalWarningsCount() {
- return 0;
- }
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMIOOperationCallback.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMIOOperationCallback.java
index 1189b51..f56e5c0 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMIOOperationCallback.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMIOOperationCallback.java
@@ -74,6 +74,7 @@
private final IIndexCheckpointManagerProvider indexCheckpointManagerProvider;
protected final DatasetInfo dsInfo;
protected final ILSMIndex lsmIndex;
+ private final int partition;
private long firstLsnForCurrentMemoryComponent = 0L;
private long persistenceLsn = 0L;
private int pendingFlushes = 0;
@@ -84,6 +85,7 @@
this.dsInfo = dsInfo;
this.lsmIndex = lsmIndex;
this.indexCheckpointManagerProvider = indexCheckpointManagerProvider;
+ this.partition = ResourceReference.ofIndex(lsmIndex.getIndexIdentifier()).getPartitionNum();
componentIds.add(componentId);
}
@@ -259,7 +261,7 @@
@Override
public synchronized void scheduled(ILSMIOOperation operation) throws HyracksDataException {
- dsInfo.declareActiveIOOperation(operation.getIOOpertionType());
+ dsInfo.declareActiveIOOperation(operation.getIOOpertionType(), partition);
if (operation.getIOOpertionType() == LSMIOOperationType.FLUSH) {
pendingFlushes++;
FlushOperation flush = (FlushOperation) operation;
@@ -282,7 +284,7 @@
pendingFlushes == 0 ? firstLsnForCurrentMemoryComponent : (Long) map.get(KEY_FLUSH_LOG_LSN);
}
}
- dsInfo.undeclareActiveIOOperation(operation.getIOOpertionType());
+ dsInfo.undeclareActiveIOOperation(operation.getIOOpertionType(), partition);
}
public synchronized boolean hasPendingFlush() {
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/library/ILibraryManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/library/ILibraryManager.java
index 93fe92d..acbce6d 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/library/ILibraryManager.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/library/ILibraryManager.java
@@ -24,13 +24,16 @@
import java.net.URI;
import java.security.MessageDigest;
import java.util.List;
+import java.util.function.Function;
import org.apache.asterix.common.metadata.DataverseName;
import org.apache.asterix.external.ipc.ExternalFunctionResultRouter;
+import org.apache.http.impl.client.CloseableHttpClient;
import org.apache.hyracks.algebricks.common.utils.Pair;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.exceptions.HyracksException;
import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.control.nc.NodeControllerService;
import org.apache.hyracks.ipc.impl.IPCSystem;
public interface ILibraryManager {
@@ -59,9 +62,13 @@
IPCSystem getIPCI();
+ NodeControllerService getNcs();
+
MessageDigest download(FileReference targetFile, String authToken, URI libLocation) throws HyracksException;
void unzip(FileReference sourceFile, FileReference outputDir) throws IOException;
void writeAndForce(FileReference outputFile, InputStream dataStream, byte[] copyBuf) throws IOException;
+
+ void setUploadClient(Function<ILibraryManager, CloseableHttpClient> f);
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/IMetadataLockUtil.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/IMetadataLockUtil.java
index 28eb553..d830868 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/IMetadataLockUtil.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/IMetadataLockUtil.java
@@ -52,6 +52,12 @@
void refreshDatasetBegin(IMetadataLockManager lockManager, LockList locks, DataverseName dataverseName,
String datasetName) throws AlgebricksException;
+ void analyzeDatasetBegin(IMetadataLockManager lockManager, LockList locks, DataverseName dataverseName,
+ String datasetName) throws AlgebricksException;
+
+ void analyzeDatasetDropBegin(IMetadataLockManager lockMgr, LockList locks, DataverseName dataverseName,
+ String datasetName) throws AlgebricksException;
+
void compactBegin(IMetadataLockManager lockManager, LockList locks, DataverseName dataverseName, String datasetName)
throws AlgebricksException;
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/AllDatasetsReplicationStrategy.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/AllDatasetsReplicationStrategy.java
index c969777..84f0a39 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/AllDatasetsReplicationStrategy.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/AllDatasetsReplicationStrategy.java
@@ -19,6 +19,7 @@
package org.apache.asterix.common.replication;
public class AllDatasetsReplicationStrategy implements IReplicationStrategy {
+ public static final IReplicationStrategy INSTANCE = new AllDatasetsReplicationStrategy();
@Override
public boolean isMatch(int datasetId) {
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/IIndexCheckpointManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/IIndexCheckpointManager.java
index beb8e07..9777879 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/IIndexCheckpointManager.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/IIndexCheckpointManager.java
@@ -140,4 +140,12 @@
* @throws HyracksDataException
*/
void setLastComponentId(long componentId) throws HyracksDataException;
+
+ /**
+ * Indicates that the index has at least one valid checkpoint
+ *
+ * @return true if the index has at least one valid checkpoint. Otherwise false
+ * @throws HyracksDataException
+ */
+ boolean isValidIndex() throws HyracksDataException;
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/JobUtils.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/JobUtils.java
index 232c8dd..c1c6f18 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/JobUtils.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/JobUtils.java
@@ -20,11 +20,14 @@
package org.apache.asterix.common.utils;
import java.util.EnumSet;
+import java.util.List;
+import org.apache.hyracks.algebricks.common.utils.Pair;
import org.apache.hyracks.api.client.IHyracksClientConnection;
import org.apache.hyracks.api.job.JobFlag;
import org.apache.hyracks.api.job.JobId;
import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
public class JobUtils {
@@ -53,4 +56,21 @@
}
return jobId;
}
+
+ public static Pair<JobId, List<IOperatorStats>> runJob(IHyracksClientConnection hcc, JobSpecification spec,
+ EnumSet<JobFlag> jobFlags, boolean waitForCompletion, List<String> statOperatorNames) throws Exception {
+ spec.setMaxReattempts(0);
+ final JobId jobId = hcc.startJob(spec, jobFlags);
+ List<IOperatorStats> opStats = null;
+ if (waitForCompletion) {
+ String nameBefore = Thread.currentThread().getName();
+ try {
+ Thread.currentThread().setName(nameBefore + " : WaitForCompletionForJobId: " + jobId);
+ opStats = hcc.waitForCompletion(jobId, statOperatorNames);
+ } finally {
+ Thread.currentThread().setName(nameBefore);
+ }
+ }
+ return new Pair<>(jobId, opStats);
+ }
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StoragePathUtil.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StoragePathUtil.java
index f84472e..c87f368 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StoragePathUtil.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StoragePathUtil.java
@@ -46,9 +46,13 @@
private StoragePathUtil() {
}
+ public static IFileSplitProvider splitProvider(FileSplit[] splits) {
+ return new ConstantFileSplitProvider(splits);
+ }
+
public static Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitProviderAndPartitionConstraints(
FileSplit[] splits) {
- IFileSplitProvider splitProvider = new ConstantFileSplitProvider(splits);
+ IFileSplitProvider splitProvider = splitProvider(splits);
String[] loc = new String[splits.length];
for (int p = 0; p < splits.length; p++) {
loc[p] = splits[p].getNodeName();
diff --git a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
index b98dc43..074245c 100644
--- a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
+++ b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
@@ -94,6 +94,7 @@
57 = Parquet file(s) contain unsigned integer that is larger than the '%1$s' range
58 = Error encountered: %1$s
59 = Invalid Parquet file: %1$s. Reason: %2$s
+60 = Function '%1$s' failed to evaluate because: %2$s
100 = Unsupported JRE: %1$s
@@ -267,7 +268,11 @@
1170 = Provided S3 region is not supported: '%1$s'
1171 = Unable to process %1$s clause. %2$s
1172 = Provided timezone is invalid: '%1$s'
-
+1173 = Invalid value for parameter '%1$s', allowed value(s): %2$s
+1174 = Sample has zero rows
+1175 = Sample size options are "low", "medium", "high", or a number
+1176 = Sample size has to be between %1$s and %2$s
+1177 = Sample seed has to be a number or a string convertible to a number
# Feed Errors
3001 = Illegal state.
3002 = Tuple is too large for a frame
diff --git a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMIOOperationCallbackTest.java b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMIOOperationCallbackTest.java
index db0911b..33d513f 100644
--- a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMIOOperationCallbackTest.java
+++ b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMIOOperationCallbackTest.java
@@ -77,6 +77,7 @@
String indexId = "mockIndexId";
Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(numMemoryComponents);
Mockito.when(mockIndex.getCurrentMemoryComponent()).thenReturn(Mockito.mock(AbstractLSMMemoryComponent.class));
+ Mockito.when(mockIndex.getIndexIdentifier()).thenReturn(getIndexPath());
DatasetInfo dsInfo = new DatasetInfo(101, null);
LSMComponentIdGenerator idGenerator = new LSMComponentIdGenerator(numMemoryComponents, MIN_VALID_COMPONENT_ID);
LSMIOOperationCallback callback = new LSMIOOperationCallback(dsInfo, mockIndex, idGenerator.getId(),
@@ -140,6 +141,7 @@
ILSMComponentIdGenerator idGenerator = new LSMComponentIdGenerator(numMemoryComponents, MIN_VALID_COMPONENT_ID);
ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(numMemoryComponents);
+ Mockito.when(mockIndex.getIndexIdentifier()).thenReturn(getIndexPath());
ILSMMemoryComponent mockComponent = Mockito.mock(AbstractLSMMemoryComponent.class);
Mockito.when(mockIndex.getCurrentMemoryComponent()).thenReturn(mockComponent);
LSMIOOperationCallback callback = new LSMIOOperationCallback(dsInfo, mockIndex, idGenerator.getId(),
@@ -161,6 +163,7 @@
ILSMComponentIdGenerator idGenerator = new LSMComponentIdGenerator(numMemoryComponents, MIN_VALID_COMPONENT_ID);
ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(numMemoryComponents);
+ Mockito.when(mockIndex.getIndexIdentifier()).thenReturn(getIndexPath());
ILSMMemoryComponent mockComponent = Mockito.mock(AbstractLSMMemoryComponent.class);
Mockito.when(mockIndex.getCurrentMemoryComponent()).thenReturn(mockComponent);
LSMIOOperationCallback callback = new LSMIOOperationCallback(dsInfo, mockIndex, idGenerator.getId(),
@@ -221,4 +224,8 @@
Mockito.doReturn(indexCheckpointManager).when(indexCheckpointManagerProvider).get(Mockito.any());
return indexCheckpointManagerProvider;
}
+
+ private static String getIndexPath() {
+ return "storage/partition_0/dataverse/dataset/0/index";
+ }
}
diff --git a/asterixdb/asterix-coverage/pom.xml b/asterixdb/asterix-coverage/pom.xml
index 19b73b9..02107f2 100644
--- a/asterixdb/asterix-coverage/pom.xml
+++ b/asterixdb/asterix-coverage/pom.xml
@@ -26,7 +26,7 @@
<parent>
<groupId>org.apache.asterix</groupId>
<artifactId>apache-asterixdb</artifactId>
- <version>0.9.8-SNAPSHOT</version>
+ <version>0.9.8.2-SNAPSHOT</version>
</parent>
<properties>
diff --git a/asterixdb/asterix-dashboard/pom.xml b/asterixdb/asterix-dashboard/pom.xml
index 9d962f8..7c9041f 100644
--- a/asterixdb/asterix-dashboard/pom.xml
+++ b/asterixdb/asterix-dashboard/pom.xml
@@ -22,7 +22,7 @@
<parent>
<artifactId>apache-asterixdb</artifactId>
<groupId>org.apache.asterix</groupId>
- <version>0.9.8-SNAPSHOT</version>
+ <version>0.9.8.2-SNAPSHOT</version>
</parent>
<modelVersion>4.0.0</modelVersion>
<artifactId>asterix-dashboard</artifactId>
diff --git a/asterixdb/asterix-doc/pom.xml b/asterixdb/asterix-doc/pom.xml
index d0825d4..23925b9 100644
--- a/asterixdb/asterix-doc/pom.xml
+++ b/asterixdb/asterix-doc/pom.xml
@@ -21,7 +21,7 @@
<parent>
<artifactId>apache-asterixdb</artifactId>
<groupId>org.apache.asterix</groupId>
- <version>0.9.8-SNAPSHOT</version>
+ <version>0.9.8.2-SNAPSHOT</version>
</parent>
<url>http://asterixdb.apache.org/</url>
diff --git a/asterixdb/asterix-doc/src/main/markdown/builtins/9_aggregate_sql.md b/asterixdb/asterix-doc/src/main/markdown/builtins/9_aggregate_sql.md
index 0657fb0..755fd39 100644
--- a/asterixdb/asterix-doc/src/main/markdown/builtins/9_aggregate_sql.md
+++ b/asterixdb/asterix-doc/src/main/markdown/builtins/9_aggregate_sql.md
@@ -48,8 +48,8 @@
* or, a `missing` value.
* Return Value:
* a `bigint` value representing the number of non-null and non-missing items in the given collection,
- * `null` is returned if the input is `null` or `missing`,
- * any other non-array and non-multiset input value will cause an error.
+ * `0` is returned if the input is `null` or `missing`,
+ * `0` is returned if the input is not an array or a multiset.
* Example:
@@ -77,8 +77,8 @@
* a `double` value representing the average of the non-null and non-missing numbers in the given collection,
* `null` is returned if the input is `null` or `missing`,
* `null` is returned if the given collection does not contain any non-null and non-missing items,
- * any other non-array and non-multiset input value will cause a type error,
- * any other non-numeric value in the input collection will cause a type error.
+ * `null` is returned if the input is not an array or a multiset,
+ * any other non-numeric value in the input collection will be ignored.
* Example:
@@ -107,8 +107,8 @@
items.
* `null` is returned if the input is `null` or `missing`,
* `null` is returned if the given collection does not contain any non-null and non-missing items,
- * any other non-array and non-multiset input value will cause a type error,
- * any other non-numeric value in the input collection will cause a type error.
+ * `null` is returned if the input is not an array or a multiset,
+ * any other non-numeric value in the input collection will be ignored.
* Example:
@@ -136,8 +136,8 @@
type promotion order (`tinyint`-> `smallint`->`integer`->`bigint`->`float`->`double`) among numeric items.
* `null` is returned if the input is `null` or `missing`,
* `null` is returned if the given collection does not contain any non-null and non-missing items,
- * multiple incomparable items in the input array or multiset will cause a type error,
- * any other non-array and non-multiset input value will cause a type error.
+ * `null` is returned if there are incomparable items in the input array or multiset,
+ * `null` is returned if the input is not an array or a multiset.
* Example:
@@ -165,8 +165,8 @@
type promotion order (`tinyint`-> `smallint`->`integer`->`bigint`->`float`->`double`) among numeric items.
* `null` is returned if the input is `null` or `missing`,
* `null` is returned if the given collection does not contain any non-null and non-missing items,
- * multiple incomparable items in the input array or multiset will cause a type error,
- * any other non-array and non-multiset input value will cause a type error.
+ * `null` is returned if there are incomparable items in the input array or multiset,
+ * `null` is returned if the input is not an array or a multiset.
* Example:
@@ -177,6 +177,44 @@
3.4
+### array_median ###
+ * Syntax:
+
+ array_median(num_collection)
+
+ * Gets the median value of the numeric items in the given collection, ignoring null, missing, and non-numeric items.
+
+ The function starts by sorting the numeric items.
+
+ - If there is an odd number of numeric items, the function returns the item that is exactly in the middle of the range: that is, it has the same number of items before and after.
+ - If there is an even number of numeric items, the function returns the mean of the two items that are exactly in the middle of the range.
+
+ * Note: You cannot use the `DISTINCT` keyword with this function, or with the `median` aggregation pseudo-function.
+ The `median` aggregation pseudo-function does support the `FILTER` clause.
+ There is no `strict_median` function corresponding to this function.
+ * Arguments:
+ * `num_collection` could be:
+ * an `array` or `multiset` of numbers,
+ * or, a `null` value,
+ * or, a `missing` value.
+ * Clauses: When used as a window function, this function supports the [Window Partition Clause](manual.html#Window_partition_clause), but not the [Window Order Clause](manual.html#Window_order_clause) or the [Window Frame Clause](manual.html#Window_frame_clause).
+ * Return Value:
+ * a `double` value representing the median of the numeric items in the given collection,
+ * `null` is returned if the input is `null` or `missing`,
+ * `null` is returned if the given collection does not contain any numeric items,
+ * `null` is returned if the input is not an array or a multiset,
+ * any other non-numeric value in the input collection will be ignored.
+ * Example:
+
+ { "v1": array_median( [1.2, 2.3, 3.4, 0, null, missing],
+ "v2": array_median( [1.2, 2.3, 3.4, 4.5, 0, null, missing] ) };
+
+ * The expected result is:
+
+ { "v1": 1.75,
+ "v2": 2.3 }
+
+
### array_stddev_samp ###
* Syntax:
@@ -193,7 +231,7 @@
* a `double` value representing the sample standard deviation of the non-null and non-missing numbers in the given collection,
* `null` is returned if the input is `null` or `missing`,
* `null` is returned if the given collection does not contain any non-null and non-missing items,
- * any other non-array and non-multiset input value will cause a type error,
+ * `null` is returned if the input is not an array or a multiset,
* any other non-numeric value in the input collection will cause a type error.
* Example:
@@ -220,7 +258,7 @@
* a `double` value representing the population standard deviation of the non-null and non-missing numbers in the given collection,
* `null` is returned if the input is `null` or `missing`,
* `null` is returned if the given collection does not contain any non-null and non-missing items,
- * any other non-array and non-multiset input value will cause a type error,
+ * `null` is returned if the input is not an array or a multiset,
* any other non-numeric value in the input collection will cause a type error.
* Example:
@@ -247,7 +285,7 @@
* a `double` value representing the sample variance of the non-null and non-missing numbers in the given collection,
* `null` is returned if the input is `null` or `missing`,
* `null` is returned if the given collection does not contain any non-null and non-missing items,
- * any other non-array and non-multiset input value will cause a type error,
+ * `null` is returned if the input is not an array or a multiset,
* any other non-numeric value in the input collection will cause a type error.
* Example:
@@ -274,7 +312,7 @@
* a `double` value representing the population variance of the non-null and non-missing numbers in the given collection,
* `null` is returned if the input is `null` or `missing`,
* `null` is returned if the given collection does not contain any non-null and non-missing items,
- * any other non-array and non-multiset input value will cause a type error,
+ * `null` is returned if the input is not an array or a multiset,
* any other non-numeric value in the input collection will cause a type error.
* Example:
@@ -301,7 +339,7 @@
* a `double` value representing the skewness of the non-null and non-missing numbers in the given collection,
* `null` is returned if the input is `null` or `missing`,
* `null` is returned if the given collection does not contain any non-null and non-missing items,
- * any other non-array and non-multiset input value will cause a type error,
+ * `null` is returned if the input is not an array or a multiset,
* any other non-numeric value in the input collection will cause a type error.
* Example:
@@ -328,7 +366,7 @@
* a `double` value representing the kurtosis from a normal distribution of the non-null and non-missing numbers in the given collection,
* `null` is returned if the input is `null` or `missing`,
* `null` is returned if the given collection does not contain any non-null and non-missing items,
- * any other non-array and non-multiset input value will cause a type error,
+ * `null` is returned if the input is not an array or a multiset,
* any other non-numeric value in the input collection will cause a type error.
* Example:
@@ -352,7 +390,8 @@
* or a `missing` value.
* Return Value:
* a `bigint` value representing the number of items in the given collection,
- * `null` is returned if the input is `null` or `missing`.
+ * `0` is returned if the input is `null` or `missing`,
+ * `0` is returned if the input is not an array or a multiset.
* Example:
@@ -377,7 +416,8 @@
* a `double` value representing the average of the numbers in the given collection,
* `null` is returned if the input is `null` or `missing`,
* `null` is returned if there is a `null` or `missing` in the input collection,
- * any other non-numeric value in the input collection will cause a type error.
+ * `null` is returned if the input is not an array or a multiset,
+ * `null` is returned if there are any other non-numeric values in the input collection.
* Example:
@@ -404,7 +444,8 @@
items.
* `null` is returned if the input is `null` or `missing`,
* `null` is returned if there is a `null` or `missing` in the input collection,
- * any other non-numeric value in the input collection will cause a type error.
+ * `null` is returned if the input is not an array or a multiset,
+ * `null` is returned if there are any other non-numeric values in the input collection.
* Example:
@@ -431,8 +472,8 @@
(`tinyint`-> `smallint`->`integer`->`bigint`->`float`->`double`) among numeric items.
* `null` is returned if the input is `null` or `missing`,
* `null` is returned if there is a `null` or `missing` in the input collection,
- * multiple incomparable items in the input array or multiset will cause a type error,
- * any other non-array and non-multiset input value will cause a type error.
+ * `null` is returned if there are incomparable items in the input array or multiset,
+ * `null` is returned if the input is not an array or a multiset.
* Example:
@@ -460,8 +501,8 @@
(`tinyint`-> `smallint`->`integer`->`bigint`->`float`->`double`) among numeric items.
* `null` is returned if the input is `null` or `missing`,
* `null` is returned if there is a `null` or `missing` in the input collection,
- * multiple incomparable items in the input array or multiset will cause a type error,
- * any other non-array and non-multiset input value will cause a type error.
+ * `null` is returned if there are incomparable items in the input array or multiset,
+ * `null` is returned if the input is not an array or a multiset.
* Example:
@@ -536,6 +577,7 @@
* a `double` value representing the sample variance of the numbers in the given collection,
* `null` is returned if the input is `null` or `missing`,
* `null` is returned if there is a `null` or `missing` in the input collection,
+ * `null` is returned if the input is not an array or a multiset,
* any other non-numeric value in the input collection will cause a type error.
* Example:
@@ -561,6 +603,7 @@
* a `double` value representing the population variance of the numbers in the given collection,
* `null` is returned if the input is `null` or `missing`,
* `null` is returned if there is a `null` or `missing` in the input collection,
+ * `null` is returned if the input is not an array or a multiset,
* any other non-numeric value in the input collection will cause a type error.
* Example:
@@ -586,6 +629,7 @@
* a `double` value representing the skewness of the numbers in the given collection,
* `null` is returned if the input is `null` or `missing`,
* `null` is returned if there is a `null` or `missing` in the input collection,
+ * `null` is returned if the input is not an array or a multiset,
* any other non-numeric value in the input collection will cause a type error.
* Example:
@@ -611,6 +655,7 @@
* a `double` value representing the kurtosis from a normal distribution of the numbers in the given collection,
* `null` is returned if the input is `null` or `missing`,
* `null` is returned if there is a `null` or `missing` in the input collection,
+ * `null` is returned if the input is not an array or a multiset,
* any other non-numeric value in the input collection will cause a type error.
* Example:
diff --git a/asterixdb/asterix-doc/src/main/markdown/sqlpp/3_query.md b/asterixdb/asterix-doc/src/main/markdown/sqlpp/3_query.md
index 04a65d1..2bd9f49 100644
--- a/asterixdb/asterix-doc/src/main/markdown/sqlpp/3_query.md
+++ b/asterixdb/asterix-doc/src/main/markdown/sqlpp/3_query.md
@@ -1215,20 +1215,21 @@
For example, `SELECT COUNT(*) FROM customers` simply returns the total number of customers, whereas `SELECT COUNT(rating) FROM customers` returns the number of customers who have known ratings (that is, their ratings are not `null` or `missing`).
-Because the aggregation pseudo-functions sometimes restructure their operands, they can be used only in query blocks where (explicit or implicit) grouping is being done. Therefore the pseudo-functions cannot operate directly on arrays or multisets. For operating directly on JSON collections, SQL++ provides a set of ordinary functions for computing aggregations. Each ordinary aggregation function (except the ones corresponding to `COUNT` and `ARRAY_AGG`) has two versions: one that ignores `null` and `missing` values and one that returns `null` if a `null` or `missing` value is encountered anywhere in the collection. The names of the aggregation functions are as follows:
+Because the aggregation pseudo-functions sometimes restructure their operands, they can be used only in query blocks where (explicit or implicit) grouping is being done. Therefore the pseudo-functions cannot operate directly on arrays or multisets. For operating directly on JSON collections, SQL++ provides a set of ordinary functions for computing aggregations. Each ordinary aggregation function (except as noted below) has two versions: one that ignores `null` and `missing` values, and one that returns `null` if a `null` or `missing` value is encountered anywhere in the collection. The names of the aggregation functions are as follows:
| Aggregation pseudo-function; operates on groups only | Ordinary function: Ignores NULL or MISSING values | Ordinary function: Returns NULL if NULL or MISSING are encountered|
|----------|----------|--------|
-|SUM| ARRAY_SUM| STRICT_SUM |
-| AVG |ARRAY_MAX| STRICT_MAX |
-| MAX | ARRAY_MIN| STRICT_MIN |
-| MIN | ARRAY_AVG| STRICT_AVG |
+| SUM | ARRAY_SUM| STRICT_SUM |
+| AVG | ARRAY_AVG| STRICT_AVG |
+| MAX | ARRAY_MAX| STRICT_MAX |
+| MIN | ARRAY_MIN| STRICT_MIN |
| COUNT |ARRAY_COUNT|STRICT_COUNT (see exception below) |
+| MEDIAN | ARRAY_MEDIAN | |
|STDDEV_SAMP|ARRAY_STDDEV_SAMP| STRICT_STDDEV_SAMP |
|STDDEV_POP|ARRAY_STDDEV_POP| STRICT_STDDEV_POP |
|VAR_SAMP|ARRAY_VAR_SAMP| STRICT_VAR_SAMP |
|VAR_POP|ARRAY_VAR_POP| STRICT_VAR_POP |
-|SKEWENESS|ARRAY_SKEWNESS| STRICT_SKEWNESS |
+|SKEWNESS|ARRAY_SKEWNESS| STRICT_SKEWNESS |
|KURTOSIS|ARRAY_KURTOSIS| STRICT_KURTOSIS |
| |ARRAY_AGG| |
diff --git a/asterixdb/asterix-doc/src/main/markdown/sqlpp/4_windowfunctions.md b/asterixdb/asterix-doc/src/main/markdown/sqlpp/4_windowfunctions.md
index d6e40e7..c41e8a3 100644
--- a/asterixdb/asterix-doc/src/main/markdown/sqlpp/4_windowfunctions.md
+++ b/asterixdb/asterix-doc/src/main/markdown/sqlpp/4_windowfunctions.md
@@ -108,7 +108,7 @@
The *window order clause* determines how tuples are ordered within each partition. The window function works on tuples in the order specified by this clause.
-This clause may be used with any [window function](builtins.html#WindowFunctions), or any [aggregate function](builtins.html#AggregateFunctions) used as a window function.
+This clause may be used with any [window function](builtins.html#WindowFunctions), and most [aggregate functions](builtins.html#AggregateFunctions) — refer to the descriptions of individual functions for more details.
This clause is optional. If omitted, all tuples are considered peers, i.e. their order is tied. When tuples in the window partition are tied, each window function behaves differently.
@@ -130,7 +130,7 @@
##### WindowFrameClause
![](../images/diagrams/WindowFrameClause.png)
-The *window frame clause* defines the window frame. It can be used with all [aggregate functions](builtins.html#AggregateFunctions) and some [window functions](builtins.html#WindowFunctions) — refer to the descriptions of individual functions for more details. It is optional and allowed only when the [window order clause](#Window_order_clause) is present.
+The *window frame clause* defines the window frame. It can be used with some [window functions](builtins.html#WindowFunctions) and most [aggregate functions](builtins.html#AggregateFunctions) — refer to the descriptions of individual functions for more details. It is optional and allowed only when the [window order clause](#Window_order_clause) is present.
* If this clause is omitted and there is no [window order clause](#Window_order_clause), the window frame is the entire partition.
diff --git a/asterixdb/asterix-docker/pom.xml b/asterixdb/asterix-docker/pom.xml
index 6c54337..39bd8c7 100644
--- a/asterixdb/asterix-docker/pom.xml
+++ b/asterixdb/asterix-docker/pom.xml
@@ -22,7 +22,7 @@
<parent>
<artifactId>apache-asterixdb</artifactId>
<groupId>org.apache.asterix</groupId>
- <version>0.9.8-SNAPSHOT</version>
+ <version>0.9.8.2-SNAPSHOT</version>
</parent>
<artifactId>asterix-docker</artifactId>
diff --git a/asterixdb/asterix-examples/pom.xml b/asterixdb/asterix-examples/pom.xml
index 7812be0..52a4d37 100644
--- a/asterixdb/asterix-examples/pom.xml
+++ b/asterixdb/asterix-examples/pom.xml
@@ -21,7 +21,7 @@
<parent>
<artifactId>apache-asterixdb</artifactId>
<groupId>org.apache.asterix</groupId>
- <version>0.9.8-SNAPSHOT</version>
+ <version>0.9.8.2-SNAPSHOT</version>
</parent>
<artifactId>asterix-examples</artifactId>
diff --git a/asterixdb/asterix-external-data/pom.xml b/asterixdb/asterix-external-data/pom.xml
index fbb1751..ee4ac89 100644
--- a/asterixdb/asterix-external-data/pom.xml
+++ b/asterixdb/asterix-external-data/pom.xml
@@ -21,7 +21,7 @@
<parent>
<artifactId>apache-asterixdb</artifactId>
<groupId>org.apache.asterix</groupId>
- <version>0.9.8-SNAPSHOT</version>
+ <version>0.9.8.2-SNAPSHOT</version>
</parent>
<licenses>
<license>
@@ -35,13 +35,14 @@
<properties>
<root.dir>${basedir}/..</root.dir>
<generatedSourcesDirectory>${project.build.directory}/generated-sources/lexer/</generatedSourcesDirectory>
+ <lexer-generator-maven-plugin.version>${project.version}</lexer-generator-maven-plugin.version>
</properties>
<build>
<plugins>
<plugin>
<groupId>org.apache.asterix</groupId>
<artifactId>lexer-generator-maven-plugin</artifactId>
- <version>${project.version}</version>
+ <version>${lexer-generator-maven-plugin.version}</version>
<configuration>
<grammarFile>src/main/resources/adm.grammar</grammarFile>
<outputDir>${project.build.directory}/generated-sources/lexer/org/apache/asterix/runtime/operators/file/adm</outputDir>
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/GenericAdapterFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/GenericAdapterFactory.java
index ac45f7a..8ec5af0 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/GenericAdapterFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/GenericAdapterFactory.java
@@ -37,7 +37,6 @@
import org.apache.asterix.external.dataset.adapter.FeedAdapter;
import org.apache.asterix.external.dataset.adapter.GenericAdapter;
import org.apache.asterix.external.indexing.ExternalFile;
-import org.apache.asterix.external.parser.factory.ADMDataParserFactory;
import org.apache.asterix.external.provider.DataflowControllerProvider;
import org.apache.asterix.external.provider.DatasourceFactoryProvider;
import org.apache.asterix.external.provider.ParserFactoryProvider;
@@ -47,7 +46,6 @@
import org.apache.asterix.external.util.FeedLogManager;
import org.apache.asterix.external.util.FeedUtils;
import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.om.utils.RecordUtil;
import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.api.application.ICCServiceContext;
@@ -209,18 +207,18 @@
}
/**
- * Use pre-configured datasource factory
- * For function datasources
+ * Use pre-configured datasource factory For function datasources
*
* @param dataSourceFactory
- * the function datasource factory
+ * the function datasource factory
+ * @param dataParserFactory
+ * the function data parser factory
* @throws AlgebricksException
*/
- public void configure(IExternalDataSourceFactory dataSourceFactory) throws AlgebricksException {
+ public void configure(IExternalDataSourceFactory dataSourceFactory, IDataParserFactory dataParserFactory)
+ throws AlgebricksException {
this.dataSourceFactory = dataSourceFactory;
- dataParserFactory = new ADMDataParserFactory();
- dataParserFactory.setRecordType(RecordUtil.FULLY_OPEN_RECORD_TYPE);
- dataParserFactory.configure(Collections.emptyMap());
+ this.dataParserFactory = dataParserFactory;
configuration = Collections.emptyMap();
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/HDFSDataSourceFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/HDFSDataSourceFactory.java
index 419782f..b190e33 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/HDFSDataSourceFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/HDFSDataSourceFactory.java
@@ -18,6 +18,10 @@
*/
package org.apache.asterix.external.input;
+import static org.apache.asterix.external.util.ExternalDataConstants.CONTAINER_NAME_FIELD_NAME;
+import static org.apache.asterix.external.util.ExternalDataConstants.FORMAT_PARQUET;
+import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
+
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
@@ -101,6 +105,7 @@
}
protected void configureHdfsConf(JobConf conf, Map<String, String> configuration) throws AlgebricksException {
+ String formatString = configuration.get(ExternalDataConstants.KEY_FORMAT);
try {
confFactory = new ConfFactory(conf);
clusterLocations = getPartitionConstraint();
@@ -120,7 +125,6 @@
inputSplitsFactory = new InputSplitsFactory(inputSplits);
read = new boolean[readSchedule.length];
Arrays.fill(read, false);
- String formatString = configuration.get(ExternalDataConstants.KEY_FORMAT);
if (formatString == null || formatString.equals(ExternalDataConstants.FORMAT_HDFS_WRITABLE)) {
RecordReader<?, ?> reader = conf.getInputFormat().getRecordReader(inputSplits[0], conf, Reporter.NULL);
this.recordClass = reader.createValue().getClass();
@@ -132,7 +136,19 @@
this.recordClass = char[].class;
}
} catch (IOException e) {
- throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, e);
+ throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, e, getMessageOrToString(e));
+ } catch (Exception e) {
+ if (FORMAT_PARQUET.equals(formatString)) {
+ String containerName = configuration.get(CONTAINER_NAME_FIELD_NAME);
+ if (containerName != null && containerName.contains(".")) {
+ throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, e,
+ getMessageOrToString(e) + " Buckets with '.' in the name can cause issues.");
+ } else {
+ throw e;
+ }
+ } else {
+ throw e;
+ }
}
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/abstracts/AbstractExternalInputStreamFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/abstracts/AbstractExternalInputStreamFactory.java
index eac4835..9760c55 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/abstracts/AbstractExternalInputStreamFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/abstracts/AbstractExternalInputStreamFactory.java
@@ -70,8 +70,8 @@
public void configure(IServiceContext ctx, Map<String, String> configuration, IWarningCollector warningCollector)
throws AlgebricksException {
this.configuration = configuration;
- this.partitionConstraint =
- ((ICcApplicationContext) ctx.getApplicationContext()).getClusterStateManager().getClusterLocations();
+ this.partitionConstraint = ((ICcApplicationContext) ctx.getApplicationContext()).getClusterStateManager()
+ .getSortedClusterLocations();
}
public static class PartitionWorkLoadBasedOnSize implements Serializable {
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStream.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStream.java
index f14af53..6386813 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStream.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStream.java
@@ -38,6 +38,7 @@
import org.apache.hyracks.api.util.CleanupUtils;
import org.apache.hyracks.util.LogRedactionUtil;
+import software.amazon.awssdk.core.ResponseInputStream;
import software.amazon.awssdk.core.exception.SdkException;
import software.amazon.awssdk.services.s3.S3Client;
import software.amazon.awssdk.services.s3.model.GetObjectRequest;
@@ -49,6 +50,7 @@
// Configuration
private final String bucket;
private final S3Client s3Client;
+ private ResponseInputStream<?> s3InStream;
private static final int MAX_RETRIES = 5; // We will retry 5 times in case of internal error from AWS S3 service
public AwsS3InputStream(Map<String, String> configuration, List<String> filePaths) throws HyracksDataException {
@@ -83,7 +85,8 @@
int retries = 0;
while (retries < MAX_RETRIES) {
try {
- in = s3Client.getObject(request);
+ s3InStream = s3Client.getObject(request);
+ in = s3InStream;
break;
} catch (NoSuchKeyException ex) {
LOGGER.debug(() -> "Key " + LogRedactionUtil.userData(request.key()) + " was not found in bucket "
@@ -115,6 +118,9 @@
@Override
public void close() throws IOException {
if (in != null) {
+ if (s3InStream != null) {
+ s3InStream.abort();
+ }
CleanupUtils.close(in, null);
}
if (s3Client != null) {
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/GCSInputStream.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/GCSInputStream.java
index 652fa3e..5da4583 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/GCSInputStream.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/GCSInputStream.java
@@ -47,7 +47,7 @@
private final Storage client;
private final String container;
- private static final int MAX_RETRIES = 5; // We will retry 5 times in case of retryable errors
+ private static final int MAX_ATTEMPTS = 5; // We try a total of 5 times in case of retryable errors
public GCSInputStream(Map<String, String> configuration, List<String> filePaths) throws HyracksDataException {
super(configuration, filePaths);
@@ -78,10 +78,10 @@
* @return true
*/
private boolean doGetInputStream(String fileName) throws RuntimeDataException {
- int retries = 0;
+ int attempt = 0;
BlobId blobId = BlobId.of(container, fileName);
- while (retries < MAX_RETRIES) {
+ while (attempt < MAX_ATTEMPTS) {
try {
Blob blob = client.get(blobId);
if (blob == null) {
@@ -93,14 +93,14 @@
in = new ByteArrayInputStream(blob.getContent());
break;
} catch (BaseServiceException ex) {
- if (!shouldRetry(retries++) && ex.isRetryable()) {
+ if (!ex.isRetryable() || !shouldRetry(++attempt)) {
throw new RuntimeDataException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
}
- LOGGER.debug(() -> "Retryable error: " + LogRedactionUtil.userData(ex.getMessage()));
+ LOGGER.debug(() -> "Retryable error: " + getMessageOrToString(ex));
- // Backoff for 1 sec for the first 2 retries, and 2 seconds from there onward
+ // Backoff for 1 sec for the first 3 attempts, and 2 seconds from there onward
try {
- Thread.sleep(TimeUnit.SECONDS.toMillis(retries < 3 ? 1 : 2));
+ Thread.sleep(TimeUnit.SECONDS.toMillis(attempt < 3 ? 1 : 2));
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
}
@@ -111,8 +111,8 @@
return true;
}
- private boolean shouldRetry(int currentRetry) {
- return currentRetry < MAX_RETRIES;
+ private boolean shouldRetry(int nextAttempt) {
+ return nextAttempt < MAX_ATTEMPTS;
}
@Override
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalLibraryManager.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalLibraryManager.java
index f71150a..050d9f0 100755
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalLibraryManager.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalLibraryManager.java
@@ -49,6 +49,7 @@
import java.util.List;
import java.util.Map;
import java.util.Set;
+import java.util.function.Function;
import java.util.zip.ZipEntry;
import java.util.zip.ZipFile;
@@ -77,7 +78,6 @@
import org.apache.http.conn.ssl.SSLConnectionSocketFactory;
import org.apache.http.impl.client.CloseableHttpClient;
import org.apache.http.impl.client.HttpClients;
-import org.apache.http.ssl.SSLContexts;
import org.apache.hyracks.algebricks.common.utils.Pair;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.exceptions.HyracksException;
@@ -92,6 +92,7 @@
import org.apache.hyracks.control.common.work.AbstractWork;
import org.apache.hyracks.control.nc.NodeControllerService;
import org.apache.hyracks.ipc.impl.IPCSystem;
+import org.apache.hyracks.ipc.security.NetworkSecurityManager;
import org.apache.hyracks.ipc.sockets.PlainSocketChannelFactory;
import org.apache.hyracks.util.file.FileUtil;
import org.apache.logging.log4j.LogManager;
@@ -102,7 +103,7 @@
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.SerializationFeature;
-public final class ExternalLibraryManager implements ILibraryManager, ILifeCycleComponent {
+public class ExternalLibraryManager implements ILibraryManager, ILifeCycleComponent {
public static final String LIBRARY_MANAGER_BASE_DIR_NAME = "library";
@@ -140,6 +141,7 @@
private final ExternalFunctionResultRouter router;
private final IIOManager ioManager;
private boolean sslEnabled;
+ private Function<ILibraryManager, CloseableHttpClient> uploadClientSupp;
public ExternalLibraryManager(NodeControllerService ncs, IPersistedResourceRegistry reg, FileReference appDir,
IIOManager ioManager) {
@@ -155,6 +157,7 @@
router = new ExternalFunctionResultRouter();
this.sslEnabled = ncs.getConfiguration().isSslEnabled();
this.ioManager = ioManager;
+ uploadClientSupp = ExternalLibraryManager::defaultHttpClient;
}
public void initialize(boolean resetStorageData) throws HyracksDataException {
@@ -486,6 +489,11 @@
return pythonIPC;
}
+ @Override
+ public NodeControllerService getNcs() {
+ return ncs;
+ }
+
private static final class DeleteDirectoryWork extends AbstractWork {
private final Path path;
@@ -630,30 +638,37 @@
}
}
- //TODO: this should probably be static so it could be reused somewhere else, or made such that the trust store is not
- // reloaded from disk on every client intialization?
- private CloseableHttpClient newClient() {
+ public CloseableHttpClient newClient() {
if (sslEnabled) {
- try {
- final INetworkSecurityManager networkSecurityManager = ncs.getNetworkSecurityManager();
- final INetworkSecurityConfig configuration = networkSecurityManager.getConfiguration();
- KeyStore trustStore = KeyStore.getInstance(KeyStore.getDefaultType());
- try (FileInputStream trustStoreFile = new FileInputStream(configuration.getTrustStoreFile())) {
- String ksPassword = configuration.getKeyStorePassword();
- trustStore.load(trustStoreFile,
- ksPassword == null || ksPassword.isEmpty() ? null : ksPassword.toCharArray());
- }
- SSLContext sslcontext = SSLContexts.custom().loadTrustMaterial(trustStore, null).build();
- SSLConnectionSocketFactory sslsf = new SSLConnectionSocketFactory(sslcontext,
- new String[] { "TLSv1.2" }, null, SSLConnectionSocketFactory.getDefaultHostnameVerifier());
- return HttpClients.custom().setSSLSocketFactory(sslsf).build();
-
- } catch (Exception e) {
- throw new IllegalStateException(e);
- }
+ return uploadClientSupp.apply(this);
} else {
return HttpClients.createDefault();
}
}
+ @Override
+ public void setUploadClient(Function<ILibraryManager, CloseableHttpClient> f) {
+ uploadClientSupp = f;
+ }
+
+ private static CloseableHttpClient defaultHttpClient(ILibraryManager extLib) {
+ try {
+ final INetworkSecurityManager networkSecurityManager = extLib.getNcs().getNetworkSecurityManager();
+ final INetworkSecurityConfig configuration = networkSecurityManager.getConfiguration();
+ KeyStore trustStore = KeyStore.getInstance(KeyStore.getDefaultType());
+ try (FileInputStream trustStoreFile = new FileInputStream(configuration.getTrustStoreFile())) {
+ String ksPassword = configuration.getKeyStorePassword();
+ trustStore.load(trustStoreFile,
+ ksPassword == null || ksPassword.isEmpty() ? null : ksPassword.toCharArray());
+ }
+ SSLContext sslcontext = NetworkSecurityManager.newSSLContext(configuration);
+ SSLConnectionSocketFactory sslsf = new SSLConnectionSocketFactory(sslcontext, new String[] { "TLSv1.2" },
+ null, SSLConnectionSocketFactory.getDefaultHostnameVerifier());
+ return HttpClients.custom().setSSLSocketFactory(sslsf).build();
+
+ } catch (Exception e) {
+ throw new IllegalStateException(e);
+ }
+ }
+
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java
index f0b9c90..050a080 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java
@@ -297,6 +297,7 @@
public static final String DEFINITION_FIELD_NAME = "definition";
public static final String CONTAINER_NAME_FIELD_NAME = "container";
+ public static final String SUBPATH = "subpath";
public static class ParquetOptions {
private ParquetOptions() {
@@ -434,6 +435,7 @@
throw new AssertionError("do not instantiate");
}
+ public static final String APPLICATION_DEFAULT_CREDENTIALS_FIELD_NAME = "applicationDefaultCredentials";
public static final String JSON_CREDENTIALS_FIELD_NAME = "jsonCredentials";
}
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
index 517a393..c879ec6 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
@@ -21,6 +21,7 @@
import static com.google.cloud.storage.Storage.BlobListOption;
import static java.nio.charset.StandardCharsets.UTF_8;
import static org.apache.asterix.common.exceptions.ErrorCode.EXTERNAL_SOURCE_ERROR;
+import static org.apache.asterix.common.exceptions.ErrorCode.INVALID_PARAM_VALUE_ALLOWED_VALUE;
import static org.apache.asterix.common.exceptions.ErrorCode.INVALID_REQ_PARAM_VAL;
import static org.apache.asterix.common.exceptions.ErrorCode.PARAMETERS_NOT_ALLOWED_AT_SAME_TIME;
import static org.apache.asterix.common.exceptions.ErrorCode.PARAMETERS_REQUIRED;
@@ -28,6 +29,16 @@
import static org.apache.asterix.common.exceptions.ErrorCode.REQUIRED_PARAM_IF_PARAM_IS_PRESENT;
import static org.apache.asterix.common.exceptions.ErrorCode.REQUIRED_PARAM_OR_PARAM_IF_PARAM_IS_PRESENT;
import static org.apache.asterix.common.exceptions.ErrorCode.S3_REGION_NOT_SUPPORTED;
+import static org.apache.asterix.external.util.ExternalDataConstants.KEY_ADAPTER_NAME_GCS;
+import static org.apache.asterix.external.util.ExternalDataConstants.KEY_DELIMITER;
+import static org.apache.asterix.external.util.ExternalDataConstants.KEY_ESCAPE;
+import static org.apache.asterix.external.util.ExternalDataConstants.KEY_EXCLUDE;
+import static org.apache.asterix.external.util.ExternalDataConstants.KEY_EXTERNAL_SCAN_BUFFER_SIZE;
+import static org.apache.asterix.external.util.ExternalDataConstants.KEY_FORMAT;
+import static org.apache.asterix.external.util.ExternalDataConstants.KEY_INCLUDE;
+import static org.apache.asterix.external.util.ExternalDataConstants.KEY_QUOTE;
+import static org.apache.asterix.external.util.ExternalDataConstants.KEY_RECORD_END;
+import static org.apache.asterix.external.util.ExternalDataConstants.KEY_RECORD_START;
import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.ACCESS_KEY_ID_FIELD_NAME;
import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.ERROR_METHOD_NOT_IMPLEMENTED;
import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_ACCESS_KEY_ID;
@@ -54,17 +65,8 @@
import static org.apache.asterix.external.util.ExternalDataConstants.Azure.RECURSIVE_FIELD_NAME;
import static org.apache.asterix.external.util.ExternalDataConstants.Azure.SHARED_ACCESS_SIGNATURE_FIELD_NAME;
import static org.apache.asterix.external.util.ExternalDataConstants.Azure.TENANT_ID_FIELD_NAME;
+import static org.apache.asterix.external.util.ExternalDataConstants.GCS.APPLICATION_DEFAULT_CREDENTIALS_FIELD_NAME;
import static org.apache.asterix.external.util.ExternalDataConstants.GCS.JSON_CREDENTIALS_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.KEY_ADAPTER_NAME_GCS;
-import static org.apache.asterix.external.util.ExternalDataConstants.KEY_DELIMITER;
-import static org.apache.asterix.external.util.ExternalDataConstants.KEY_ESCAPE;
-import static org.apache.asterix.external.util.ExternalDataConstants.KEY_EXCLUDE;
-import static org.apache.asterix.external.util.ExternalDataConstants.KEY_EXTERNAL_SCAN_BUFFER_SIZE;
-import static org.apache.asterix.external.util.ExternalDataConstants.KEY_FORMAT;
-import static org.apache.asterix.external.util.ExternalDataConstants.KEY_INCLUDE;
-import static org.apache.asterix.external.util.ExternalDataConstants.KEY_QUOTE;
-import static org.apache.asterix.external.util.ExternalDataConstants.KEY_RECORD_END;
-import static org.apache.asterix.external.util.ExternalDataConstants.KEY_RECORD_START;
import static org.apache.asterix.runtime.evaluators.functions.StringEvaluatorUtils.RESERVED_REGEX_CHARS;
import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
@@ -147,7 +149,8 @@
import com.azure.storage.file.datalake.models.ListPathsOptions;
import com.azure.storage.file.datalake.models.PathItem;
import com.google.api.gax.paging.Page;
-import com.google.auth.oauth2.ServiceAccountCredentials;
+import com.google.auth.oauth2.GoogleCredentials;
+import com.google.cloud.NoCredentials;
import com.google.cloud.storage.Blob;
import com.google.cloud.storage.Storage;
import com.google.cloud.storage.StorageOptions;
@@ -736,10 +739,28 @@
public static String getPrefix(Map<String, String> configuration, boolean appendSlash) {
String definition = configuration.get(ExternalDataConstants.DEFINITION_FIELD_NAME);
- if (definition != null && !definition.isEmpty()) {
+ String subPath = configuration.get(ExternalDataConstants.SUBPATH);
+ boolean hasDefinition = definition != null && !definition.isEmpty();
+ boolean hasSubPath = subPath != null && !subPath.isEmpty();
+ if (hasDefinition && !hasSubPath) {
return appendSlash ? definition + (!definition.endsWith("/") ? "/" : "") : definition;
}
- return "";
+ String fullPath = "";
+ if (hasSubPath) {
+ if (!hasDefinition) {
+ fullPath = subPath.startsWith("/") ? subPath.substring(1) : subPath;
+ } else {
+ // concatenate definition + subPath:
+ if (definition.endsWith("/") && subPath.startsWith("/")) {
+ subPath = subPath.substring(1);
+ } else if (!definition.endsWith("/") && !subPath.startsWith("/")) {
+ definition = definition + "/";
+ }
+ fullPath = definition + subPath;
+ }
+ fullPath = appendSlash ? fullPath + (!fullPath.endsWith("/") ? "/" : "") : fullPath;
+ }
+ return fullPath;
}
/**
@@ -1838,17 +1859,46 @@
* @throws CompilationException CompilationException
*/
public static Storage buildClient(Map<String, String> configuration) throws CompilationException {
+ String applicationDefaultCredentials = configuration.get(APPLICATION_DEFAULT_CREDENTIALS_FIELD_NAME);
String jsonCredentials = configuration.get(JSON_CREDENTIALS_FIELD_NAME);
+ String endpoint = configuration.get(ENDPOINT_FIELD_NAME);
StorageOptions.Builder builder = StorageOptions.newBuilder();
- // Use credentials if available
- if (jsonCredentials != null) {
- try (InputStream credentialsStream = new ByteArrayInputStream(jsonCredentials.getBytes())) {
- builder.setCredentials(ServiceAccountCredentials.fromStream(credentialsStream));
- } catch (IOException ex) {
- throw new CompilationException(EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+ // default credentials provider
+ if (applicationDefaultCredentials != null) {
+ // only "true" value is allowed
+ if (!applicationDefaultCredentials.equalsIgnoreCase("true")) {
+ throw new CompilationException(INVALID_PARAM_VALUE_ALLOWED_VALUE,
+ APPLICATION_DEFAULT_CREDENTIALS_FIELD_NAME, "true");
}
+
+ // no other authentication parameters are allowed
+ if (jsonCredentials != null) {
+ throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, JSON_CREDENTIALS_FIELD_NAME,
+ APPLICATION_DEFAULT_CREDENTIALS_FIELD_NAME);
+ }
+
+ try {
+ builder.setCredentials(GoogleCredentials.getApplicationDefault());
+ } catch (Exception ex) {
+ throw CompilationException.create(EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+ }
+ } else if (jsonCredentials != null) {
+ try (InputStream credentialsStream = new ByteArrayInputStream(jsonCredentials.getBytes())) {
+ builder.setCredentials(GoogleCredentials.fromStream(credentialsStream));
+ } catch (IOException ex) {
+ throw CompilationException.create(EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+ } catch (Exception ex) {
+ throw new CompilationException(EXTERNAL_SOURCE_ERROR,
+ "Encountered an issue while processing the JSON credentials. Please ensure the provided credentials are valid.");
+ }
+ } else {
+ builder.setCredentials(NoCredentials.getInstance());
+ }
+
+ if (endpoint != null) {
+ builder.setHost(endpoint);
}
return builder.build().getService();
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/HDFSUtils.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/HDFSUtils.java
index 3506216..f4edafc 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/HDFSUtils.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/HDFSUtils.java
@@ -265,7 +265,7 @@
public static AlgebricksAbsolutePartitionConstraint getPartitionConstraints(IApplicationContext appCtx,
AlgebricksAbsolutePartitionConstraint clusterLocations) {
if (clusterLocations == null) {
- return ((ICcApplicationContext) appCtx).getClusterStateManager().getClusterLocations();
+ return ((ICcApplicationContext) appCtx).getClusterStateManager().getSortedClusterLocations();
}
return clusterLocations;
diff --git a/asterixdb/asterix-fuzzyjoin/pom.xml b/asterixdb/asterix-fuzzyjoin/pom.xml
index 65ffa19..42bda94 100644
--- a/asterixdb/asterix-fuzzyjoin/pom.xml
+++ b/asterixdb/asterix-fuzzyjoin/pom.xml
@@ -21,7 +21,7 @@
<parent>
<artifactId>apache-asterixdb</artifactId>
<groupId>org.apache.asterix</groupId>
- <version>0.9.8-SNAPSHOT</version>
+ <version>0.9.8.2-SNAPSHOT</version>
</parent>
<artifactId>asterix-fuzzyjoin</artifactId>
diff --git a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/fuzzyjoin/IntArray.java b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/fuzzyjoin/IntArray.java
deleted file mode 100644
index 91fb8b4..0000000
--- a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/fuzzyjoin/IntArray.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.asterix.fuzzyjoin;
-
-import java.util.Arrays;
-
-public class IntArray {
- private static final int SIZE = 128;
-
- private int[] data;
- private int length;
-
- public IntArray() {
- data = new int[SIZE];
- length = 0;
- }
-
- public void add(int d) {
- if (length == data.length) {
- data = Arrays.copyOf(data, data.length << 1);
- }
- data[length++] = d;
- }
-
- public int[] get() {
- return data;
- }
-
- public int get(int i) {
- return data[i];
- }
-
- public int length() {
- return length;
- }
-
- public void reset() {
- length = 0;
- }
-
- public void sort() {
- sort(0, length);
- }
-
- public void sort(int start, int end) {
- Arrays.sort(data, start, end);
- }
-
- @Override
- public String toString() {
- StringBuilder out = new StringBuilder();
- out.append('[');
- for (int i = 0; i < length; ++i) {
- out.append(data[i]);
- if (i < length - 1) {
- out.append(',');
- out.append(' ');
- }
- }
- out.append(']');
- return out.toString();
- }
-}
diff --git a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/FuzzyJoinFunctionRegistrant.java b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/FuzzyJoinFunctionRegistrant.java
index 83ebc98..c21ac58 100644
--- a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/FuzzyJoinFunctionRegistrant.java
+++ b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/FuzzyJoinFunctionRegistrant.java
@@ -37,7 +37,6 @@
import org.apache.asterix.runtime.evaluators.functions.SimilarityJaccardPrefixDescriptor;
import org.apache.asterix.runtime.evaluators.functions.SimilarityJaccardSortedCheckDescriptor;
import org.apache.asterix.runtime.evaluators.functions.SimilarityJaccardSortedDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.SpatialIntersectDescriptor;
import org.apache.asterix.runtime.evaluators.functions.WordTokensDescriptor;
public class FuzzyJoinFunctionRegistrant implements IFunctionRegistrant {
@@ -48,9 +47,6 @@
fc.add(SimilarityJaccardPrefixDescriptor.FACTORY);
fc.add(SimilarityJaccardPrefixCheckDescriptor.FACTORY);
- // Spatial
- fc.add(SpatialIntersectDescriptor.FACTORY);
-
// fuzzyjoin function
fc.add(PrefixLenJaccardDescriptor.FACTORY);
fc.add(WordTokensDescriptor.FACTORY);
diff --git a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityDescriptor.java b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityDescriptor.java
index 95fd5bf..341dfe8 100644
--- a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityDescriptor.java
+++ b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityDescriptor.java
@@ -26,7 +26,6 @@
import org.apache.asterix.dataflow.data.nontagged.serde.AOrderedListSerializerDeserializer;
import org.apache.asterix.dataflow.data.nontagged.serde.AUnorderedListSerializerDeserializer;
import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.fuzzyjoin.IntArray;
import org.apache.asterix.fuzzyjoin.similarity.PartialIntersect;
import org.apache.asterix.fuzzyjoin.similarity.SimilarityFilters;
import org.apache.asterix.fuzzyjoin.similarity.SimilarityMetric;
@@ -36,6 +35,7 @@
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.common.IntArray;
import org.apache.asterix.runtime.evaluators.common.SimilarityFiltersCache;
import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
diff --git a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java
deleted file mode 100644
index 5a01007..0000000
--- a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java
+++ /dev/null
@@ -1,1102 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.runtime.evaluators.functions;
-
-import java.io.DataOutput;
-
-import org.apache.asterix.common.annotations.MissingNullInOutFunction;
-import org.apache.asterix.dataflow.data.nontagged.Coordinate;
-import org.apache.asterix.dataflow.data.nontagged.serde.ACircleSerializerDeserializer;
-import org.apache.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
-import org.apache.asterix.dataflow.data.nontagged.serde.AInt16SerializerDeserializer;
-import org.apache.asterix.dataflow.data.nontagged.serde.ALineSerializerDeserializer;
-import org.apache.asterix.dataflow.data.nontagged.serde.AObjectSerializerDeserializer;
-import org.apache.asterix.dataflow.data.nontagged.serde.APointSerializerDeserializer;
-import org.apache.asterix.dataflow.data.nontagged.serde.APolygonSerializerDeserializer;
-import org.apache.asterix.dataflow.data.nontagged.serde.ARectangleSerializerDeserializer;
-import org.apache.asterix.formats.nontagged.BinaryComparatorFactoryProvider;
-import org.apache.asterix.fuzzyjoin.IntArray;
-import org.apache.asterix.om.base.ABoolean;
-import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.EnumDeserializer;
-import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.asterix.runtime.evaluators.common.DoubleArray;
-import org.apache.asterix.runtime.evaluators.common.SpatialUtils;
-import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
-import org.apache.asterix.runtime.exceptions.TypeMismatchException;
-import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
-import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
-import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
-import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
-import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.data.std.primitive.VoidPointable;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
-
-@MissingNullInOutFunction
-public class SpatialIntersectDescriptor extends AbstractScalarFunctionDynamicDescriptor {
-
- private static final long serialVersionUID = 1L;
-
- public static final IFunctionDescriptorFactory FACTORY = SpatialIntersectDescriptor::new;
-
- @Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
- return new IScalarEvaluatorFactory() {
- private static final long serialVersionUID = 1L;
-
- @Override
- public IScalarEvaluator createScalarEvaluator(final IEvaluatorContext ctx) throws HyracksDataException {
- return new IScalarEvaluator() {
-
- private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
- private final DataOutput out = resultStorage.getDataOutput();
- private final IPointable inputArg0 = new VoidPointable();
- private final IPointable inputArg1 = new VoidPointable();
- private final IScalarEvaluator eval0 = args[0].createScalarEvaluator(ctx);
- private final IScalarEvaluator eval1 = args[1].createScalarEvaluator(ctx);
- private final IBinaryComparator ascDoubleComp =
- BinaryComparatorFactoryProvider.DOUBLE_POINTABLE_INSTANCE.createBinaryComparator();
- private final SpatialUtils spatialUtils = new SpatialUtils();
- private final IntArray pointsOffsets0 = new IntArray();
- private final IntArray pointsOffsets1 = new IntArray();
- private final DoubleArray trianglesX0 = new DoubleArray();
- private final DoubleArray trianglesY0 = new DoubleArray();
- private final DoubleArray trianglesX1 = new DoubleArray();
- private final DoubleArray trianglesY1 = new DoubleArray();
- private final AObjectSerializerDeserializer aBooleanSerDer = AObjectSerializerDeserializer.INSTANCE;
-
- private boolean pointOnLine(double pX, double pY, double startX, double startY, double endX,
- double endY) throws HyracksDataException {
- double crossProduct =
- SpatialUtils.crossProduct(pY - startY, pX - startX, endY - startY, endX - startX);
- if (Math.abs(crossProduct) > SpatialUtils.doubleEpsilon()) { // crossProduct != 0
- return false;
- }
-
- double dotProduct =
- SpatialUtils.dotProduct((pX - startX), (pY - startY), (endX - startX), (endY - startY));
- if (dotProduct < 0.0) {
- return false;
- }
-
- double squaredLengthBA = (endX - startX) * (endX - startX) + (endY - startY) * (endY - startY);
- if (dotProduct > squaredLengthBA) {
- return false;
- }
- return true;
- }
-
- private boolean pointInPolygon(byte[] bytes0, int offset0, byte[] bytes1, int offset1)
- throws HyracksDataException { // ray casting
-
- double pX = ADoubleSerializerDeserializer.getDouble(bytes0,
- offset0 + 1 + APointSerializerDeserializer.getCoordinateOffset(Coordinate.X));
- double pY = ADoubleSerializerDeserializer.getDouble(bytes0,
- offset0 + 1 + APointSerializerDeserializer.getCoordinateOffset(Coordinate.Y));
- int numOfPoints1 = AInt16SerializerDeserializer.getShort(bytes1,
- offset1 + 1 + APolygonSerializerDeserializer.getNumberOfPointsOffset());
-
- if (numOfPoints1 < 3) {
- throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
- ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
- }
-
- int counter = 0;
- double xInters;
- double x1, x2, y1, y2;
- x1 = ADoubleSerializerDeserializer.getDouble(bytes1,
- offset1 + 1 + APolygonSerializerDeserializer.getCoordinateOffset(0, Coordinate.X));
- y1 = ADoubleSerializerDeserializer.getDouble(bytes1,
- offset1 + 1 + APolygonSerializerDeserializer.getCoordinateOffset(0, Coordinate.Y));
-
- for (int i = 1; i <= numOfPoints1; i++) {
- if (i == numOfPoints1) {
- x2 = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
- + APolygonSerializerDeserializer.getCoordinateOffset(0, Coordinate.X));
- y2 = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
- + APolygonSerializerDeserializer.getCoordinateOffset(0, Coordinate.Y));
- } else {
- x2 = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
- + APolygonSerializerDeserializer.getCoordinateOffset(i, Coordinate.X));
- y2 = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
- + APolygonSerializerDeserializer.getCoordinateOffset(i, Coordinate.Y));
- }
-
- if (!pointOnLine(pX, pY, x1, y1, x2, y2)) {
- if (pY > Math.min(y1, y2)) {
- if (pY <= Math.max(y1, y2)) {
- if (pX <= Math.max(x1, x2)) {
- if (y1 != y2) {
- xInters = (pY - y1) * (x2 - x1) / (y2 - y1) + x1;
- if (x1 == x2 || pX <= xInters) {
- counter++;
- }
- }
- }
- }
- }
- }
- x1 = x2;
- y1 = y2;
- }
- if (counter % 2 == 1) {
- return true;
- } else {
- return false;
- }
- }
-
- private boolean pointInCircle(byte[] bytes0, int offset0, byte[] bytes1, int offset1)
- throws HyracksDataException {
- double x = ADoubleSerializerDeserializer.getDouble(bytes0,
- offset0 + 1 + APointSerializerDeserializer.getCoordinateOffset(Coordinate.X));
- double y = ADoubleSerializerDeserializer.getDouble(bytes0,
- offset0 + 1 + APointSerializerDeserializer.getCoordinateOffset(Coordinate.Y));
-
- double cX = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
- + ACircleSerializerDeserializer.getCenterPointCoordinateOffset(Coordinate.X));
- double cY = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
- + ACircleSerializerDeserializer.getCenterPointCoordinateOffset(Coordinate.Y));
- double radius = ADoubleSerializerDeserializer.getDouble(bytes1,
- offset1 + 1 + ACircleSerializerDeserializer.getRadiusOffset());
-
- if ((x - cX) * (x - cX) + (y - cY) * (y - cY) <= (radius * radius)) {
- return true;
- }
- return false;
- }
-
- private boolean lineLineIntersection(double startX1, double startY1, double endX1, double endY1,
- double startX2, double startY2, double endX2, double endY2) {
- double A1 = endY1 - startY1;
- double B1 = startX1 - endX1;
- double C1 = A1 * startX1 + B1 * startY1;
-
- double A2 = endY2 - startY2;
- double B2 = startX2 - endX2;
- double C2 = A2 * startX2 + B2 * startY2;
-
- double det = (A1 * B2) - (A2 * B1);
- if (Math.abs(det) > SpatialUtils.doubleEpsilon()) { // det != 0
- double x = (B2 * C1 - B1 * C2) / det;
- double y = (A1 * C2 - A2 * C1) / det;
-
- if ((x >= Math.min(startX1, endX1) && x <= Math.max(startX1, endX1))
- && (y >= Math.min(startY1, endY1) && y <= Math.max(startY1, endY1))) {
- if ((x >= Math.min(startX2, endX2) && x <= Math.max(startX2, endX2))
- && (y >= Math.min(startY2, endY2) && y <= Math.max(startY2, endY2))) {
- return true;
- }
- }
- }
- return false;
- }
-
- private boolean linePolygonIntersection(byte[] bytes0, int offset0, byte[] bytes1, int offset1)
- throws HyracksDataException {
- double startX1 = ADoubleSerializerDeserializer.getDouble(bytes0,
- offset0 + 1 + ALineSerializerDeserializer.getStartPointCoordinateOffset(Coordinate.X));
- double startY1 = ADoubleSerializerDeserializer.getDouble(bytes0,
- offset0 + 1 + ALineSerializerDeserializer.getStartPointCoordinateOffset(Coordinate.Y));
- double endX1 = ADoubleSerializerDeserializer.getDouble(bytes0,
- offset0 + 1 + ALineSerializerDeserializer.getEndPointCoordinateOffset(Coordinate.X));
- double endY1 = ADoubleSerializerDeserializer.getDouble(bytes0,
- offset0 + 1 + ALineSerializerDeserializer.getEndPointCoordinateOffset(Coordinate.Y));
-
- int numOfPoints1 = AInt16SerializerDeserializer.getShort(bytes1,
- offset1 + 1 + APolygonSerializerDeserializer.getNumberOfPointsOffset());
-
- if (numOfPoints1 < 3) {
- throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
- ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
- }
- for (int i = 0; i < numOfPoints1; i++) {
- double startX2 = ADoubleSerializerDeserializer.getDouble(bytes1,
- offset1 + 1 + APolygonSerializerDeserializer.getCoordinateOffset(i, Coordinate.X));
- double startY2 = ADoubleSerializerDeserializer.getDouble(bytes1,
- offset1 + 1 + APolygonSerializerDeserializer.getCoordinateOffset(i, Coordinate.Y));
-
- double endX2;
- double endY2;
- if (i + 1 == numOfPoints1) {
- endX2 = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
- + APolygonSerializerDeserializer.getCoordinateOffset(0, Coordinate.X));
- endY2 = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
- + APolygonSerializerDeserializer.getCoordinateOffset(0, Coordinate.Y));
- } else {
- endX2 = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
- + APolygonSerializerDeserializer.getCoordinateOffset(i + 1, Coordinate.X));
- endY2 = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
- + APolygonSerializerDeserializer.getCoordinateOffset(i + 1, Coordinate.Y));
- }
-
- boolean intersect = lineLineIntersection(startX1, startY1, endX1, endY1, startX2, startY2,
- endX2, endY2);
- if (intersect) {
- return true;
- }
- }
- return false;
- }
-
- private boolean lineRectangleIntersection(byte[] bytes0, int offset0, byte[] bytes1, int offset1)
- throws HyracksDataException {
- double startX1 = ADoubleSerializerDeserializer.getDouble(bytes0,
- offset0 + 1 + ALineSerializerDeserializer.getStartPointCoordinateOffset(Coordinate.X));
- double startY1 = ADoubleSerializerDeserializer.getDouble(bytes0,
- offset0 + 1 + ALineSerializerDeserializer.getStartPointCoordinateOffset(Coordinate.Y));
- double endX1 = ADoubleSerializerDeserializer.getDouble(bytes0,
- offset0 + 1 + ALineSerializerDeserializer.getEndPointCoordinateOffset(Coordinate.X));
- double endY1 = ADoubleSerializerDeserializer.getDouble(bytes0,
- offset0 + 1 + ALineSerializerDeserializer.getEndPointCoordinateOffset(Coordinate.Y));
-
- double x1 = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
- + ARectangleSerializerDeserializer.getBottomLeftCoordinateOffset(Coordinate.X));
- double y1 = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
- + ARectangleSerializerDeserializer.getBottomLeftCoordinateOffset(Coordinate.Y));
-
- double x2 = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
- + ARectangleSerializerDeserializer.getUpperRightCoordinateOffset(Coordinate.X));
- double y2 = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
- + ARectangleSerializerDeserializer.getUpperRightCoordinateOffset(Coordinate.Y));
-
- if (lineLineIntersection(startX1, startY1, endX1, endY1, x1, y1, x1, y2)
- || lineLineIntersection(startX1, startY1, endX1, endY1, x1, y2, x2, y2)
- || lineLineIntersection(startX1, startY1, endX1, endY1, x2, y2, x2, y1)
- || lineLineIntersection(startX1, startY1, endX1, endY1, x2, y1, x1, y1)) {
- return true;
- }
- return false;
-
- }
-
- private boolean lineCircleIntersection(byte[] bytes0, int offset0, byte[] bytes1, int offset1)
- throws HyracksDataException {
- double startX = ADoubleSerializerDeserializer.getDouble(bytes0,
- offset0 + 1 + ALineSerializerDeserializer.getStartPointCoordinateOffset(Coordinate.X));
- double startY = ADoubleSerializerDeserializer.getDouble(bytes0,
- offset0 + 1 + ALineSerializerDeserializer.getStartPointCoordinateOffset(Coordinate.Y));
- double endX = ADoubleSerializerDeserializer.getDouble(bytes0,
- offset0 + 1 + ALineSerializerDeserializer.getEndPointCoordinateOffset(Coordinate.X));
- double endY = ADoubleSerializerDeserializer.getDouble(bytes0,
- offset0 + 1 + ALineSerializerDeserializer.getEndPointCoordinateOffset(Coordinate.Y));
-
- double cX = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
- + ACircleSerializerDeserializer.getCenterPointCoordinateOffset(Coordinate.X));
- double cY = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
- + ACircleSerializerDeserializer.getCenterPointCoordinateOffset(Coordinate.Y));
- double radius = ADoubleSerializerDeserializer.getDouble(bytes1,
- offset1 + 1 + ACircleSerializerDeserializer.getRadiusOffset());
-
- double dx = endX - startX;
- double dy = endY - startY;
- double t = -((startX - cX) * dx + (startY - cY) * dy) / ((dx * dx) + (dy * dy));
-
- if (t < 0.0) {
- t = 0.0;
- } else if (t > 1.0) {
- t = 1.0;
- }
-
- dx = (startX + t * (endX - startX)) - cX;
- dy = (startY + t * (endY - startY)) - cY;
- double rt = (dx * dx) + (dy * dy);
- if (rt <= (radius * radius)) {
- return true;
- }
- return false;
- }
-
- private boolean findEar(byte[] bytes, int offset, int u, int v, int w, int n,
- IntArray pointsOffsets) throws HyracksDataException {
- int p;
- double Ax, Ay, Bx, By, Cx, Cy, Px, Py;
-
- Ax = ADoubleSerializerDeserializer.getDouble(bytes, offset + 1 + APolygonSerializerDeserializer
- .getCoordinateOffset(pointsOffsets.get(u), Coordinate.X));
- Ay = ADoubleSerializerDeserializer.getDouble(bytes, offset + 1 + APolygonSerializerDeserializer
- .getCoordinateOffset(pointsOffsets.get(u), Coordinate.Y));
-
- Bx = ADoubleSerializerDeserializer.getDouble(bytes, offset + 1 + APolygonSerializerDeserializer
- .getCoordinateOffset(pointsOffsets.get(v), Coordinate.X));
- By = ADoubleSerializerDeserializer.getDouble(bytes, offset + 1 + APolygonSerializerDeserializer
- .getCoordinateOffset(pointsOffsets.get(v), Coordinate.Y));
-
- Cx = ADoubleSerializerDeserializer.getDouble(bytes, offset + 1 + APolygonSerializerDeserializer
- .getCoordinateOffset(pointsOffsets.get(w), Coordinate.X));
- Cy = ADoubleSerializerDeserializer.getDouble(bytes, offset + 1 + APolygonSerializerDeserializer
- .getCoordinateOffset(pointsOffsets.get(w), Coordinate.Y));
-
- if (SpatialUtils.doubleEpsilon() > (((Bx - Ax) * (Cy - Ay)) - ((By - Ay) * (Cx - Ax)))) {
-
- return false;
- }
-
- for (p = 0; p < n; p++) {
- if ((p == u) || (p == v) || (p == w)) {
- continue;
- }
- Px = ADoubleSerializerDeserializer.getDouble(bytes,
- offset + 1 + APolygonSerializerDeserializer
- .getCoordinateOffset(pointsOffsets.get(p), Coordinate.X));
- Py = ADoubleSerializerDeserializer.getDouble(bytes,
- offset + 1 + APolygonSerializerDeserializer
- .getCoordinateOffset(pointsOffsets.get(p), Coordinate.Y));
- if (pointInsideTriangle(Ax, Ay, Bx, By, Cx, Cy, Px, Py)) {
- return false;
- }
- }
-
- return true;
- }
-
- private int triangulatePolygon(byte[] bytes, int offset, int numOfPoints, IntArray pointsOffsets,
- DoubleArray trianglesX, DoubleArray trianglesY, int triangleId,
- int nonSimplePolygonDetection, int middleVertex) throws HyracksDataException { // Ear clipping
-
- if (numOfPoints < 3) {
- return -1;
- }
-
- boolean foundEar = false;
- int v = middleVertex;
- while (!foundEar) {
- if (0 >= (nonSimplePolygonDetection--)) {
- throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
- ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
- }
- int u = v;
- if (numOfPoints <= u) {
- u = 0;
- }
- v = u + 1;
- if (numOfPoints <= v) {
- v = 0;
- }
- int w = v + 1;
- if (numOfPoints <= w) {
- w = 0;
- }
-
- if (findEar(bytes, offset, u, v, w, numOfPoints, pointsOffsets)) {
- int s, t;
-
- addRectangle(trianglesX, trianglesY);
-
- SpatialUtils.setTriangleXCoordinate(trianglesX, triangleId, 0,
- ADoubleSerializerDeserializer.getDouble(bytes,
- offset + 1 + APolygonSerializerDeserializer
- .getCoordinateOffset(pointsOffsets.get(u), Coordinate.X)));
-
- SpatialUtils.setTriangleYCoordinate(trianglesY, triangleId, 0,
- ADoubleSerializerDeserializer.getDouble(bytes,
- offset + 1 + APolygonSerializerDeserializer
- .getCoordinateOffset(pointsOffsets.get(u), Coordinate.Y)));
-
- SpatialUtils.setTriangleXCoordinate(trianglesX, triangleId, 1,
- ADoubleSerializerDeserializer.getDouble(bytes,
- offset + 1 + APolygonSerializerDeserializer
- .getCoordinateOffset(pointsOffsets.get(v), Coordinate.X)));
-
- SpatialUtils.setTriangleYCoordinate(trianglesY, triangleId, 1,
- ADoubleSerializerDeserializer.getDouble(bytes,
- offset + 1 + APolygonSerializerDeserializer
- .getCoordinateOffset(pointsOffsets.get(v), Coordinate.Y)));
-
- SpatialUtils.setTriangleXCoordinate(trianglesX, triangleId, 2,
- ADoubleSerializerDeserializer.getDouble(bytes,
- offset + 1 + APolygonSerializerDeserializer
- .getCoordinateOffset(pointsOffsets.get(w), Coordinate.X)));
-
- SpatialUtils.setTriangleYCoordinate(trianglesY, triangleId, 2,
- ADoubleSerializerDeserializer.getDouble(bytes,
- offset + 1 + APolygonSerializerDeserializer
- .getCoordinateOffset(pointsOffsets.get(w), Coordinate.Y)));
-
- // remove v from polygon
- for (s = v, t = v + 1; t < numOfPoints; s++, t++) {
- pointsOffsets.get()[s] = pointsOffsets.get(t);
- }
- foundEar = true;
- }
- }
-
- return v;
- }
-
- private boolean triangleTriangleIntersection(DoubleArray trianglesX0, DoubleArray trianglesY0,
- int triangleId0, DoubleArray trianglesX1, DoubleArray trianglesY1, int triangleId1)
- throws HyracksDataException { // separating axis theorem
-
- for (int side = 0; side < 3; side++) {
- spatialUtils.findNormals(trianglesX0, trianglesY0, triangleId0, side);
- spatialUtils.projectPolygon(trianglesX0, trianglesY0, triangleId0, spatialUtils.getXAxis(),
- spatialUtils.getYAxis());
- double min1 = spatialUtils.getMinProjection();
- double max1 = spatialUtils.getMaxProjection();
- spatialUtils.projectPolygon(trianglesX1, trianglesY1, triangleId1, spatialUtils.getXAxis(),
- spatialUtils.getYAxis());
- double min2 = spatialUtils.getMinProjection();
- double max2 = spatialUtils.getMaxProjection();
-
- if (max1 < min2 || min1 > max2) {
- return false;
- }
- }
- return true;
- }
-
- private boolean pointInsideTriangle(double x1, double y1, double x2, double y2, double x3,
- double y3, double pX, double pY) {
- return pointsOnSameSide(pX, pY, x1, y1, x2, y2, x3, y3)
- && pointsOnSameSide(pX, pY, x2, y2, x1, y1, x3, y3)
- && pointsOnSameSide(pX, pY, x3, y3, x1, y1, x2, y2);
- }
-
- private boolean pointsOnSameSide(double pX, double pY, double x1, double y1, double x2, double y2,
- double x3, double y3) {
- double cp1 = SpatialUtils.crossProduct(x3 - x2, y3 - y2, pX - x2, pY - y2);
- double cp2 = SpatialUtils.crossProduct(x3 - x2, y3 - y2, x1 - x2, y1 - y2);
- return (cp1 * cp2) >= 0.0;
- }
-
- private boolean circleTriangleIntersection(byte[] bytes0, int offset0, DoubleArray trianglesX,
- DoubleArray trianglesY, int triangleId) throws HyracksDataException { // separating axis theorem
-
- double cX = ADoubleSerializerDeserializer.getDouble(bytes0, offset0 + 1
- + ACircleSerializerDeserializer.getCenterPointCoordinateOffset(Coordinate.X));
- double cY = ADoubleSerializerDeserializer.getDouble(bytes0, offset0 + 1
- + ACircleSerializerDeserializer.getCenterPointCoordinateOffset(Coordinate.Y));
- double radius = ADoubleSerializerDeserializer.getDouble(bytes0,
- offset0 + 1 + ACircleSerializerDeserializer.getRadiusOffset());
-
- double distance = Double.MAX_VALUE;
- double distanceSquared;
-
- double temp;
- double closestPointX = 0.0;
- double closestPointY = 0.0;
- for (int i = 0; i < 3; i++) {
- double pX = SpatialUtils.getTriangleXCoordinate(trianglesX, triangleId, i);
- double pY = SpatialUtils.getTriangleXCoordinate(trianglesY, triangleId, i);
-
- distanceSquared = (cX - pX) * (cX - pX) + (cY - pY) * (cY - pY);
- if (distanceSquared < distance) {
- distance = distanceSquared;
- closestPointX = pX;
- closestPointY = pY;
- }
- }
-
- double x = Math.abs(cX - closestPointX);
- double y = Math.abs(cY - closestPointY);
-
- temp = Math.sqrt(SpatialUtils.dotProduct(x, y, x, y));
- x /= temp;
- y /= temp;
-
- spatialUtils.projectPolygon(trianglesX, trianglesY, triangleId, x, y);
-
- double min1 = spatialUtils.getMinProjection();
- double max1 = spatialUtils.getMaxProjection();
-
- double dotProduct = SpatialUtils.dotProduct(x, y, cX, cY);
- double max2 = dotProduct + radius;
- double min2 = dotProduct - radius;
-
- if (max1 < min2 || min1 > max2) {
- return false;
- }
-
- for (int side = 0; side < 3; side++) {
- spatialUtils.findNormals(trianglesX, trianglesY, triangleId, side);
- spatialUtils.projectPolygon(trianglesX, trianglesY, triangleId, spatialUtils.getXAxis(),
- spatialUtils.getYAxis());
- min1 = spatialUtils.getMinProjection();
- max1 = spatialUtils.getMaxProjection();
-
- dotProduct =
- SpatialUtils.dotProduct(spatialUtils.getXAxis(), spatialUtils.getYAxis(), cX, cY);
- max2 = dotProduct + radius;
- min2 = dotProduct - radius;
-
- if (max1 < min2 || min1 > max2) {
- return false;
- }
- }
- return true;
- }
-
- private boolean circleCircleIntersection(byte[] bytes0, int offset0, byte[] bytes1, int offset1)
- throws HyracksDataException {
- double cX0 = ADoubleSerializerDeserializer.getDouble(bytes0, offset0 + 1
- + ACircleSerializerDeserializer.getCenterPointCoordinateOffset(Coordinate.X));
- double cY0 = ADoubleSerializerDeserializer.getDouble(bytes0, offset0 + 1
- + ACircleSerializerDeserializer.getCenterPointCoordinateOffset(Coordinate.Y));
- double radius0 = ADoubleSerializerDeserializer.getDouble(bytes0,
- offset0 + 1 + ACircleSerializerDeserializer.getRadiusOffset());
-
- double cX1 = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
- + ACircleSerializerDeserializer.getCenterPointCoordinateOffset(Coordinate.X));
- double cY1 = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
- + ACircleSerializerDeserializer.getCenterPointCoordinateOffset(Coordinate.Y));
- double radius1 = ADoubleSerializerDeserializer.getDouble(bytes1,
- offset1 + 1 + ACircleSerializerDeserializer.getRadiusOffset());
-
- double distanceSquared = SpatialUtils.dotProduct(cX0 - cX1, cY0 - cY1, cX0 - cX1, cY0 - cY1);
- double radiusDistanceSquared = (radius0 + radius1) * (radius0 + radius1);
- if (distanceSquared <= radiusDistanceSquared) {
- return true;
- }
- return false;
- }
-
- private void getCounterClockWisePolygon(byte[] bytes, int offset, IntArray pointsOffsets,
- int numOfPoints) throws HyracksDataException {
- pointsOffsets.reset();
- if (SpatialUtils.polygonArea(bytes, offset, numOfPoints) > 0.0) {
- for (int i = 0; i < numOfPoints; i++) {
- pointsOffsets.add(i);
- }
- } else {
- for (int i = 0; i < numOfPoints; i++) {
- pointsOffsets.add((numOfPoints - 1) - i);
- }
- }
- }
-
- private boolean pointInRectangle(byte[] bytes0, int offset0, byte[] bytes1, int offset1)
- throws HyracksDataException {
-
- double pX = ADoubleSerializerDeserializer.getDouble(bytes0,
- offset0 + 1 + APointSerializerDeserializer.getCoordinateOffset(Coordinate.X));
- double pY = ADoubleSerializerDeserializer.getDouble(bytes0,
- offset0 + 1 + APointSerializerDeserializer.getCoordinateOffset(Coordinate.Y));
-
- double x1 = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
- + ARectangleSerializerDeserializer.getBottomLeftCoordinateOffset(Coordinate.X));
- double y1 = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
- + ARectangleSerializerDeserializer.getBottomLeftCoordinateOffset(Coordinate.Y));
-
- double x2 = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
- + ARectangleSerializerDeserializer.getUpperRightCoordinateOffset(Coordinate.X));
- double y2 = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
- + ARectangleSerializerDeserializer.getUpperRightCoordinateOffset(Coordinate.Y));
-
- if (pointInsideTriangle(x1, y1, x1, y2, x2, y2, pX, pY)
- || pointInsideTriangle(x1, y1, x2, y1, x2, y2, pX, pY)) {
- return true;
- }
- return false;
-
- }
-
- private void addRectangle(DoubleArray trianglesX, DoubleArray trianglesY) {
- for (int i = 0; i < 3; i++) {
- double temp = 0;
- trianglesX.add(temp);
- trianglesY.add(temp);
- }
- }
-
- private boolean rectangleCircleIntersection(byte[] bytes0, int offset0, byte[] bytes1, int offset1)
- throws HyracksDataException {
- triangulateRectangle(bytes0, offset0, trianglesX0, trianglesY0);
- boolean res = false;
- // 2 triangles in a rectangle
- for (int i = 0; i < 2; i++) {
- res = circleTriangleIntersection(bytes1, offset1, trianglesX0, trianglesY0, i);
- if (res) {
- break;
- }
- }
- return res;
- }
-
- private void triangulateRectangle(byte[] bytes, int offset, DoubleArray trianglesX,
- DoubleArray trianglesY) throws HyracksDataException {
- double x1 = ADoubleSerializerDeserializer.getDouble(bytes, offset + 1
- + ARectangleSerializerDeserializer.getBottomLeftCoordinateOffset(Coordinate.X));
- double y1 = ADoubleSerializerDeserializer.getDouble(bytes, offset + 1
- + ARectangleSerializerDeserializer.getBottomLeftCoordinateOffset(Coordinate.Y));
-
- double x2 = ADoubleSerializerDeserializer.getDouble(bytes, offset + 1
- + ARectangleSerializerDeserializer.getUpperRightCoordinateOffset(Coordinate.X));
- double y2 = ADoubleSerializerDeserializer.getDouble(bytes, offset + 1
- + ARectangleSerializerDeserializer.getUpperRightCoordinateOffset(Coordinate.Y));
- trianglesX.reset();
- trianglesY.reset();
-
- addRectangle(trianglesX, trianglesY);
- addRectangle(trianglesX, trianglesY);
-
- SpatialUtils.setTriangleXCoordinate(trianglesX, 0, 0, x1);
- SpatialUtils.setTriangleYCoordinate(trianglesY, 0, 0, y1);
-
- SpatialUtils.setTriangleXCoordinate(trianglesX, 0, 1, x2);
- SpatialUtils.setTriangleYCoordinate(trianglesY, 0, 1, y1);
-
- SpatialUtils.setTriangleXCoordinate(trianglesX, 0, 2, x2);
- SpatialUtils.setTriangleYCoordinate(trianglesY, 0, 2, y2);
-
- SpatialUtils.setTriangleXCoordinate(trianglesX, 1, 0, x2);
- SpatialUtils.setTriangleYCoordinate(trianglesY, 1, 0, y2);
-
- SpatialUtils.setTriangleXCoordinate(trianglesX, 1, 1, x1);
- SpatialUtils.setTriangleYCoordinate(trianglesY, 1, 1, y2);
-
- SpatialUtils.setTriangleXCoordinate(trianglesX, 1, 2, x1);
- SpatialUtils.setTriangleYCoordinate(trianglesY, 1, 2, y1);
- }
-
- private boolean rectanglePolygonIntersection(byte[] bytes0, int offset0, byte[] bytes1, int offset1)
- throws HyracksDataException {
- int numOfPoints1 = AInt16SerializerDeserializer.getShort(bytes1,
- offset1 + 1 + APolygonSerializerDeserializer.getNumberOfPointsOffset());
-
- if (numOfPoints1 < 3) {
- throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
- ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
- }
-
- getCounterClockWisePolygon(bytes1, offset1, pointsOffsets1, numOfPoints1);
- int nonSimplePolygonDetection1 = 2 * numOfPoints1;
- int middleVertex1 = numOfPoints1 - 1;
- int numOfTriangles1 = 0;
-
- trianglesX1.reset();
- trianglesY1.reset();
- while (true) {
- middleVertex1 =
- triangulatePolygon(bytes1, offset1, numOfPoints1, pointsOffsets1, trianglesX1,
- trianglesY1, numOfTriangles1, nonSimplePolygonDetection1, middleVertex1);
-
- if (middleVertex1 == -1) {
- break;
- }
-
- numOfPoints1--;
- nonSimplePolygonDetection1 = 2 * numOfPoints1;
- numOfTriangles1++;
- }
-
- triangulateRectangle(bytes0, offset0, trianglesX0, trianglesY0);
- boolean res = false;
- // 2 triangles in a rectangle
- for (int j = 0; j < 2; j++) {
- for (int i = 0; i < numOfTriangles1; i++) {
-
- res = triangleTriangleIntersection(trianglesX1, trianglesY1, i, trianglesX0,
- trianglesY0, j);
-
- if (res) {
- res = triangleTriangleIntersection(trianglesX0, trianglesY0, j, trianglesX1,
- trianglesY1, i);
-
- if (res) {
- return true;
- }
- }
- }
- }
- return false;
- }
-
- private boolean polygonCircleIntersection(byte[] bytes0, int offset0, byte[] bytes1, int offset1)
- throws HyracksDataException {
- int numOfPoints = AInt16SerializerDeserializer.getShort(bytes0,
- offset0 + 1 + APolygonSerializerDeserializer.getNumberOfPointsOffset());
-
- if (numOfPoints < 3) {
- throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
- ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
- }
-
- getCounterClockWisePolygon(bytes0, offset0, pointsOffsets0, numOfPoints);
- int nonSimplePolygonDetection = 2 * numOfPoints;
- int middleVertex = numOfPoints - 1;
- int numOfTriangles = 0;
-
- trianglesX0.reset();
- trianglesY0.reset();
- boolean res = false;
- while (true) {
- middleVertex = triangulatePolygon(bytes0, offset0, numOfPoints, pointsOffsets0, trianglesX0,
- trianglesY0, numOfTriangles, nonSimplePolygonDetection, middleVertex);
-
- if (middleVertex == -1) {
- break;
- }
- numOfPoints--;
- nonSimplePolygonDetection = 2 * numOfPoints;
- numOfTriangles++;
- int lastTriangle = (trianglesX0.length() / 3) - 1;
-
- res = circleTriangleIntersection(bytes1, offset1, trianglesX0, trianglesY0, lastTriangle);
- if (res) {
- return true;
- }
- }
- return false;
- }
-
- @Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
- resultStorage.reset();
- eval0.evaluate(tuple, inputArg0);
- eval1.evaluate(tuple, inputArg1);
-
- if (PointableHelper.checkAndSetMissingOrNull(result, inputArg0, inputArg1)) {
- return;
- }
-
- byte[] bytes0 = inputArg0.getByteArray();
- byte[] bytes1 = inputArg1.getByteArray();
- int offset0 = inputArg0.getStartOffset();
- int offset1 = inputArg1.getStartOffset();
-
- boolean res = false;
- ATypeTag tag0 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes0[offset0]);
- ATypeTag tag1 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]);
-
- switch (tag0) {
- case POINT:
- switch (tag1) {
- case POINT:
- if (ascDoubleComp.compare(bytes0,
- offset0 + 1
- + APointSerializerDeserializer
- .getCoordinateOffset(Coordinate.X),
- 8, bytes1, offset1 + 1 + APointSerializerDeserializer
- .getCoordinateOffset(Coordinate.X),
- 8) == 0) {
- if (ascDoubleComp
- .compare(bytes0,
- offset0 + 1
- + APointSerializerDeserializer
- .getCoordinateOffset(Coordinate.Y),
- 8, bytes1,
- offset1 + 1
- + APointSerializerDeserializer
- .getCoordinateOffset(Coordinate.Y),
- 8) == 0) {
- res = true;
- }
- }
- break;
- case LINE:
- double pX = ADoubleSerializerDeserializer.getDouble(bytes0, offset0 + 1
- + APointSerializerDeserializer.getCoordinateOffset(Coordinate.X));
- double pY = ADoubleSerializerDeserializer.getDouble(bytes0, offset0 + 1
- + APointSerializerDeserializer.getCoordinateOffset(Coordinate.Y));
-
- double startX = ADoubleSerializerDeserializer.getDouble(bytes1,
- offset1 + 1 + ALineSerializerDeserializer
- .getStartPointCoordinateOffset(Coordinate.X));
- double startY = ADoubleSerializerDeserializer.getDouble(bytes1,
- offset1 + 1 + ALineSerializerDeserializer
- .getStartPointCoordinateOffset(Coordinate.Y));
- double endX = ADoubleSerializerDeserializer.getDouble(bytes1,
- offset1 + 1 + ALineSerializerDeserializer
- .getEndPointCoordinateOffset(Coordinate.X));
- double endY = ADoubleSerializerDeserializer.getDouble(bytes1,
- offset1 + 1 + ALineSerializerDeserializer
- .getEndPointCoordinateOffset(Coordinate.Y));
-
- res = pointOnLine(pX, pY, startX, startY, endX, endY);
- break;
- case POLYGON:
- res = pointInPolygon(bytes0, offset0, bytes1, offset1);
- break;
- case CIRCLE:
- res = pointInCircle(bytes0, offset0, bytes1, offset1);
- break;
- case RECTANGLE:
- res = pointInRectangle(bytes0, offset0, bytes1, offset1);
- break;
- default:
- throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
- ATypeTag.SERIALIZED_POINT_TYPE_TAG, ATypeTag.SERIALIZED_LINE_TYPE_TAG,
- ATypeTag.SERIALIZED_POLYGON_TYPE_TAG,
- ATypeTag.SERIALIZED_CIRCLE_TYPE_TAG,
- ATypeTag.SERIALIZED_RECTANGLE_TYPE_TAG);
- }
- break;
- case LINE:
- switch (tag1) {
- case POINT:
- double pX = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
- + APointSerializerDeserializer.getCoordinateOffset(Coordinate.X));
- double pY = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
- + APointSerializerDeserializer.getCoordinateOffset(Coordinate.Y));
-
- double startX = ADoubleSerializerDeserializer.getDouble(bytes0,
- offset0 + 1 + ALineSerializerDeserializer
- .getStartPointCoordinateOffset(Coordinate.X));
- double startY = ADoubleSerializerDeserializer.getDouble(bytes0,
- offset0 + 1 + ALineSerializerDeserializer
- .getStartPointCoordinateOffset(Coordinate.Y));
- double endX = ADoubleSerializerDeserializer.getDouble(bytes0,
- offset0 + 1 + ALineSerializerDeserializer
- .getEndPointCoordinateOffset(Coordinate.X));
- double endY = ADoubleSerializerDeserializer.getDouble(bytes0,
- offset0 + 1 + ALineSerializerDeserializer
- .getEndPointCoordinateOffset(Coordinate.Y));
-
- res = pointOnLine(pX, pY, startX, startY, endX, endY);
- break;
- case LINE:
- double startX1 = ADoubleSerializerDeserializer.getDouble(bytes0,
- offset0 + 1 + ALineSerializerDeserializer
- .getStartPointCoordinateOffset(Coordinate.X));
- double startY1 = ADoubleSerializerDeserializer.getDouble(bytes0,
- offset0 + 1 + ALineSerializerDeserializer
- .getStartPointCoordinateOffset(Coordinate.Y));
- double endX1 = ADoubleSerializerDeserializer.getDouble(bytes0,
- offset0 + 1 + ALineSerializerDeserializer
- .getEndPointCoordinateOffset(Coordinate.X));
- double endY1 = ADoubleSerializerDeserializer.getDouble(bytes0,
- offset0 + 1 + ALineSerializerDeserializer
- .getEndPointCoordinateOffset(Coordinate.Y));
-
- double startX2 = ADoubleSerializerDeserializer.getDouble(bytes1,
- offset1 + 1 + ALineSerializerDeserializer
- .getStartPointCoordinateOffset(Coordinate.X));
- double startY2 = ADoubleSerializerDeserializer.getDouble(bytes1,
- offset1 + 1 + ALineSerializerDeserializer
- .getStartPointCoordinateOffset(Coordinate.Y));
- double endX2 = ADoubleSerializerDeserializer.getDouble(bytes1,
- offset1 + 1 + ALineSerializerDeserializer
- .getEndPointCoordinateOffset(Coordinate.X));
- double endY2 = ADoubleSerializerDeserializer.getDouble(bytes1,
- offset1 + 1 + ALineSerializerDeserializer
- .getEndPointCoordinateOffset(Coordinate.Y));
- res = lineLineIntersection(startX1, startY1, endX1, endY1, startX2, startY2,
- endX2, endY2);
- break;
- case POLYGON:
- res = linePolygonIntersection(bytes0, offset0, bytes1, offset1);
- break;
- case CIRCLE:
- res = lineCircleIntersection(bytes0, offset0, bytes1, offset1);
- break;
- case RECTANGLE:
- res = lineRectangleIntersection(bytes0, offset0, bytes1, offset1);
- break;
- default:
- throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
- ATypeTag.SERIALIZED_POINT_TYPE_TAG, ATypeTag.SERIALIZED_LINE_TYPE_TAG,
- ATypeTag.SERIALIZED_POLYGON_TYPE_TAG,
- ATypeTag.SERIALIZED_CIRCLE_TYPE_TAG,
- ATypeTag.SERIALIZED_RECTANGLE_TYPE_TAG);
- }
- break;
- case POLYGON:
- switch (tag1) {
- case POINT:
- res = pointInPolygon(bytes1, offset1, bytes0, offset0);
- break;
- case LINE:
- res = linePolygonIntersection(bytes1, offset1, bytes0, offset0);
- break;
- case POLYGON:
- int numOfPoints0 = AInt16SerializerDeserializer.getShort(bytes0,
- offset0 + 1 + APolygonSerializerDeserializer.getNumberOfPointsOffset());
- int numOfPoints1 = AInt16SerializerDeserializer.getShort(bytes1,
- offset1 + 1 + APolygonSerializerDeserializer.getNumberOfPointsOffset());
-
- if (numOfPoints0 < 3 || numOfPoints1 < 3) {
- throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
- ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
- }
-
- getCounterClockWisePolygon(bytes0, offset0, pointsOffsets0, numOfPoints0);
- getCounterClockWisePolygon(bytes1, offset1, pointsOffsets1, numOfPoints1);
- int nonSimplePolygonDetection0 = 2 * numOfPoints0;
- int nonSimplePolygonDetection1 = 2 * numOfPoints1;
- boolean intersect = false;
- int middleVertex0 = numOfPoints0 - 1;
-
- int numOfTriangles1 = 0;
- int middleVertex1 = numOfPoints1 - 1;
- trianglesX1.reset();
- trianglesY1.reset();
- while (true) {
- middleVertex1 = triangulatePolygon(bytes1, offset1, numOfPoints1,
- pointsOffsets1, trianglesX1, trianglesY1, numOfTriangles1,
- nonSimplePolygonDetection1, middleVertex1);
-
- if (middleVertex1 == -1) {
- break;
- }
-
- numOfPoints1--;
- nonSimplePolygonDetection1 = 2 * numOfPoints1;
- numOfTriangles1++;
- }
- int numOfTriangles0 = 0;
- trianglesX0.reset();
- trianglesY0.reset();
- while (true) {
- middleVertex0 = triangulatePolygon(bytes0, offset0, numOfPoints0,
- pointsOffsets0, trianglesX0, trianglesY0, numOfTriangles0,
- nonSimplePolygonDetection0, middleVertex0);
-
- if (middleVertex0 == -1) {
- break;
- }
- numOfPoints0--;
- nonSimplePolygonDetection0 = 2 * numOfPoints0;
- numOfTriangles0++;
- int lastTriangle = (trianglesX0.length() / 3) - 1;
-
- for (int i = 0; i < numOfTriangles1; i++) {
-
- res = triangleTriangleIntersection(trianglesX0, trianglesY0,
- lastTriangle, trianglesX1, trianglesY1, i);
-
- if (res) {
- res = triangleTriangleIntersection(trianglesX1, trianglesY1, i,
- trianglesX0, trianglesY0, lastTriangle);
-
- if (res) {
- intersect = true;
- break;
- }
- }
- }
- if (intersect) {
- break;
- }
- }
- break;
- case CIRCLE:
- res = polygonCircleIntersection(bytes0, offset0, bytes1, offset1);
- break;
- case RECTANGLE:
- res = rectanglePolygonIntersection(bytes1, offset1, bytes0, offset0);
- break;
- default:
- throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
- ATypeTag.SERIALIZED_POINT_TYPE_TAG, ATypeTag.SERIALIZED_LINE_TYPE_TAG,
- ATypeTag.SERIALIZED_POLYGON_TYPE_TAG,
- ATypeTag.SERIALIZED_CIRCLE_TYPE_TAG,
- ATypeTag.SERIALIZED_RECTANGLE_TYPE_TAG);
- }
- break;
- case CIRCLE:
- switch (tag1) {
- case POINT:
- res = pointInCircle(bytes1, offset1, bytes0, offset0);
- break;
- case LINE:
- res = lineCircleIntersection(bytes1, offset1, bytes0, offset0);
- break;
- case POLYGON:
- res = polygonCircleIntersection(bytes1, offset1, bytes0, offset0);
- break;
- case CIRCLE:
- res = circleCircleIntersection(bytes0, offset0, bytes1, offset1);
- break;
- case RECTANGLE:
- res = rectangleCircleIntersection(bytes1, offset1, bytes0, offset0);
- break;
- default:
- throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
- ATypeTag.SERIALIZED_POINT_TYPE_TAG, ATypeTag.SERIALIZED_LINE_TYPE_TAG,
- ATypeTag.SERIALIZED_POLYGON_TYPE_TAG,
- ATypeTag.SERIALIZED_CIRCLE_TYPE_TAG,
- ATypeTag.SERIALIZED_RECTANGLE_TYPE_TAG);
- }
- break;
- case RECTANGLE:
- switch (tag1) {
- case POINT:
- res = pointInRectangle(bytes1, offset1, bytes0, offset0);
- break;
- case LINE:
- res = lineRectangleIntersection(bytes1, offset1, bytes0, offset0);
- break;
- case POLYGON:
- res = rectanglePolygonIntersection(bytes0, offset0, bytes1, offset1);
- break;
- case CIRCLE:
- res = rectangleCircleIntersection(bytes0, offset0, bytes1, offset1);
- break;
- case RECTANGLE:
- triangulateRectangle(bytes0, offset0, trianglesX0, trianglesY0);
- triangulateRectangle(bytes1, offset1, trianglesX1, trianglesY1);
-
- boolean intersect = false;
- // 2 triangles in a rectangle
- for (int j = 0; j < 2; j++) {
- for (int i = 0; i < 2; i++) {
-
- res = triangleTriangleIntersection(trianglesX1, trianglesY1, i,
- trianglesX0, trianglesY0, j);
-
- if (res) {
- res = triangleTriangleIntersection(trianglesX0, trianglesY0, j,
- trianglesX1, trianglesY1, i);
-
- if (res) {
- intersect = true;
- break;
- }
- }
- }
- if (intersect) {
- break;
- }
- }
- break;
- default:
- throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
- ATypeTag.SERIALIZED_POINT_TYPE_TAG, ATypeTag.SERIALIZED_LINE_TYPE_TAG,
- ATypeTag.SERIALIZED_POLYGON_TYPE_TAG,
- ATypeTag.SERIALIZED_CIRCLE_TYPE_TAG,
- ATypeTag.SERIALIZED_RECTANGLE_TYPE_TAG);
- }
- break;
- default:
- throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
- ATypeTag.SERIALIZED_POINT_TYPE_TAG, ATypeTag.SERIALIZED_LINE_TYPE_TAG,
- ATypeTag.SERIALIZED_POLYGON_TYPE_TAG, ATypeTag.SERIALIZED_CIRCLE_TYPE_TAG,
- ATypeTag.SERIALIZED_RECTANGLE_TYPE_TAG);
- }
-
- ABoolean aResult = res ? ABoolean.TRUE : ABoolean.FALSE;
- aBooleanSerDer.serialize(aResult, out);
- result.set(resultStorage);
- }
- };
- }
- };
- }
-
- @Override
- public FunctionIdentifier getIdentifier() {
- return BuiltinFunctions.SPATIAL_INTERSECT;
- }
-}
diff --git a/asterixdb/asterix-geo/pom.xml b/asterixdb/asterix-geo/pom.xml
index 667db8e..6214d9c 100644
--- a/asterixdb/asterix-geo/pom.xml
+++ b/asterixdb/asterix-geo/pom.xml
@@ -21,7 +21,7 @@
<parent>
<artifactId>apache-asterixdb</artifactId>
<groupId>org.apache.asterix</groupId>
- <version>0.9.8-SNAPSHOT</version>
+ <version>0.9.8.2-SNAPSHOT</version>
</parent>
<artifactId>asterix-geo</artifactId>
diff --git a/asterixdb/asterix-lang-common/pom.xml b/asterixdb/asterix-lang-common/pom.xml
index e58125c..62567d1 100644
--- a/asterixdb/asterix-lang-common/pom.xml
+++ b/asterixdb/asterix-lang-common/pom.xml
@@ -21,7 +21,7 @@
<parent>
<artifactId>apache-asterixdb</artifactId>
<groupId>org.apache.asterix</groupId>
- <version>0.9.8-SNAPSHOT</version>
+ <version>0.9.8.2-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Statement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Statement.java
index 2594cdd..4330c4e 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Statement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Statement.java
@@ -107,6 +107,8 @@
CREATE_SYNONYM,
SYNONYM_DROP,
VIEW_DROP,
+ ANALYZE,
+ ANALYZE_DROP,
COMPACT,
EXTERNAL_DATASET_REFRESH,
SUBSCRIBE_FEED,
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/AnalyzeDropStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/AnalyzeDropStatement.java
new file mode 100644
index 0000000..67cfeec
--- /dev/null
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/AnalyzeDropStatement.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.lang.common.statement;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.lang.common.base.AbstractStatement;
+import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
+
+public class AnalyzeDropStatement extends AbstractStatement {
+
+ private final DataverseName dataverseName;
+ private final String datasetName;
+
+ public AnalyzeDropStatement(DataverseName dataverseName, String datasetName) {
+ this.dataverseName = dataverseName;
+ this.datasetName = datasetName;
+ }
+
+ @Override
+ public Kind getKind() {
+ return Kind.ANALYZE_DROP;
+ }
+
+ public DataverseName getDataverseName() {
+ return dataverseName;
+ }
+
+ public String getDatasetName() {
+ return datasetName;
+ }
+
+ @Override
+ public <R, T> R accept(ILangVisitor<R, T> visitor, T arg) throws CompilationException {
+ return visitor.visit(this, arg);
+ }
+
+ @Override
+ public byte getCategory() {
+ return Category.DDL;
+ }
+}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/AnalyzeStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/AnalyzeStatement.java
new file mode 100644
index 0000000..a719c34
--- /dev/null
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/AnalyzeStatement.java
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.lang.common.statement;
+
+import java.util.List;
+import java.util.Locale;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.lang.common.base.AbstractStatement;
+import org.apache.asterix.lang.common.base.Expression;
+import org.apache.asterix.lang.common.expression.FieldBinding;
+import org.apache.asterix.lang.common.expression.RecordConstructor;
+import org.apache.asterix.lang.common.util.ExpressionUtils;
+import org.apache.asterix.lang.common.util.LangRecordParseUtil;
+import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
+import org.apache.asterix.object.base.AdmBigIntNode;
+import org.apache.asterix.object.base.AdmDoubleNode;
+import org.apache.asterix.object.base.AdmObjectNode;
+import org.apache.asterix.object.base.AdmStringNode;
+import org.apache.asterix.object.base.IAdmNode;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+
+public class AnalyzeStatement extends AbstractStatement {
+
+ private static final String SAMPLE_FIELD_NAME = "sample";
+ private static final String SAMPLE_LOW = "low";
+ private static final String SAMPLE_MEDIUM = "medium";
+ private static final String SAMPLE_HIGH = "high";
+ private static final int SAMPLE_LOW_SIZE = 1063;
+ private static final int SAMPLE_MEDIUM_SIZE = SAMPLE_LOW_SIZE * 4;
+ private static final int SAMPLE_HIGH_SIZE = SAMPLE_MEDIUM_SIZE * 4;
+ private static final int SAMPLE_DEFAULT_SIZE = SAMPLE_LOW_SIZE;
+
+ private static final String SAMPLE_SEED_FIELD_NAME = "sample-seed";
+
+ private final DataverseName dataverseName;
+ private final String datasetName;
+ private final AdmObjectNode options;
+
+ public AnalyzeStatement(DataverseName dataverseName, String datasetName, RecordConstructor options)
+ throws CompilationException {
+ this.dataverseName = dataverseName;
+ this.datasetName = datasetName;
+ this.options = options == null ? null : validateOptions(options);
+ }
+
+ private static AdmObjectNode validateOptions(RecordConstructor options) throws CompilationException {
+ final List<FieldBinding> fbList = options.getFbList();
+ for (int i = 0; i < fbList.size(); i++) {
+ FieldBinding binding = fbList.get(i);
+ String key = LangRecordParseUtil.exprToStringLiteral(binding.getLeftExpr()).getStringValue();
+ Expression value = binding.getRightExpr();
+ switch (key) {
+ case SAMPLE_FIELD_NAME:
+ if (value.getKind() != Expression.Kind.LITERAL_EXPRESSION) {
+ throw new CompilationException(ErrorCode.INVALID_SAMPLE_SIZE);
+ }
+ break;
+ case SAMPLE_SEED_FIELD_NAME:
+ if (value.getKind() != Expression.Kind.LITERAL_EXPRESSION
+ && value.getKind() != Expression.Kind.UNARY_EXPRESSION) {
+ throw new CompilationException(ErrorCode.INVALID_SAMPLE_SEED);
+ }
+ break;
+ default:
+ throw new CompilationException(ErrorCode.INVALID_PARAM, key);
+ }
+ }
+ return (ExpressionUtils.toNode(options));
+ }
+
+ @Override
+ public Kind getKind() {
+ return Kind.ANALYZE;
+ }
+
+ public DataverseName getDataverseName() {
+ return dataverseName;
+ }
+
+ public String getDatasetName() {
+ return datasetName;
+ }
+
+ public int getSampleSize() throws CompilationException {
+ IAdmNode n = getOption(SAMPLE_FIELD_NAME);
+ if (n == null) {
+ return SAMPLE_DEFAULT_SIZE;
+ }
+ switch (n.getType()) {
+ case STRING:
+ String s = ((AdmStringNode) n).get();
+ switch (s.toLowerCase(Locale.ROOT)) {
+ case SAMPLE_LOW:
+ return SAMPLE_LOW_SIZE;
+ case SAMPLE_MEDIUM:
+ return SAMPLE_MEDIUM_SIZE;
+ case SAMPLE_HIGH:
+ return SAMPLE_HIGH_SIZE;
+ default:
+ throw new CompilationException(ErrorCode.INVALID_SAMPLE_SIZE);
+ }
+ case BIGINT:
+ int v = (int) ((AdmBigIntNode) n).get();
+ if (!isValidSampleSize(v)) {
+ throw new CompilationException(ErrorCode.OUT_OF_RANGE_SAMPLE_SIZE, SAMPLE_LOW_SIZE,
+ SAMPLE_HIGH_SIZE);
+ }
+ return v;
+ case DOUBLE:
+ v = (int) ((AdmDoubleNode) n).get();
+ if (!isValidSampleSize(v)) {
+ throw new CompilationException(ErrorCode.OUT_OF_RANGE_SAMPLE_SIZE, SAMPLE_LOW_SIZE,
+ SAMPLE_HIGH_SIZE);
+ }
+ return v;
+ default:
+ throw new CompilationException(ErrorCode.WITH_FIELD_MUST_BE_OF_TYPE, SAMPLE_FIELD_NAME,
+ BuiltinType.ASTRING.getTypeName(), n.getType().toString());
+ }
+ }
+
+ public long getOrCreateSampleSeed() throws AlgebricksException {
+ IAdmNode n = getOption(SAMPLE_SEED_FIELD_NAME);
+ return n != null ? getSampleSeed(n) : createSampleSeed();
+ }
+
+ private long getSampleSeed(IAdmNode n) throws CompilationException {
+ switch (n.getType()) {
+ case BIGINT:
+ return ((AdmBigIntNode) n).get();
+ case DOUBLE:
+ return (long) ((AdmDoubleNode) n).get();
+ case STRING:
+ String s = ((AdmStringNode) n).get();
+ try {
+ return Long.parseLong(s);
+ } catch (NumberFormatException e) {
+ throw new CompilationException(ErrorCode.INVALID_SAMPLE_SEED);
+ }
+ default:
+ throw new CompilationException(ErrorCode.WITH_FIELD_MUST_BE_OF_TYPE, SAMPLE_SEED_FIELD_NAME,
+ BuiltinType.AINT64.getTypeName(), n.getType().toString());
+ }
+ }
+
+ private long createSampleSeed() {
+ return System.nanoTime() + System.identityHashCode(this);
+ }
+
+ private boolean isValidSampleSize(int v) {
+ return v >= SAMPLE_LOW_SIZE && v <= SAMPLE_HIGH_SIZE;
+ }
+
+ private IAdmNode getOption(String optionName) {
+ return options != null ? options.get(optionName) : null;
+ }
+
+ @Override
+ public <R, T> R accept(ILangVisitor<R, T> visitor, T arg) throws CompilationException {
+ return visitor.visit(this, arg);
+ }
+
+ @Override
+ public byte getCategory() {
+ return Category.DDL;
+ }
+}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/IndexDropStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/IndexDropStatement.java
index c3583698..d479626 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/IndexDropStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/IndexDropStatement.java
@@ -27,10 +27,10 @@
public class IndexDropStatement extends AbstractStatement {
- private DataverseName dataverseName;
- private Identifier datasetName;
- private Identifier indexName;
- private boolean ifExists;
+ private final DataverseName dataverseName;
+ private final Identifier datasetName;
+ private final Identifier indexName;
+ private final boolean ifExists;
public IndexDropStatement(DataverseName dataverseName, Identifier datasetName, Identifier indexName,
boolean ifExists) {
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/ExpressionUtils.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/ExpressionUtils.java
index cab1aca..5c91aef 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/ExpressionUtils.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/ExpressionUtils.java
@@ -42,6 +42,7 @@
import org.apache.asterix.lang.common.expression.ListConstructor;
import org.apache.asterix.lang.common.expression.LiteralExpr;
import org.apache.asterix.lang.common.expression.RecordConstructor;
+import org.apache.asterix.lang.common.expression.UnaryExpr;
import org.apache.asterix.lang.common.literal.DoubleLiteral;
import org.apache.asterix.lang.common.literal.FloatLiteral;
import org.apache.asterix.lang.common.literal.IntegerLiteral;
@@ -50,6 +51,7 @@
import org.apache.asterix.lang.common.statement.FunctionDecl;
import org.apache.asterix.lang.common.statement.Query;
import org.apache.asterix.lang.common.statement.ViewDecl;
+import org.apache.asterix.lang.common.struct.UnaryExprType;
import org.apache.asterix.lang.common.visitor.GatherFunctionCallsVisitor;
import org.apache.asterix.object.base.AdmArrayNode;
import org.apache.asterix.object.base.AdmBigIntNode;
@@ -80,6 +82,26 @@
return toNode((LiteralExpr) expr);
case RECORD_CONSTRUCTOR_EXPRESSION:
return toNode((RecordConstructor) expr);
+ case UNARY_EXPRESSION:
+ UnaryExpr unaryExpr = (UnaryExpr) expr;
+ UnaryExprType unaryExprType = unaryExpr.getExprType();
+ if (unaryExprType == UnaryExprType.POSITIVE || unaryExprType == UnaryExprType.NEGATIVE) {
+ Expression uexpr = unaryExpr.getExpr();
+ if (uexpr.getKind() == Expression.Kind.LITERAL_EXPRESSION) {
+ if (unaryExprType == UnaryExprType.POSITIVE) {
+ return toNode(uexpr);
+ } else {
+ Literal lit = ((LiteralExpr) uexpr).getValue();
+ return toNode(new LiteralExpr(reverseSign(lit)));
+ }
+ } else {
+ throw new CompilationException(ErrorCode.LITERAL_TYPE_NOT_SUPPORTED_IN_CONSTANT_RECORD,
+ uexpr.getKind());
+ }
+ } else {
+ throw new CompilationException(ErrorCode.EXPRESSION_NOT_SUPPORTED_IN_CONSTANT_RECORD,
+ unaryExprType);
+ }
default:
throw new CompilationException(ErrorCode.EXPRESSION_NOT_SUPPORTED_IN_CONSTANT_RECORD, expr.getKind());
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/FunctionUtil.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/FunctionUtil.java
index 42a35f3..24b38ab 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/FunctionUtil.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/FunctionUtil.java
@@ -21,11 +21,13 @@
import java.io.StringReader;
import java.util.ArrayList;
+import java.util.BitSet;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Objects;
+import java.util.Set;
import java.util.function.BiFunction;
import org.apache.asterix.common.exceptions.AsterixException;
@@ -58,6 +60,7 @@
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.common.utils.Triple;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.IWarningCollector;
@@ -66,6 +69,8 @@
public class FunctionUtil {
public static final String IMPORT_PRIVATE_FUNCTIONS = "import-private-functions";
+ //TODO(wyk) add Multiply and Add
+ private static final Set<FunctionIdentifier> COMMUTATIVE_FUNCTIONS = Set.of(BuiltinFunctions.EQ);
private static final DataverseName FN_DATASET_DATAVERSE_NAME =
FunctionSignature.getDataverseName(BuiltinFunctions.DATASET);
@@ -303,4 +308,68 @@
function.getSignature(), e.getMessage());
}
}
+
+ public static boolean isFieldAccessFunction(ILogicalExpression expression) {
+ if (expression.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+ return false;
+ }
+
+ AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expression;
+ FunctionIdentifier fid = funcExpr.getFunctionIdentifier();
+
+ return BuiltinFunctions.FIELD_ACCESS_BY_INDEX.equals(fid) || BuiltinFunctions.FIELD_ACCESS_BY_NAME.equals(fid)
+ || BuiltinFunctions.FIELD_ACCESS_NESTED.equals(fid);
+ }
+
+ /**
+ * Compares two commutative expressions
+ * TODO It doesn't support add and multiply (e.g., add(x, add(y, z) & add(add(x, y), z) will return false)
+ *
+ * @param expr1 left expression
+ * @param expr2 right expression
+ * @return true if equals, false otherwise
+ */
+ public static boolean commutativeEquals(ILogicalExpression expr1, ILogicalExpression expr2) {
+ if (expr1.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL
+ || expr2.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+ return expr1.equals(expr2);
+ }
+
+ AbstractFunctionCallExpression funcExpr1 = (AbstractFunctionCallExpression) expr1;
+ AbstractFunctionCallExpression funcExpr2 = (AbstractFunctionCallExpression) expr2;
+
+ FunctionIdentifier fid1 = funcExpr1.getFunctionIdentifier();
+ FunctionIdentifier fid2 = funcExpr2.getFunctionIdentifier();
+
+ if (!fid1.equals(fid2) || funcExpr1.getArguments().size() != funcExpr2.getArguments().size()) {
+ return false;
+ } else if (!COMMUTATIVE_FUNCTIONS.contains(fid1)) {
+ return expr1.equals(expr2);
+ }
+
+ List<Mutable<ILogicalExpression>> args1 = funcExpr1.getArguments();
+ List<Mutable<ILogicalExpression>> args2 = funcExpr2.getArguments();
+
+ BitSet matched = new BitSet();
+ int numberOfMatches = 0;
+ for (Mutable<ILogicalExpression> arg1 : args1) {
+ int prevNumberOfMatches = numberOfMatches;
+
+ for (int i = 0; i < args2.size(); i++) {
+ Mutable<ILogicalExpression> arg2 = args2.get(i);
+ if (!matched.get(i) && commutativeEquals(arg1.getValue(), arg2.getValue())) {
+ matched.set(i);
+ numberOfMatches++;
+ break;
+ }
+ }
+
+ if (numberOfMatches == prevNumberOfMatches) {
+ // Early exit as one operand didn't match with any of the other operands
+ return false;
+ }
+ }
+
+ return numberOfMatches == args1.size();
+ }
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/AbstractInlineUdfsVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/AbstractInlineUdfsVisitor.java
index 52775d3..0fcdc99 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/AbstractInlineUdfsVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/AbstractInlineUdfsVisitor.java
@@ -153,7 +153,14 @@
public Boolean visit(IndexAccessor fa, Void arg) throws CompilationException {
Pair<Boolean, Expression> p = inlineUdfsAndViewsInExpr(fa.getExpr());
fa.setExpr(p.second);
- return p.first;
+ boolean inlined = p.first;
+ Expression indexExpr = fa.getIndexExpr();
+ if (indexExpr != null) {
+ Pair<Boolean, Expression> p2 = inlineUdfsAndViewsInExpr(indexExpr);
+ fa.setIndexExpr(p2.second);
+ inlined |= p2.first;
+ }
+ return inlined;
}
@Override
@@ -250,7 +257,9 @@
@Override
public Boolean visit(UnaryExpr u, Void arg) throws CompilationException {
- return u.getExpr().accept(this, arg);
+ Pair<Boolean, Expression> p = inlineUdfsAndViewsInExpr(u.getExpr());
+ u.setExpr(p.second);
+ return p.first;
}
@Override
@@ -270,7 +279,7 @@
if (returnExpression != null) {
Pair<Boolean, Expression> rewrittenReturnExpr = inlineUdfsAndViewsInExpr(returnExpression);
insert.setReturnExpression(rewrittenReturnExpr.second);
- changed |= rewrittenReturnExpr.first;
+ changed = rewrittenReturnExpr.first;
}
Pair<Boolean, Expression> rewrittenBodyExpression = inlineUdfsAndViewsInExpr(insert.getBody());
insert.setBody(rewrittenBodyExpression.second);
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java
index 0ddbeb4..f42f2f0 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java
@@ -65,6 +65,8 @@
import org.apache.asterix.lang.common.expression.UnorderedListTypeDefinition;
import org.apache.asterix.lang.common.expression.VariableExpr;
import org.apache.asterix.lang.common.statement.AdapterDropStatement;
+import org.apache.asterix.lang.common.statement.AnalyzeDropStatement;
+import org.apache.asterix.lang.common.statement.AnalyzeStatement;
import org.apache.asterix.lang.common.statement.CompactStatement;
import org.apache.asterix.lang.common.statement.ConnectFeedStatement;
import org.apache.asterix.lang.common.statement.CreateAdapterStatement;
@@ -929,6 +931,23 @@
}
@Override
+ public Void visit(AnalyzeStatement as, Integer step) throws CompilationException {
+ out.print(skip(step) + "analyze dataset ");
+ out.print(generateFullName(as.getDataverseName(), as.getDatasetName()));
+ out.println(SEMICOLON);
+ return null;
+ }
+
+ @Override
+ public Void visit(AnalyzeDropStatement as, Integer step) throws CompilationException {
+ out.print(skip(step) + "analyze dataset ");
+ out.print(generateFullName(as.getDataverseName(), as.getDatasetName()));
+ out.print(" drop statistics");
+ out.println(SEMICOLON);
+ return null;
+ }
+
+ @Override
public Void visit(CompactStatement del, Integer step) throws CompilationException {
return null;
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/AbstractQueryExpressionVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/AbstractQueryExpressionVisitor.java
index 91fe664..a060d1e 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/AbstractQueryExpressionVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/AbstractQueryExpressionVisitor.java
@@ -25,6 +25,8 @@
import org.apache.asterix.lang.common.expression.TypeReferenceExpression;
import org.apache.asterix.lang.common.expression.UnorderedListTypeDefinition;
import org.apache.asterix.lang.common.statement.AdapterDropStatement;
+import org.apache.asterix.lang.common.statement.AnalyzeDropStatement;
+import org.apache.asterix.lang.common.statement.AnalyzeStatement;
import org.apache.asterix.lang.common.statement.CompactStatement;
import org.apache.asterix.lang.common.statement.ConnectFeedStatement;
import org.apache.asterix.lang.common.statement.CreateAdapterStatement;
@@ -261,6 +263,16 @@
}
@Override
+ public R visit(AnalyzeStatement as, T arg) throws CompilationException {
+ return null;
+ }
+
+ @Override
+ public R visit(AnalyzeDropStatement as, T arg) throws CompilationException {
+ return null;
+ }
+
+ @Override
public R visit(CompactStatement del, T arg) throws CompilationException {
return null;
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/ILangVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/ILangVisitor.java
index c2b1311..b9de347 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/ILangVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/ILangVisitor.java
@@ -42,6 +42,8 @@
import org.apache.asterix.lang.common.expression.UnorderedListTypeDefinition;
import org.apache.asterix.lang.common.expression.VariableExpr;
import org.apache.asterix.lang.common.statement.AdapterDropStatement;
+import org.apache.asterix.lang.common.statement.AnalyzeDropStatement;
+import org.apache.asterix.lang.common.statement.AnalyzeStatement;
import org.apache.asterix.lang.common.statement.CompactStatement;
import org.apache.asterix.lang.common.statement.ConnectFeedStatement;
import org.apache.asterix.lang.common.statement.CreateAdapterStatement;
@@ -207,6 +209,10 @@
R visit(SynonymDropStatement del, T arg) throws CompilationException;
+ R visit(AnalyzeStatement as, T arg) throws CompilationException;
+
+ R visit(AnalyzeDropStatement as, T step) throws CompilationException;
+
R visit(CompactStatement del, T arg) throws CompilationException;
R visit(ListSliceExpression expression, T arg) throws CompilationException;
diff --git a/asterixdb/asterix-lang-sqlpp/pom.xml b/asterixdb/asterix-lang-sqlpp/pom.xml
index 0248ed7..0485509 100644
--- a/asterixdb/asterix-lang-sqlpp/pom.xml
+++ b/asterixdb/asterix-lang-sqlpp/pom.xml
@@ -21,7 +21,7 @@
<parent>
<artifactId>apache-asterixdb</artifactId>
<groupId>org.apache.asterix</groupId>
- <version>0.9.8-SNAPSHOT</version>
+ <version>0.9.8.2-SNAPSHOT</version>
</parent>
<licenses>
<license>
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/parser/SqlppHint.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/parser/SqlppHint.java
index 925140b..da3504d 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/parser/SqlppHint.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/parser/SqlppHint.java
@@ -33,6 +33,7 @@
DATETIME_ADD_RAND_HOURS_HINT("datetime-add-rand-hours"),
DATETIME_BETWEEN_YEARS_HINT("datetime-between-years"),
HASH_GROUP_BY_HINT("hash"),
+ HASH_JOIN_HINT("hashjoin"),
INDEXED_NESTED_LOOP_JOIN_HINT("indexnl"),
INMEMORY_HINT("inmem"),
INSERT_RAND_INT_HINT("insert-rand-int"),
@@ -46,6 +47,9 @@
VAL_FILE_HINT("val-files"),
VAL_FILE_SAME_INDEX_HINT("val-file-same-idx"),
GEN_FIELDS_HINT("gen-fields"),
+ SINGLE_DATASET_PREDICATE_SELECTIVITY_HINT("selectivity"),
+ JOIN_PREDICATE_PRODUCTIVITY_HINT("productivity"),
+ SUBPATH_HINT("subpath"),
// data generator hints
DGEN_HINT("dgen");
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/OperatorExpressionVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/OperatorExpressionVisitor.java
index 93a14ca..7967f36 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/OperatorExpressionVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/OperatorExpressionVisitor.java
@@ -45,6 +45,7 @@
import org.apache.asterix.lang.sqlpp.visitor.base.AbstractSqlppExpressionScopingVisitor;
import org.apache.asterix.om.functions.BuiltinFunctions;
import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionAnnotation;
+import org.apache.hyracks.algebricks.core.algebra.expressions.PredicateCardinalityAnnotation;
import org.apache.hyracks.api.exceptions.SourceLocation;
public class OperatorExpressionVisitor extends AbstractSqlppExpressionScopingVisitor {
@@ -85,6 +86,7 @@
private Expression processLikeOperator(OperatorExpr operatorExpr, OperatorType opType) {
CallExpr likeExpr =
new CallExpr(new FunctionSignature(BuiltinFunctions.STRING_LIKE), operatorExpr.getExprList());
+ likeExpr.addHints(operatorExpr.getHints());
likeExpr.setSourceLocation(operatorExpr.getSourceLocation());
switch (opType) {
case LIKE:
@@ -135,6 +137,20 @@
return callExpr;
}
+ private List<IExpressionAnnotation> removeSelectivityHints(OperatorExpr expr) {
+ if (expr.hasHints()) {
+ List<IExpressionAnnotation> copyHintsExceptSelectivity = new ArrayList<>();
+ for (IExpressionAnnotation h : expr.getHints()) {
+ if (!(h.getClass().equals(PredicateCardinalityAnnotation.class))) {
+ copyHintsExceptSelectivity.add(h);
+ }
+ }
+ return copyHintsExceptSelectivity;
+ } else {
+ return expr.getHints();
+ }
+ }
+
private Expression processBetweenOperator(OperatorExpr operatorExpr, OperatorType opType)
throws CompilationException {
// The grammar guarantees that the BETWEEN operator gets exactly three expressions.
@@ -147,8 +163,11 @@
operatorExpr.getSourceLocation());
// Creates the expression target <= right.
Expression targetCopy = (Expression) SqlppRewriteUtil.deepCopy(target);
+
+ // remove any selectivity hints from operatorExpr; do not want to duplicate those hints
Expression rightComparison = createOperatorExpression(OperatorType.LE, targetCopy, right,
- operatorExpr.getHints(), operatorExpr.getSourceLocation());
+ removeSelectivityHints(operatorExpr), operatorExpr.getSourceLocation());
+
Expression andExpr = createOperatorExpression(OperatorType.AND, leftComparison, rightComparison, null,
operatorExpr.getSourceLocation());
switch (opType) {
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/Sql92AggregateFunctionVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/Sql92AggregateFunctionVisitor.java
index f93e843..73f1202 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/Sql92AggregateFunctionVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/Sql92AggregateFunctionVisitor.java
@@ -187,7 +187,8 @@
new FieldAccessor(groupItemVar, new VarIdentifier(groupVarFieldMap.get(usedVar).getValue()));
fa.setSourceLocation(usedVar.getSourceLocation());
varExprMap.put(usedVar, fa);
- } else if (outerVars.contains(usedVar)) {
+ } else if (outerVars.contains(usedVar)
+ || SqlppVariableUtil.isExternalVariableIdentifier(usedVar.getVar())) {
// Do nothing
} else if (preGroupUnmappedVars != null && preGroupUnmappedVars.contains(usedVar)) {
throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_USE_OF_IDENTIFIER,
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/VariableCheckAndRewriteVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/VariableCheckAndRewriteVisitor.java
index 4b0caca..8fbad4f 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/VariableCheckAndRewriteVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/VariableCheckAndRewriteVisitor.java
@@ -203,6 +203,7 @@
argList.add(new LiteralExpr(new StringLiteral(datasetName)));
}
CallExpr callExpr = new CallExpr(new FunctionSignature(BuiltinFunctions.DATASET), argList);
+ callExpr.addHints(varExpr.getHints());
callExpr.setSourceLocation(sourceLoc);
return callExpr;
}
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
index 4e64944..eda67bb 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
+++ b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
@@ -52,6 +52,7 @@
import org.apache.asterix.common.annotations.DateBetweenYearsDataGen;
import org.apache.asterix.common.annotations.DatetimeAddRandHoursDataGen;
import org.apache.asterix.common.annotations.DatetimeBetweenYearsDataGen;
+import org.apache.asterix.common.annotations.ExternalSubpathAnnotation;
import org.apache.asterix.common.annotations.FieldIntervalDataGen;
import org.apache.asterix.common.annotations.FieldValFileDataGen;
import org.apache.asterix.common.annotations.FieldValFileSameIndexDataGen;
@@ -76,6 +77,7 @@
import org.apache.asterix.common.functions.FunctionSignature;
import org.apache.asterix.common.metadata.DataverseName;
import org.apache.asterix.common.metadata.DatasetFullyQualifiedName;
+import org.apache.asterix.external.dataset.adapter.AdapterIdentifier;
import org.apache.asterix.lang.common.base.AbstractClause;
import org.apache.asterix.lang.common.base.AbstractLangExpression;
import org.apache.asterix.lang.common.base.AbstractStatement;
@@ -124,6 +126,8 @@
import org.apache.asterix.lang.common.literal.TrueLiteral;
import org.apache.asterix.lang.common.parser.ScopeChecker;
import org.apache.asterix.lang.common.statement.AdapterDropStatement;
+import org.apache.asterix.lang.common.statement.AnalyzeStatement;
+import org.apache.asterix.lang.common.statement.AnalyzeDropStatement;
import org.apache.asterix.lang.common.statement.CompactStatement;
import org.apache.asterix.lang.common.statement.ConnectFeedStatement;
import org.apache.asterix.lang.common.statement.StartFeedStatement;
@@ -205,7 +209,7 @@
import org.apache.asterix.lang.sqlpp.util.ExpressionToVariableUtil;
import org.apache.asterix.lang.sqlpp.util.FunctionMapUtil;
import org.apache.asterix.lang.sqlpp.util.SqlppVariableUtil;
-import org.apache.asterix.external.dataset.adapter.AdapterIdentifier;
+import org.apache.asterix.metadata.utils.MetadataConstants;
import org.apache.asterix.om.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.BuiltinFunctions;
import org.apache.asterix.om.types.BuiltinType;
@@ -215,7 +219,10 @@
import org.apache.hyracks.algebricks.common.utils.Triple;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.expressions.BroadcastExpressionAnnotation;
+import org.apache.hyracks.algebricks.core.algebra.expressions.HashJoinExpressionAnnotation;
import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionAnnotation;
+import org.apache.hyracks.algebricks.core.algebra.expressions.JoinProductivityAnnotation;
+import org.apache.hyracks.algebricks.core.algebra.expressions.PredicateCardinalityAnnotation;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.IWarningCollector;
import org.apache.hyracks.api.exceptions.SourceLocation;
@@ -223,6 +230,8 @@
import org.apache.hyracks.dataflow.common.data.partition.range.RangeMap;
import org.apache.hyracks.util.LogRedactionUtil;
import org.apache.hyracks.util.StringUtil;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
class SQLPPParser extends ScopeChecker implements IParser {
@@ -257,6 +266,7 @@
private static final String REPLACE = "REPLACE";
private static final String RETURNS = "RETURNS";
private static final String CONFIG = "CONFIG";
+ private static final String STATISTICS = "STATISTICS";
private static final String INT_TYPE_NAME = "int";
@@ -405,6 +415,32 @@
return new SQLPPParser(text).parseParenthesizedIdentifierList();
}
+ private Pair<HashJoinExpressionAnnotation.BuildOrProbe, String> parseHashJoinParams() throws CompilationException {
+ return parseImpl(new ParseFunction<Pair<HashJoinExpressionAnnotation.BuildOrProbe, String>>() {
+ @Override
+ public Pair<HashJoinExpressionAnnotation.BuildOrProbe, String> parse() throws ParseException {
+ return SQLPPParser.this.buildOrProbeParenthesizedIdentifier();
+ }
+ });
+ }
+
+ private static Pair<HashJoinExpressionAnnotation.BuildOrProbe, String> parseHashJoinParams(String text) throws CompilationException {
+ return new SQLPPParser(text).parseHashJoinParams();
+ }
+
+ private String parseBroadcastJoinParams() throws CompilationException {
+ return parseImpl(new ParseFunction<String>() {
+ @Override
+ public String parse() throws ParseException {
+ return SQLPPParser.this.parenthesizedIdentifier();
+ }
+ });
+ }
+
+ private static String parseBroadcastJoinParams(String text) throws CompilationException {
+ return new SQLPPParser(text).parseBroadcastJoinParams();
+ }
+
private List<Literal> parseParenthesizedLiteralList() throws CompilationException {
return parseImpl(new ParseFunction<List<Literal>>() {
@Override
@@ -647,10 +683,77 @@
private IExpressionAnnotation parseExpressionAnnotation(Token hintToken) {
// placeholder for the annotation that should be returned if this hint's parameters cannot be parsed
IExpressionAnnotation onParseErrorReturn = null;
+ double selectivity, cardinality, productivity;
+ Pattern number = Pattern.compile("\\d+\\.\\d+");
+ Pattern stringNumber = Pattern.compile("\\w+\\s+\\d+\\.\\d+");
+ Pattern lessThanOnePat = Pattern.compile("0\\.\\d+");
try {
switch (hintToken.hint) {
+ case SINGLE_DATASET_PREDICATE_SELECTIVITY_HINT:
+ if (hintToken.hintParams == null) {
+ throw new SqlppParseException(getSourceLocation(hintToken), "Expected selectivity value");
+ }
+ else {
+ selectivity = 1.0; // uninitialized
+ Matcher mat = lessThanOnePat.matcher(hintToken.hintParams);
+ if (mat.find()) {
+ selectivity = Double.parseDouble (mat.group());
+ }
+ else {
+ throw new SqlppParseException(getSourceLocation(hintToken), "Selectivity has to be a decimal value greater than 0 and less than 1");
+ }
+ return new PredicateCardinalityAnnotation(selectivity);
+ }
+ case JOIN_PREDICATE_PRODUCTIVITY_HINT:
+ if (hintToken.hintParams == null) {
+ throw new SqlppParseException(getSourceLocation(hintToken), "Expected productivity collection name and value");
+ }
+ else {
+ productivity = 1.0; // uninitialized
+ String leftSideDataSet = null;
+ Matcher StringNum = stringNumber.matcher(hintToken.hintParams);
+ if (StringNum.find()) {
+ String matchedGroup = StringNum.group();
+ Pattern var = Pattern.compile("[a-zA-Z]\\w*"); // any word character [a-zA-Z_0-9]
+ Matcher matVar = var.matcher(matchedGroup);
+ if (matVar.find()) {
+ leftSideDataSet = matVar.group();
+ }
+ else {
+ throw new SqlppParseException(getSourceLocation(hintToken), "Expected productivity collection name");
+ }
+ Matcher numMat = number.matcher(matchedGroup);
+ if (numMat.find()) {
+ productivity = Double.parseDouble (numMat.group());
+ }
+ else {
+ throw new SqlppParseException(getSourceLocation(hintToken), "Productivity has to be a decimal value greater than 0");
+ }
+ }
+ else {
+ throw new SqlppParseException(getSourceLocation(hintToken), "Invalid format for productivity values");
+ }
+ // attach hint to global scope
+ return new JoinProductivityAnnotation (productivity, leftSideDataSet);
+ }
case HASH_BROADCAST_JOIN_HINT:
- return new BroadcastExpressionAnnotation(BroadcastExpressionAnnotation.BroadcastSide.RIGHT);
+ if (hintToken.hintParams == null) {
+ return new BroadcastExpressionAnnotation(BroadcastExpressionAnnotation.BroadcastSide.RIGHT);
+ }
+ else {
+ // if parameter parsing fails then ignore this hint.
+ String name = parseBroadcastJoinParams(hintToken.hintParams);
+ return new BroadcastExpressionAnnotation(name);
+ }
+ case HASH_JOIN_HINT:
+ if (hintToken.hintParams == null) {
+ throw new SqlppParseException(getSourceLocation(hintToken), "Expected hash join build/probe collection name");
+ }
+ else {
+ // if parameter parsing fails then ignore this hint.
+ Pair<HashJoinExpressionAnnotation.BuildOrProbe, String> pair = parseHashJoinParams(hintToken.hintParams);
+ return new HashJoinExpressionAnnotation(pair);
+ }
case INDEXED_NESTED_LOOP_JOIN_HINT:
if (hintToken.hintParams == null) {
return IndexedNLJoinExpressionAnnotation.INSTANCE_ANY_INDEX;
@@ -836,6 +939,7 @@
| stmt = UpsertStatement()
| stmt = ConnectionStatement()
| stmt = CompactStatement()
+ | stmt = AnalyzeStatement()
| stmt = Query()
| stmt = RefreshExternalDatasetStatement()
)
@@ -987,6 +1091,13 @@
(<DATASET>|<COLLECTION>)
}
+void DatasetToken() throws ParseException:
+{
+}
+{
+ Dataset()
+}
+
DatasetDecl CreateDatasetStatement(Token startStmtToken) throws ParseException:
{
DatasetDecl stmt = null;
@@ -1378,7 +1489,7 @@
<ON> nameComponents = QualifiedName() (<TYPE> <BTREE>)?
{
if (indexName == null) {
- indexName = "primary_idx_" + nameComponents.second;
+ indexName = MetadataConstants.PRIMARY_INDEX_PREFIX + nameComponents.second;
}
CreateIndexStatement stmt = new CreateIndexStatement(nameComponents.first, nameComponents.second,
new Identifier(indexName), IndexType.BTREE, Collections.emptyList(), false, -1, null, ifNotExists, null, null,
@@ -2628,6 +2739,50 @@
}
}
+Statement AnalyzeStatement() throws ParseException:
+{
+ Token startToken = null;
+ Statement stmt = null;
+ Pair<DataverseName,Identifier> nameComponents = null;
+}
+{
+ <ANALYZE> { startToken = token; } DatasetToken() nameComponents = QualifiedName()
+ (
+ stmt = AnalyzeDatasetDropStatement(startToken, nameComponents.first, nameComponents.second)
+ | stmt = AnalyzeDatasetStatement(startToken, nameComponents.first, nameComponents.second)
+ )
+ {
+ return stmt;
+ }
+}
+
+Statement AnalyzeDatasetStatement(Token startToken, DataverseName dvName, Identifier identifier) throws ParseException:
+{
+ RecordConstructor withRecord = null;
+}
+{
+ ( <WITH> withRecord = RecordConstructor() )?
+ {
+ try {
+ AnalyzeStatement stmt = new AnalyzeStatement(dvName, identifier.getValue(), withRecord);
+ return addSourceLocation(stmt, startToken);
+ } catch (CompilationException e) {
+ throw new SqlppParseException(getSourceLocation(startToken), e.getMessage());
+ }
+ }
+}
+
+Statement AnalyzeDatasetDropStatement(Token startToken, DataverseName dvName, Identifier identifier) throws ParseException:
+{
+}
+{
+ <DROP> <IDENTIFIER> { expectToken(STATISTICS); }
+ {
+ AnalyzeDropStatement stmt = new AnalyzeDropStatement(dvName, identifier.getValue());
+ return addSourceLocation(stmt, startToken);
+ }
+}
+
Statement CompactStatement() throws ParseException:
{
Token startToken = null;
@@ -2994,7 +3149,7 @@
// Note: there's a copy of this production in PrimaryExpr() (LOOKAHEAD for FunctionCallExpr())
// that copy must be kept in sync with this code
prefix = MultipartIdentifierWithHints(
- SqlppHint.INDEXED_NESTED_LOOP_JOIN_HINT, SqlppHint.RANGE_HINT, SqlppHint.SKIP_SECONDARY_INDEX_SEARCH_HINT,
+ SqlppHint.INDEXED_NESTED_LOOP_JOIN_HINT, SqlppHint.RANGE_HINT, SqlppHint.HASH_JOIN_HINT, SqlppHint.SKIP_SECONDARY_INDEX_SEARCH_HINT,
SqlppHint.SPATIAL_JOIN_HINT, SqlppHint.USE_SECONDARY_INDEX_SEARCH_HINT
)
(<SHARP> suffix = Identifier())?
@@ -3074,6 +3229,39 @@
}
}
+Pair<HashJoinExpressionAnnotation.BuildOrProbe, String> buildOrProbeParenthesizedIdentifier() throws ParseException:
+{
+ String ident1 = null;
+ String ident2 = null;
+}
+{
+ ident1 = Identifier() <LEFTPAREN> ident2 = Identifier() <RIGHTPAREN>
+ {
+ // check
+ if (ident1.equals("build")) {
+ return new Pair<HashJoinExpressionAnnotation.BuildOrProbe, String>(HashJoinExpressionAnnotation.BuildOrProbe.BUILD, ident2);
+ }
+ else if (ident1.equals("probe")) {
+ return new Pair<HashJoinExpressionAnnotation.BuildOrProbe, String>(HashJoinExpressionAnnotation.BuildOrProbe.PROBE, ident2);
+ }
+ else {
+ throw new SqlppParseException(getSourceLocation(token), "The string after hashjoin has to be \"build\" or \"probe\".");
+ }
+ return null;
+ }
+}
+
+String parenthesizedIdentifier() throws ParseException:
+{
+ String ident = null;
+}
+{
+ <LEFTPAREN> ident = Identifier() <RIGHTPAREN>
+ {
+ return ident;
+ }
+}
+
List<Literal> ParenthesizedLiteralList() throws ParseException:
{
List<Literal> list = new ArrayList<Literal>();
@@ -3481,6 +3669,7 @@
Token opToken = null;
Expression operand = null;
IExpressionAnnotation annotation = null;
+ List<IExpressionAnnotation> annotationList = new ArrayList<IExpressionAnnotation>();
}
{
operand = BetweenExpr()
@@ -3494,10 +3683,20 @@
}
Token hintToken = fetchHint(token,
SqlppHint.HASH_BROADCAST_JOIN_HINT, SqlppHint.INDEXED_NESTED_LOOP_JOIN_HINT,
- SqlppHint.SKIP_SECONDARY_INDEX_SEARCH_HINT, SqlppHint.USE_SECONDARY_INDEX_SEARCH_HINT
+ SqlppHint.HASH_JOIN_HINT, SqlppHint.SKIP_SECONDARY_INDEX_SEARCH_HINT, SqlppHint.USE_SECONDARY_INDEX_SEARCH_HINT,
+ SqlppHint.SINGLE_DATASET_PREDICATE_SELECTIVITY_HINT, SqlppHint.JOIN_PREDICATE_PRODUCTIVITY_HINT
);
- if (hintToken != null) {
+ while (hintToken != null) {
annotation = parseExpressionAnnotation(hintToken);
+ if (annotation != null) {
+ // annotation may be null if hints are malformed
+ annotationList.add(annotation);
+ }
+ hintToken = hintToken.specialToken;
+ if (hintToken != null) {
+ SourceLocation sourceLoc = getSourceLocation(hintToken);
+ hintCollector.remove(sourceLoc);
+ }
}
String operator = opToken.image.toLowerCase();
if (operator.equals("<>")){
@@ -3527,7 +3726,7 @@
{
if (annotation != null) {
- op.addHint(annotation);
+ op.addHints(annotationList);
}
return op==null? operand: op;
}
@@ -3547,8 +3746,8 @@
(<NOT> { not = true; })? <BETWEEN>
{
Token hintToken = fetchHint(token,
- SqlppHint.INDEXED_NESTED_LOOP_JOIN_HINT, SqlppHint.SKIP_SECONDARY_INDEX_SEARCH_HINT,
- SqlppHint.USE_SECONDARY_INDEX_SEARCH_HINT
+ SqlppHint.INDEXED_NESTED_LOOP_JOIN_HINT, SqlppHint.SKIP_SECONDARY_INDEX_SEARCH_HINT, SqlppHint.HASH_JOIN_HINT,
+ SqlppHint.USE_SECONDARY_INDEX_SEARCH_HINT, SqlppHint.SINGLE_DATASET_PREDICATE_SELECTIVITY_HINT
);
if (hintToken != null) {
annotation = parseExpressionAnnotation(hintToken);
@@ -3632,6 +3831,7 @@
boolean not = false;
OperatorExpr op = null;
Expression operand = null;
+ IExpressionAnnotation annotation = null;
}
{
operand = ConcatExpr()
@@ -3639,6 +3839,10 @@
LOOKAHEAD(2)
(<NOT> { not = true; })? <LIKE>
{
+ Token hintToken = fetchHint(token, SqlppHint.SINGLE_DATASET_PREDICATE_SELECTIVITY_HINT);
+ if (hintToken != null) {
+ annotation = parseExpressionAnnotation(hintToken);
+ }
op = new OperatorExpr();
op.addOperand(operand);
op.setCurrentop(true);
@@ -3653,6 +3857,9 @@
} catch (CompilationException e){
throw new SqlppParseException(getSourceLocation(token), e.getMessage());
}
+ if (annotation != null) {
+ op.addHint(annotation);
+ }
}
operand = ConcatExpr()
@@ -4797,7 +5004,16 @@
List<AbstractBinaryCorrelateClause> correlateClauses = new ArrayList<AbstractBinaryCorrelateClause>();
}
{
- leftExpr = Expression() ((<AS>)? leftVar = Variable())? (<AT> posVar = Variable())?
+ leftExpr = Expression()
+ {
+ if (leftExpr.getKind() == Expression.Kind.VARIABLE_EXPRESSION) {
+ Token hintToken = fetchHint(token, SqlppHint.SUBPATH_HINT);
+ if (hintToken != null) {
+ String subPath = hintToken.hintParams;
+ ((VariableExpr) leftExpr).addHint(new ExternalSubpathAnnotation(subPath));
+ }
+ }
+ } ((<AS>)? leftVar = Variable())? (<AT> posVar = Variable())?
(
(
correlateClause = JoinOrUnnestClause(JoinType.INNER, UnnestType.INNER)
@@ -4868,7 +5084,16 @@
VariableExpr posVar = null;
}
{
- rightExpr = Expression() ((<AS>)? rightVar = Variable())? (<AT> posVar = Variable())?
+ rightExpr = Expression()
+ {
+ if (rightExpr.getKind() == Expression.Kind.VARIABLE_EXPRESSION) {
+ Token hintToken = fetchHint(token, SqlppHint.SUBPATH_HINT);
+ if (hintToken != null) {
+ String subPath = hintToken.hintParams;
+ ((VariableExpr) rightExpr).addHint(new ExternalSubpathAnnotation(subPath));
+ }
+ }
+ } ((<AS>)? rightVar = Variable())? (<AT> posVar = Variable())?
{
if (rightVar == null) {
rightVar = ExpressionToVariableUtil.getGeneratedVariable(rightExpr, true);
@@ -5304,11 +5529,12 @@
void CommonTokenAction(Token token) {
Token hintToken = token.specialToken;
- if (hintToken != null) {
+ while (hintToken != null) { // make this a while loop
hintToken.sourceLocation = new SourceLocation(hintToken.beginLine, hintToken.beginColumn);
String text = hintToken.image.substring(1).trim();
boolean hintFound = hintToken.parseHint(text);
hintCollector.put(hintToken.sourceLocation, hintFound ? hintToken.hint.getIdentifier() : hintToken.hintParams);
+ hintToken = hintToken.specialToken;
}
}
}
@@ -5318,6 +5544,7 @@
{
<ADAPTER: "adapter">
| <ALL : "all">
+ | <ANALYZE: "analyze">
| <AND : "and">
| <ANY : "any">
| <APPLY : "apply">
diff --git a/asterixdb/asterix-lang-sqlpp/src/test/java/org/apache/asterix/lang/expression/CommutativeEqualsTest.java b/asterixdb/asterix-lang-sqlpp/src/test/java/org/apache/asterix/lang/expression/CommutativeEqualsTest.java
new file mode 100644
index 0000000..aba83db
--- /dev/null
+++ b/asterixdb/asterix-lang-sqlpp/src/test/java/org/apache/asterix/lang/expression/CommutativeEqualsTest.java
@@ -0,0 +1,165 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.lang.expression;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.asterix.lang.common.util.FunctionUtil;
+import org.apache.asterix.om.base.AInt32;
+import org.apache.asterix.om.constants.AsterixConstantValue;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * TODO include multiply and add when supported
+ *
+ * @see FunctionUtil#commutativeEquals(ILogicalExpression, ILogicalExpression)
+ */
+public class CommutativeEqualsTest {
+ private final Map<Character, LogicalVariable> varNameToVarMap = new HashMap<>();
+ private int varCounter;
+
+ @Test
+ public void testExpressions() {
+ // EQ
+ reset();
+ ILogicalExpression expr1 = createExpression(BuiltinFunctions.EQ, 'x', 'y');
+ ILogicalExpression expr2 = createExpression(BuiltinFunctions.EQ, 'x', 'y');
+ Assert.assertTrue(FunctionUtil.commutativeEquals(expr1, expr2));
+
+ reset();
+ expr1 = createExpression(BuiltinFunctions.EQ, 'x', 'y');
+ expr2 = createExpression(BuiltinFunctions.EQ, 'y', 'x');
+ Assert.assertTrue(FunctionUtil.commutativeEquals(expr1, expr2));
+
+ reset();
+ expr1 = createExpression(BuiltinFunctions.EQ, 'x', 'x');
+ expr2 = createExpression(BuiltinFunctions.EQ, 'x', 'y');
+ Assert.assertFalse(FunctionUtil.commutativeEquals(expr1, expr2));
+
+ // ( ( NOT ( bool_field1 ) ) ) AND ( ( bool_field1 ) )
+ // ( ( bool_field1 ) ) AND ( ( bool_field1 = true ) )
+ reset();
+ expr1 = createExpression(BuiltinFunctions.AND,
+ createExpression(BuiltinFunctions.NOT, getVariableExpression('x')), getVariableExpression('x'));
+
+ expr2 = createExpression(BuiltinFunctions.AND, getVariableExpression('x'),
+ createExpression(BuiltinFunctions.EQ, getVariableExpression('x'), ConstantExpression.TRUE));
+ Assert.assertFalse(FunctionUtil.commutativeEquals(expr1, expr2));
+
+ // ( UPPER(varchar_field1) = varchar_field1 AND int_field1 = 2 )
+ // ( LOWER(varchar_field1) = varchar_field1 AND int_field1 = 2 )
+ reset();
+ expr1 = createExpression(BuiltinFunctions.AND,
+ createExpression(BuiltinFunctions.EQ,
+ createExpression(BuiltinFunctions.STRING_UPPERCASE, getVariableExpression('x')),
+ ConstantExpression.TRUE),
+ createExpression(BuiltinFunctions.EQ, getVariableExpression('y'),
+ new ConstantExpression(new AsterixConstantValue(new AInt32(2)))));
+
+ expr2 = createExpression(BuiltinFunctions.AND,
+ createExpression(BuiltinFunctions.EQ,
+ createExpression(BuiltinFunctions.STRING_LOWERCASE, getVariableExpression('x')),
+ ConstantExpression.TRUE),
+ createExpression(BuiltinFunctions.EQ, getVariableExpression('y'),
+ new ConstantExpression(new AsterixConstantValue(new AInt32(2)))));
+
+ Assert.assertFalse(FunctionUtil.commutativeEquals(expr1, expr2));
+
+ expr2 = createExpression(BuiltinFunctions.EQ,
+ createExpression(BuiltinFunctions.EQ,
+ createExpression(BuiltinFunctions.STRING_LOWERCASE, getVariableExpression('x')),
+ ConstantExpression.TRUE),
+ createExpression(BuiltinFunctions.EQ, getVariableExpression('y'),
+ new ConstantExpression(new AsterixConstantValue(new AInt32(2)))));
+
+ // ( LOWER(varchar_field1) = varchar_field1 AND int_field1 = 2 )
+ // ( int_field1 = 2 AND LOWER(varchar_field1) = varchar_field1)
+ // should evaluate to true
+ ILogicalExpression expr3 = createExpression(BuiltinFunctions.EQ,
+ createExpression(BuiltinFunctions.EQ, getVariableExpression('y'),
+ new ConstantExpression(new AsterixConstantValue(new AInt32(2)))),
+ createExpression(BuiltinFunctions.EQ,
+ createExpression(BuiltinFunctions.STRING_LOWERCASE, getVariableExpression('x')),
+ ConstantExpression.TRUE));
+
+ Assert.assertTrue(FunctionUtil.commutativeEquals(expr2, expr3));
+
+ // ( 10/int_field1=6911432 ) AND ( bool_field1=true )
+ // ( int_field1/10=6911432 ) AND ( bool_field1 = true )
+ reset();
+ expr1 = createExpression(BuiltinFunctions.AND,
+ createExpression(BuiltinFunctions.NUMERIC_DIV,
+ new ConstantExpression(new AsterixConstantValue(new AInt32(10))), getVariableExpression('i')),
+ createExpression(BuiltinFunctions.EQ, getVariableExpression('b'), ConstantExpression.TRUE));
+
+ expr2 = createExpression(BuiltinFunctions.AND,
+ createExpression(BuiltinFunctions.NUMERIC_DIV, getVariableExpression('i'),
+ new ConstantExpression(new AsterixConstantValue(new AInt32(10)))),
+ createExpression(BuiltinFunctions.EQ, getVariableExpression('b'), ConstantExpression.TRUE));
+
+ Assert.assertFalse(FunctionUtil.commutativeEquals(expr1, expr2));
+ }
+
+ private void reset() {
+ varCounter = 0;
+ varNameToVarMap.clear();
+ }
+
+ private ILogicalExpression createExpression(FunctionIdentifier fid, ILogicalExpression... left) {
+ List<Mutable<ILogicalExpression>> args = new ArrayList<>();
+
+ for (ILogicalExpression expr : left) {
+ args.add(new MutableObject<>(expr));
+ }
+
+ IFunctionInfo funcInfo = BuiltinFunctions.getBuiltinFunctionInfo(fid);
+ return new ScalarFunctionCallExpression(funcInfo, args);
+ }
+
+ private ILogicalExpression createExpression(FunctionIdentifier fid, char... left) {
+ List<Mutable<ILogicalExpression>> args = new ArrayList<>();
+
+ for (int i = 0; i < left.length; i++) {
+ args.add(new MutableObject<>(getVariableExpression(left[i])));
+ }
+
+ IFunctionInfo funcInfo = BuiltinFunctions.getBuiltinFunctionInfo(fid);
+ return new ScalarFunctionCallExpression(funcInfo, args);
+ }
+
+ private ILogicalExpression getVariableExpression(Character displayName) {
+ LogicalVariable variable = varNameToVarMap.computeIfAbsent(displayName,
+ k -> new LogicalVariable(varCounter++, displayName.toString()));
+ return new VariableReferenceExpression(variable);
+ }
+}
diff --git a/asterixdb/asterix-license/pom.xml b/asterixdb/asterix-license/pom.xml
index 3a6f1c3..f625c22 100644
--- a/asterixdb/asterix-license/pom.xml
+++ b/asterixdb/asterix-license/pom.xml
@@ -23,7 +23,7 @@
<parent>
<groupId>org.apache.asterix</groupId>
<artifactId>apache-asterixdb</artifactId>
- <version>0.9.8-SNAPSHOT</version>
+ <version>0.9.8.2-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/asterixdb/asterix-maven-plugins/asterix-grammar-extension-maven-plugin/pom.xml b/asterixdb/asterix-maven-plugins/asterix-grammar-extension-maven-plugin/pom.xml
index cd00397..0dff8a7 100644
--- a/asterixdb/asterix-maven-plugins/asterix-grammar-extension-maven-plugin/pom.xml
+++ b/asterixdb/asterix-maven-plugins/asterix-grammar-extension-maven-plugin/pom.xml
@@ -21,7 +21,7 @@
<parent>
<groupId>org.apache.asterix</groupId>
<artifactId>asterix-maven-plugins</artifactId>
- <version>0.9.8-SNAPSHOT</version>
+ <version>0.9.8.2-SNAPSHOT</version>
</parent>
<artifactId>asterix-grammar-extension-maven-plugin</artifactId>
<packaging>maven-plugin</packaging>
diff --git a/asterixdb/asterix-maven-plugins/asterix-test-datagenerator-maven-plugin/pom.xml b/asterixdb/asterix-maven-plugins/asterix-test-datagenerator-maven-plugin/pom.xml
index 88aaf2b..bc2e79f 100644
--- a/asterixdb/asterix-maven-plugins/asterix-test-datagenerator-maven-plugin/pom.xml
+++ b/asterixdb/asterix-maven-plugins/asterix-test-datagenerator-maven-plugin/pom.xml
@@ -26,7 +26,7 @@
<parent>
<artifactId>asterix-maven-plugins</artifactId>
<groupId>org.apache.asterix</groupId>
- <version>0.9.8-SNAPSHOT</version>
+ <version>0.9.8.2-SNAPSHOT</version>
</parent>
<properties>
diff --git a/asterixdb/asterix-maven-plugins/lexer-generator-maven-plugin/pom.xml b/asterixdb/asterix-maven-plugins/lexer-generator-maven-plugin/pom.xml
index dccb9d7..08be1b1 100644
--- a/asterixdb/asterix-maven-plugins/lexer-generator-maven-plugin/pom.xml
+++ b/asterixdb/asterix-maven-plugins/lexer-generator-maven-plugin/pom.xml
@@ -22,7 +22,7 @@
<parent>
<artifactId>asterix-maven-plugins</artifactId>
<groupId>org.apache.asterix</groupId>
- <version>0.9.8-SNAPSHOT</version>
+ <version>0.9.8.2-SNAPSHOT</version>
</parent>
<packaging>maven-plugin</packaging>
diff --git a/asterixdb/asterix-maven-plugins/pom.xml b/asterixdb/asterix-maven-plugins/pom.xml
index fd15ac5..318a45a 100644
--- a/asterixdb/asterix-maven-plugins/pom.xml
+++ b/asterixdb/asterix-maven-plugins/pom.xml
@@ -25,7 +25,7 @@
<parent>
<artifactId>apache-asterixdb</artifactId>
<groupId>org.apache.asterix</groupId>
- <version>0.9.8-SNAPSHOT</version>
+ <version>0.9.8.2-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/asterixdb/asterix-maven-plugins/record-manager-generator-maven-plugin/pom.xml b/asterixdb/asterix-maven-plugins/record-manager-generator-maven-plugin/pom.xml
index ab1b13a..67f6b3e 100644
--- a/asterixdb/asterix-maven-plugins/record-manager-generator-maven-plugin/pom.xml
+++ b/asterixdb/asterix-maven-plugins/record-manager-generator-maven-plugin/pom.xml
@@ -22,7 +22,7 @@
<parent>
<artifactId>asterix-maven-plugins</artifactId>
<groupId>org.apache.asterix</groupId>
- <version>0.9.8-SNAPSHOT</version>
+ <version>0.9.8.2-SNAPSHOT</version>
</parent>
<packaging>maven-plugin</packaging>
diff --git a/asterixdb/asterix-metadata/pom.xml b/asterixdb/asterix-metadata/pom.xml
index 61507d1..2ddf0a5 100644
--- a/asterixdb/asterix-metadata/pom.xml
+++ b/asterixdb/asterix-metadata/pom.xml
@@ -22,7 +22,7 @@
<parent>
<artifactId>apache-asterixdb</artifactId>
<groupId>org.apache.asterix</groupId>
- <version>0.9.8-SNAPSHOT</version>
+ <version>0.9.8.2-SNAPSHOT</version>
</parent>
<artifactId>asterix-metadata</artifactId>
<properties>
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/RMIServerFactory.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/RMIServerFactory.java
index 0128a87..6a0edc0 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/RMIServerFactory.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/RMIServerFactory.java
@@ -39,10 +39,7 @@
}
public static RMIServerSocketFactory getSocketFactory(INetworkSecurityManager securityManager) {
- if (securityManager.getConfiguration().isSslEnabled()) {
- return new RMIServerFactory(securityManager);
- }
- return null;
+ return new RMIServerFactory(securityManager);
}
@Override
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/AsterixStateProxy.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/AsterixStateProxy.java
index cedcccf..b4aeaf1 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/AsterixStateProxy.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/AsterixStateProxy.java
@@ -39,10 +39,11 @@
private static final Logger LOGGER = LogManager.getLogger();
private IMetadataNode metadataNode;
- private static final IAsterixStateProxy cc = new AsterixStateProxy();
+ private static IAsterixStateProxy cc;
public static IAsterixStateProxy registerRemoteObject(INetworkSecurityManager networkSecurityManager,
int metadataCallbackPort) throws RemoteException {
+ cc = new AsterixStateProxy();
IAsterixStateProxy stub = (IAsterixStateProxy) UnicastRemoteObject.exportObject(cc, metadataCallbackPort,
RMIClientFactory.getSocketFactory(networkSecurityManager),
RMIServerFactory.getSocketFactory(networkSecurityManager));
@@ -51,8 +52,10 @@
}
public static void unregisterRemoteObject() throws RemoteException {
- UnicastRemoteObject.unexportObject(cc, true);
- LOGGER.info("Asterix Distributed State Proxy Unbound");
+ if (cc != null) {
+ UnicastRemoteObject.unexportObject(cc, true);
+ LOGGER.info("Asterix Distributed State Proxy Unbound");
+ }
}
@Override
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
index fd0d30b..8880461 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
@@ -423,16 +423,16 @@
if (createMetadataDataset) {
final double bloomFilterFalsePositiveRate =
appContext.getStorageProperties().getBloomFilterFalsePositiveRate();
- LSMBTreeLocalResourceFactory lsmBtreeFactory =
- new LSMBTreeLocalResourceFactory(storageComponentProvider.getStorageManager(), typeTraits,
- cmpFactories, null, null, null, opTrackerFactory, ioOpCallbackFactory,
- pageWriteCallbackFactory, storageComponentProvider.getMetadataPageManagerFactory(),
- new AsterixVirtualBufferCacheProvider(datasetId),
- storageComponentProvider.getIoOperationSchedulerProvider(),
- appContext.getMetadataMergePolicyFactory(),
- StorageConstants.DEFAULT_COMPACTION_POLICY_PROPERTIES, true, bloomFilterKeyFields,
- bloomFilterFalsePositiveRate, true, null, NoOpCompressorDecompressorFactory.INSTANCE, true,
- TypeTraitProvider.INSTANCE.getTypeTrait(BuiltinType.ANULL), NullIntrospector.INSTANCE);
+ LSMBTreeLocalResourceFactory lsmBtreeFactory = new LSMBTreeLocalResourceFactory(
+ storageComponentProvider.getStorageManager(), typeTraits, cmpFactories, null, null, null,
+ opTrackerFactory, ioOpCallbackFactory, pageWriteCallbackFactory,
+ storageComponentProvider.getMetadataPageManagerFactory(),
+ new AsterixVirtualBufferCacheProvider(datasetId),
+ storageComponentProvider.getIoOperationSchedulerProvider(),
+ appContext.getMetadataMergePolicyFactory(), StorageConstants.DEFAULT_COMPACTION_POLICY_PROPERTIES,
+ true, bloomFilterKeyFields, bloomFilterFalsePositiveRate, true, null,
+ NoOpCompressorDecompressorFactory.INSTANCE, true,
+ TypeTraitProvider.INSTANCE.getTypeTrait(BuiltinType.ANULL), NullIntrospector.INSTANCE, false);
DatasetLocalResourceFactory dsLocalResourceFactory =
new DatasetLocalResourceFactory(datasetId, lsmBtreeFactory);
// TODO(amoudi) Creating the index should be done through the same code path as
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/ArrayBTreeResourceFactoryProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/ArrayBTreeResourceFactoryProvider.java
index c0bdc75..9a2821e 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/ArrayBTreeResourceFactoryProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/ArrayBTreeResourceFactoryProvider.java
@@ -100,7 +100,7 @@
pageWriteCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider,
mergePolicyFactory, mergePolicyProperties, true, null, bloomFilterFalsePositiveRate,
index.isPrimaryIndex(), btreeFields, compDecompFactory, false,
- typeTraitProvider.getTypeTrait(BuiltinType.ANULL), NullIntrospector.INSTANCE);
+ typeTraitProvider.getTypeTrait(BuiltinType.ANULL), NullIntrospector.INSTANCE, false);
default:
throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_DATASET_TYPE,
dataset.getDatasetType().toString());
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/BTreeResourceFactoryProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/BTreeResourceFactoryProvider.java
index 168b56e..d119067 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/BTreeResourceFactoryProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/BTreeResourceFactoryProvider.java
@@ -21,6 +21,7 @@
import java.util.List;
import java.util.Map;
+import org.apache.asterix.common.config.DatasetConfig;
import org.apache.asterix.common.config.DatasetConfig.DatasetType;
import org.apache.asterix.common.context.AsterixVirtualBufferCacheProvider;
import org.apache.asterix.common.context.IStorageComponentProvider;
@@ -92,13 +93,13 @@
filterTypeTraits, filterCmpFactories, filterFields, opTrackerFactory,
ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, true, bloomFilterFields,
- bloomFilterFalsePositiveRate, false, btreeFields, hasBloomFilter,
+ bloomFilterFalsePositiveRate, btreeFields, hasBloomFilter,
typeTraitProvider.getTypeTrait(BuiltinType.ANULL), NullIntrospector.INSTANCE)
: new ExternalBTreeWithBuddyLocalResourceFactory(storageManager, typeTraits, cmpFactories,
filterTypeTraits, filterCmpFactories, filterFields, opTrackerFactory,
ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, true, bloomFilterFields,
- bloomFilterFalsePositiveRate, false, btreeFields, hasBloomFilter,
+ bloomFilterFalsePositiveRate, btreeFields, hasBloomFilter,
typeTraitProvider.getTypeTrait(BuiltinType.ANULL), NullIntrospector.INSTANCE);
case INTERNAL:
AsterixVirtualBufferCacheProvider vbcProvider =
@@ -112,12 +113,15 @@
compDecompFactory = NoOpCompressorDecompressorFactory.INSTANCE;
}
+ boolean isSecondaryNoIncrementalMaintenance = index.getIndexType() == DatasetConfig.IndexType.SAMPLE;
+
return new LSMBTreeLocalResourceFactory(storageManager, typeTraits, cmpFactories, filterTypeTraits,
filterCmpFactories, filterFields, opTrackerFactory, ioOpCallbackFactory,
pageWriteCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider,
mergePolicyFactory, mergePolicyProperties, true, bloomFilterFields,
bloomFilterFalsePositiveRate, index.isPrimaryIndex(), btreeFields, compDecompFactory,
- hasBloomFilter, typeTraitProvider.getTypeTrait(BuiltinType.ANULL), NullIntrospector.INSTANCE);
+ hasBloomFilter, typeTraitProvider.getTypeTrait(BuiltinType.ANULL), NullIntrospector.INSTANCE,
+ isSecondaryNoIncrementalMaintenance);
default:
throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_DATASET_TYPE,
dataset.getDatasetType().toString());
@@ -127,7 +131,7 @@
private static ITypeTraits[] getTypeTraits(MetadataProvider metadataProvider, Dataset dataset, Index index,
ARecordType recordType, ARecordType metaType) throws AlgebricksException {
ITypeTraits[] primaryTypeTraits = dataset.getPrimaryTypeTraits(metadataProvider, recordType, metaType);
- if (index.isPrimaryIndex()) {
+ if (index.isPrimaryIndex() || index.getIndexType() == DatasetConfig.IndexType.SAMPLE) {
return primaryTypeTraits;
} else if (dataset.getDatasetType() == DatasetType.EXTERNAL
&& index.getIndexName().equals(IndexingConstants.getFilesIndexName(dataset.getDatasetName()))) {
@@ -162,7 +166,7 @@
Index index, ARecordType recordType, ARecordType metaType) throws AlgebricksException {
IBinaryComparatorFactory[] primaryCmpFactories =
dataset.getPrimaryComparatorFactories(metadataProvider, recordType, metaType);
- if (index.isPrimaryIndex()) {
+ if (index.isPrimaryIndex() || index.getIndexType() == DatasetConfig.IndexType.SAMPLE) {
return dataset.getPrimaryComparatorFactories(metadataProvider, recordType, metaType);
} else if (dataset.getDatasetType() == DatasetType.EXTERNAL
&& index.getIndexName().equals(IndexingConstants.getFilesIndexName(dataset.getDatasetName()))) {
@@ -200,7 +204,8 @@
if (index.isPrimaryIndex() || index.isPrimaryKeyIndex()) {
return dataset.getPrimaryBloomFilterFields();
}
- if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+ if (dataset.getDatasetType() == DatasetType.EXTERNAL
+ && index.getIndexType() != DatasetConfig.IndexType.SAMPLE) {
if (index.getIndexName().equals(IndexingConstants.getFilesIndexName(dataset.getDatasetName()))) {
return FilesIndexDescription.BLOOM_FILTER_FIELDS;
} else {
@@ -224,6 +229,8 @@
bloomFilterKeyFields[i] = i;
}
return bloomFilterKeyFields;
+ case SAMPLE:
+ return null;
default:
throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE,
String.valueOf(index.getIndexType()));
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DataSource.java
index 454d3da..36273d2 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DataSource.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DataSource.java
@@ -24,6 +24,7 @@
import java.util.HashMap;
import java.util.List;
import java.util.Map;
+import java.util.Objects;
import org.apache.asterix.om.types.IAType;
import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
@@ -60,6 +61,7 @@
public static final byte FEED = 0x02;
public static final byte LOADABLE = 0x03;
public static final byte FUNCTION = 0x04;
+ public static final byte SAMPLE = 0x05;
// Hide implicit public constructor
private Type() {
@@ -165,4 +167,27 @@
ITupleFilterFactory tupleFilterFactory, long outputLimit, IOperatorSchema opSchema,
IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig,
IProjectionInfo<?> projectionInfo) throws AlgebricksException;
+
+ @Override
+ public boolean sameAs(IDataSource<?> other) {
+ if (this == other) {
+ return true;
+ }
+
+ if (!(other instanceof DataSource)) {
+ return false;
+ }
+
+ DataSource that = (DataSource) other;
+ if (!Objects.equals(this.id, that.getId()) || !Objects.equals(this.datasourceType, that.getDatasourceType())) {
+ return false;
+ }
+
+ if (this.datasourceType == Type.EXTERNAL_DATASET && that.getDatasourceType() == Type.EXTERNAL_DATASET
+ && !Objects.equals(this.getProperties(), other.getProperties())) {
+ return false;
+ }
+
+ return true;
+ }
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DataSourcePartitioningProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DataSourcePartitioningProvider.java
index 21b744f..324c660 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DataSourcePartitioningProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DataSourcePartitioningProvider.java
@@ -61,6 +61,7 @@
pp = getFeedPartitioningProperty(ds, domain, scanVariables);
break;
case DataSource.Type.INTERNAL_DATASET:
+ case DataSource.Type.SAMPLE:
Set<LogicalVariable> pvars = new ListSet<>();
pp = getInternalDatasetPartitioningProperty(ds, domain, scanVariables, pvars);
propsLocal.add(new LocalOrderProperty(getOrderColumns(pvars)));
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
index 1d82b19..6b42ff2 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
@@ -19,8 +19,10 @@
package org.apache.asterix.metadata.declared;
import static org.apache.asterix.external.util.ExternalDataConstants.KEY_EXTERNAL_SCAN_BUFFER_SIZE;
+import static org.apache.asterix.external.util.ExternalDataConstants.SUBPATH;
import java.io.IOException;
+import java.io.Serializable;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@@ -78,13 +80,18 @@
private void initInternalDataset(IAType itemType, IAType metaItemType, IDatasetDetails datasetDetails)
throws AlgebricksException {
- InternalDatasetDetails internalDatasetDetails = (InternalDatasetDetails) datasetDetails;
+ schemaTypes =
+ createSchemaTypesForInternalDataset(itemType, metaItemType, (InternalDatasetDetails) datasetDetails);
+ }
+
+ static IAType[] createSchemaTypesForInternalDataset(IAType itemType, IAType metaItemType,
+ InternalDatasetDetails internalDatasetDetails) throws AlgebricksException {
ARecordType recordType = (ARecordType) itemType;
ARecordType metaRecordType = (ARecordType) metaItemType;
List<IAType> partitioningKeyTypes =
KeyFieldTypeUtil.getPartitioningKeyTypes(internalDatasetDetails, recordType, metaRecordType);
int n = partitioningKeyTypes.size();
- schemaTypes = metaItemType == null ? new IAType[n + 1] : new IAType[n + 2];
+ IAType[] schemaTypes = metaItemType == null ? new IAType[n + 1] : new IAType[n + 2];
for (int keyIndex = 0; keyIndex < n; ++keyIndex) {
schemaTypes[keyIndex] = partitioningKeyTypes.get(keyIndex);
}
@@ -92,11 +99,17 @@
if (metaItemType != null) {
schemaTypes[n + 1] = metaItemType;
}
+ return schemaTypes;
}
private void initExternalDataset(IAType itemType) {
- schemaTypes = new IAType[1];
+ schemaTypes = createSchemaTypesForExternalDataset(itemType);
+ }
+
+ static IAType[] createSchemaTypesForExternalDataset(IAType itemType) {
+ IAType[] schemaTypes = new IAType[1];
schemaTypes[0] = itemType;
+ return schemaTypes;
}
@Override
@@ -109,7 +122,8 @@
IProjectionInfo<?> projectionInfo) throws AlgebricksException {
switch (dataset.getDatasetType()) {
case EXTERNAL:
- Dataset externalDataset = ((DatasetDataSource) dataSource).getDataset();
+ DatasetDataSource externalDataSource = (DatasetDataSource) dataSource;
+ Dataset externalDataset = externalDataSource.getDataset();
String itemTypeName = externalDataset.getItemTypeName();
IAType itemType = MetadataManager.INSTANCE.getDatatype(metadataProvider.getMetadataTxnContext(),
externalDataset.getItemTypeDataverseName(), itemTypeName).getDatatype();
@@ -118,6 +132,7 @@
PhysicalOptimizationConfig physicalOptimizationConfig = context.getPhysicalOptimizationConfig();
int externalScanBufferSize = physicalOptimizationConfig.getExternalScanBufferSize();
Map<String, String> properties = addExternalProjectionInfo(projectionInfo, edd.getProperties());
+ properties = addSubPath(externalDataSource.getProperties(), properties);
properties.put(KEY_EXTERNAL_SCAN_BUFFER_SIZE, String.valueOf(externalScanBufferSize));
ITypedAdapterFactory adapterFactory = metadataProvider.getConfiguredAdapterFactory(externalDataset,
edd.getAdapter(), properties, (ARecordType) itemType, null, context.getWarningCollector());
@@ -157,6 +172,16 @@
return propertiesCopy;
}
+ private Map<String, String> addSubPath(Map<String, Serializable> dataSourceProps, Map<String, String> properties) {
+ Serializable subPath = dataSourceProps.get(SUBPATH);
+ if (!(subPath instanceof String)) {
+ return properties;
+ }
+ Map<String, String> propertiesCopy = new HashMap<>(properties);
+ propertiesCopy.put(SUBPATH, (String) subPath);
+ return propertiesCopy;
+ }
+
private int[] createFilterIndexes(List<LogicalVariable> filterVars, IOperatorSchema opSchema) {
if (filterVars != null && !filterVars.isEmpty()) {
final int size = filterVars.size();
@@ -173,5 +198,4 @@
public boolean isScanAccessPathALeaf() {
return dataset.getDatasetType() == DatasetType.EXTERNAL;
}
-
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FunctionDataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FunctionDataSource.java
index f5fd7dd..bce22c1 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FunctionDataSource.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FunctionDataSource.java
@@ -20,13 +20,13 @@
import java.util.Arrays;
import java.util.Collections;
-import java.util.HashSet;
import java.util.List;
-import java.util.Set;
import org.apache.asterix.common.cluster.IClusterStateManager;
import org.apache.asterix.common.functions.FunctionSignature;
import org.apache.asterix.external.adapter.factory.GenericAdapterFactory;
+import org.apache.asterix.external.api.IDataParserFactory;
+import org.apache.asterix.external.parser.factory.ADMDataParserFactory;
import org.apache.asterix.metadata.api.IDatasourceFunction;
import org.apache.asterix.om.types.IAType;
import org.apache.asterix.om.utils.RecordUtil;
@@ -53,10 +53,19 @@
protected final FunctionIdentifier functionId;
+ public FunctionDataSource(DataSourceId id, FunctionIdentifier functionId, INodeDomain domain, IAType itemType)
+ throws AlgebricksException {
+ super(id, itemType, null, DataSource.Type.FUNCTION, domain);
+ this.functionId = functionId;
+ initSchemaType(itemType);
+ }
+
public FunctionDataSource(DataSourceId id, FunctionIdentifier functionId, INodeDomain domain)
throws AlgebricksException {
- super(id, RecordUtil.FULLY_OPEN_RECORD_TYPE, null, DataSource.Type.FUNCTION, domain);
- this.functionId = functionId;
+ this(id, functionId, domain, RecordUtil.FULLY_OPEN_RECORD_TYPE);
+ }
+
+ protected void initSchemaType(IAType itemType) {
schemaTypes = new IAType[] { itemType };
}
@@ -89,21 +98,48 @@
IClusterStateManager csm = metadataProvider.getApplicationContext().getClusterStateManager();
FunctionDataSourceFactory factory =
new FunctionDataSourceFactory(createFunction(metadataProvider, getLocations(csm)));
- adapterFactory.configure(factory);
+ IDataParserFactory dataParserFactory = createDataParserFactory();
+ dataParserFactory.setRecordType(RecordUtil.FULLY_OPEN_RECORD_TYPE);
+ dataParserFactory.configure(Collections.emptyMap());
+ adapterFactory.configure(factory, dataParserFactory);
return metadataProvider.buildExternalDatasetDataScannerRuntime(jobSpec, itemType, adapterFactory,
tupleFilterFactory, outputLimit);
}
+ public boolean skipJobCapacityAssignment() {
+ return false;
+ }
+
protected abstract IDatasourceFunction createFunction(MetadataProvider metadataProvider,
AlgebricksAbsolutePartitionConstraint locations);
protected AlgebricksAbsolutePartitionConstraint getLocations(IClusterStateManager csm) {
- String[] allPartitions = csm.getClusterLocations().getLocations();
- Set<String> ncs = new HashSet<>(Arrays.asList(allPartitions));
- return new AlgebricksAbsolutePartitionConstraint(ncs.toArray(new String[ncs.size()]));
+ String[] sortedLocations = csm.getSortedClusterLocations().getLocations();
+ return new AlgebricksAbsolutePartitionConstraint(
+ Arrays.stream(sortedLocations).distinct().toArray(String[]::new));
+ }
+
+ protected IDataParserFactory createDataParserFactory() {
+ return new ADMDataParserFactory();
}
protected static DataSourceId createDataSourceId(FunctionIdentifier fid, String... parameters) {
return new DataSourceId(FunctionSignature.getDataverseName(fid), fid.getName(), parameters);
}
+
+ protected abstract boolean sameFunctionDatasource(FunctionDataSource other);
+
+ @Override
+ public boolean sameAs(IDataSource<?> other) {
+ if (!super.sameAs(other)) {
+ return false;
+ }
+
+ if (!(other instanceof FunctionDataSource)) {
+ return false;
+ }
+
+ FunctionDataSource that = (FunctionDataSource) other;
+ return sameFunctionDatasource(that);
+ }
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
index 4cc83ab..46ba9dd 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
@@ -26,11 +26,13 @@
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
+import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Optional;
+import java.util.stream.Collectors;
import org.apache.asterix.common.cluster.IClusterStateManager;
import org.apache.asterix.common.config.DatasetConfig.DatasetType;
@@ -93,7 +95,9 @@
import org.apache.asterix.metadata.lock.ExternalDatasetsRegistry;
import org.apache.asterix.metadata.utils.DatasetUtil;
import org.apache.asterix.metadata.utils.FullTextUtil;
+import org.apache.asterix.metadata.utils.IndexUtil;
import org.apache.asterix.metadata.utils.MetadataConstants;
+import org.apache.asterix.metadata.utils.MetadataUtil;
import org.apache.asterix.metadata.utils.SplitsAndConstraintsUtil;
import org.apache.asterix.om.functions.BuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionExtensionManager;
@@ -115,6 +119,7 @@
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.common.utils.Pair;
import org.apache.hyracks.algebricks.common.utils.Triple;
+import org.apache.hyracks.algebricks.core.algebra.base.Counter;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionRuntimeProvider;
@@ -182,12 +187,11 @@
private Dataverse defaultDataverse;
private MetadataTransactionContext mdTxnCtx;
private boolean isWriteTransaction;
- private IAWriterFactory writerFactory;
private FileSplit outputFile;
private boolean asyncResults;
private long maxResultReads;
private ResultSetId resultSetId;
- private IResultSerializerFactoryProvider resultSerializerFactoryProvider;
+ private Counter resultSetIdCounter;
private TxnId txnId;
private Map<String, Integer> externalDataLocks;
private boolean blockingOperatorDisabled = false;
@@ -262,10 +266,6 @@
this.isWriteTransaction = writeTransaction;
}
- public void setWriterFactory(IAWriterFactory writerFactory) {
- this.writerFactory = writerFactory;
- }
-
public void setMetadataTxnContext(MetadataTransactionContext mdTxnCtx) {
this.mdTxnCtx = mdTxnCtx;
}
@@ -274,10 +274,6 @@
return mdTxnCtx;
}
- public IAWriterFactory getWriterFactory() {
- return this.writerFactory;
- }
-
public FileSplit getOutputFile() {
return outputFile;
}
@@ -310,12 +306,12 @@
this.resultSetId = resultSetId;
}
- public void setResultSerializerFactoryProvider(IResultSerializerFactoryProvider rafp) {
- this.resultSerializerFactoryProvider = rafp;
+ public Counter getResultSetIdCounter() {
+ return resultSetIdCounter;
}
- public IResultSerializerFactoryProvider getResultSerializerFactoryProvider() {
- return resultSerializerFactoryProvider;
+ public void setResultSetIdCounter(Counter resultSetIdCounter) {
+ this.resultSetIdCounter = resultSetIdCounter;
}
public boolean isWriteTransaction() {
@@ -437,6 +433,16 @@
return MetadataManagerUtil.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
}
+ public Index findSampleIndex(DataverseName dataverseName, String datasetName) throws AlgebricksException {
+ Pair<String, String> sampleIndexNames = IndexUtil.getSampleIndexNames(datasetName);
+ Index sampleIndex = getIndex(dataverseName, datasetName, sampleIndexNames.first);
+ if (sampleIndex != null && sampleIndex.getPendingOp() == MetadataUtil.PENDING_NO_OP) {
+ return sampleIndex;
+ }
+ sampleIndex = getIndex(dataverseName, datasetName, sampleIndexNames.second);
+ return sampleIndex != null && sampleIndex.getPendingOp() == MetadataUtil.PENDING_NO_OP ? sampleIndex : null;
+ }
+
public Triple<DataverseName, String, Boolean> resolveDatasetNameUsingSynonyms(DataverseName dataverseName,
String datasetName, boolean includingViews) throws AlgebricksException {
DataverseName dvName = getActiveDataverseName(dataverseName);
@@ -561,6 +567,12 @@
case BTREE:
numSecondaryKeys = ((Index.ValueIndexDetails) theIndex.getIndexDetails()).getKeyFieldNames().size();
break;
+ case SAMPLE:
+ if (isIndexOnlyPlan) {
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, "");
+ }
+ numSecondaryKeys = 0;
+ break;
default:
throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE,
theIndex.getIndexType().toString());
@@ -683,7 +695,8 @@
@Override
public Pair<IPushRuntimeFactory, AlgebricksPartitionConstraint> getWriteFileRuntime(IDataSink sink,
- int[] printColumns, IPrinterFactory[] printerFactories, RecordDescriptor inputDesc) {
+ int[] printColumns, IPrinterFactory[] printerFactories, IAWriterFactory writerFactory,
+ RecordDescriptor inputDesc) {
FileSplitDataSink fsds = (FileSplitDataSink) sink;
FileSplitSinkId fssi = fsds.getId();
FileSplit fs = fssi.getFileSplit();
@@ -691,14 +704,15 @@
String nodeId = fs.getNodeName();
SinkWriterRuntimeFactory runtime =
- new SinkWriterRuntimeFactory(printColumns, printerFactories, outFile, getWriterFactory(), inputDesc);
+ new SinkWriterRuntimeFactory(printColumns, printerFactories, outFile, writerFactory, inputDesc);
AlgebricksPartitionConstraint apc = new AlgebricksAbsolutePartitionConstraint(new String[] { nodeId });
return new Pair<>(runtime, apc);
}
@Override
public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getResultHandleRuntime(IDataSink sink,
- int[] printColumns, IPrinterFactory[] printerFactories, RecordDescriptor inputDesc,
+ int[] printColumns, IPrinterFactory[] printerFactories, IAWriterFactory writerFactory,
+ IResultSerializerFactoryProvider resultSerializerFactoryProvider, RecordDescriptor inputDesc,
IResultMetadata metadata, JobSpecification spec) throws AlgebricksException {
ResultSetDataSink rsds = (ResultSetDataSink) sink;
ResultSetSinkId rssId = rsds.getId();
@@ -706,7 +720,7 @@
ResultWriterOperatorDescriptor resultWriter = null;
try {
IResultSerializerFactory resultSerializedAppenderFactory = resultSerializerFactoryProvider
- .getAqlResultSerializerFactoryProvider(printColumns, printerFactories, getWriterFactory());
+ .getResultSerializerFactoryProvider(printColumns, printerFactories, writerFactory);
resultWriter = new ResultWriterOperatorDescriptor(spec, rsId, metadata, getResultAsyncMode(),
resultSerializedAppenderFactory, getMaxResultReads());
} catch (IOException e) {
@@ -946,7 +960,7 @@
}
public AlgebricksAbsolutePartitionConstraint getClusterLocations() {
- return appCtx.getClusterStateManager().getClusterLocations();
+ return appCtx.getClusterStateManager().getSortedClusterLocations();
}
public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildExternalDataLookupRuntime(
@@ -1818,6 +1832,24 @@
return StoragePathUtil.splitProviderAndPartitionConstraints(splits);
}
+ public List<Pair<IFileSplitProvider, String>> getSplitProviderOfAllIndexes(Dataset ds) throws AlgebricksException {
+ List<Index> dsIndexes = getDatasetIndexes(ds.getDataverseName(), ds.getDatasetName()).stream()
+ .filter(idx -> idx.getIndexType() != IndexType.SAMPLE && idx.isSecondaryIndex())
+ .collect(Collectors.toList());
+ if (dsIndexes.isEmpty()) {
+ return Collections.emptyList();
+ }
+ List<String> datasetNodes = findNodes(ds.getNodeGroupName());
+ List<Pair<IFileSplitProvider, String>> indexesSplits =
+ dsIndexes.stream()
+ .map(idx -> new Pair<>(
+ StoragePathUtil.splitProvider(SplitsAndConstraintsUtil.getIndexSplits(
+ appCtx.getClusterStateManager(), ds, idx.getIndexName(), datasetNodes)),
+ idx.getIndexName()))
+ .collect(Collectors.toList());
+ return indexesSplits;
+ }
+
public LockList getLocks() {
return locks;
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/SampleDataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/SampleDataSource.java
new file mode 100644
index 0000000..885723c
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/SampleDataSource.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.metadata.declared;
+
+import java.util.List;
+
+import org.apache.asterix.metadata.entities.Dataset;
+import org.apache.asterix.metadata.entities.InternalDatasetDetails;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSource;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IProjectionInfo;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
+import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
+import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
+
+public class SampleDataSource extends DataSource {
+
+ private final Dataset dataset;
+
+ private final String sampleIndexName;
+
+ public SampleDataSource(Dataset dataset, String sampleIndexName, IAType itemType, IAType metaItemType,
+ INodeDomain domain) throws AlgebricksException {
+ super(createSampleDataSourceId(dataset, sampleIndexName), itemType, metaItemType, Type.SAMPLE, domain);
+ this.dataset = dataset;
+ this.sampleIndexName = sampleIndexName;
+ this.schemaTypes = DatasetDataSource.createSchemaTypesForInternalDataset(itemType, metaItemType,
+ (InternalDatasetDetails) dataset.getDatasetDetails());
+ }
+
+ @Override
+ public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildDatasourceScanRuntime(
+ MetadataProvider metadataProvider, IDataSource<DataSourceId> dataSource,
+ List<LogicalVariable> scanVariables, List<LogicalVariable> projectVariables, boolean projectPushed,
+ List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars,
+ ITupleFilterFactory tupleFilterFactory, long outputLimit, IOperatorSchema opSchema,
+ IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig,
+ IProjectionInfo<?> projectionInfo) throws AlgebricksException {
+ return metadataProvider.buildBtreeRuntime(jobSpec, opSchema, typeEnv, context, true, false, null, dataset,
+ sampleIndexName, null, null, true, true, false, null, null, null, tupleFilterFactory, outputLimit,
+ false, false);
+ }
+
+ @Override
+ public boolean isScanAccessPathALeaf() {
+ return false;
+ }
+
+ private static DataSourceId createSampleDataSourceId(Dataset dataset, String sampleIndexName) {
+ return new DataSourceId(dataset.getDataverseName(), dataset.getDatasetName(), new String[] { sampleIndexName });
+ }
+}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
index 37e22ed..e3935a4 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
@@ -497,6 +497,10 @@
recordType, metaType, mergePolicyFactory, mergePolicyProperties, filterTypeTraits,
filterCmpFactories);
break;
+ case SAMPLE:
+ resourceFactory = BTreeResourceFactoryProvider.INSTANCE.getResourceFactory(mdProvider, this, index,
+ recordType, metaType, mergePolicyFactory, mergePolicyProperties, null, null);
+ break;
default:
throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE,
index.getIndexType().toString());
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Index.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Index.java
index 9e67292..6b12bf8 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Index.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Index.java
@@ -22,6 +22,7 @@
import java.io.Serializable;
import java.util.Collections;
import java.util.List;
+import java.util.Map;
import java.util.Objects;
import org.apache.asterix.common.config.DatasetConfig.IndexType;
@@ -39,6 +40,7 @@
import org.apache.asterix.om.utils.NonTaggedFormatUtil;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.api.job.profiling.IndexStats;
import org.apache.hyracks.util.OptionalBoolean;
/**
@@ -136,6 +138,10 @@
this.pendingOp = pendingOp;
}
+ public boolean isSampleIndex() {
+ return indexType == IndexType.SAMPLE;
+ }
+
public boolean isSecondaryIndex() {
return !isPrimaryIndex();
}
@@ -275,6 +281,7 @@
switch (indexType) {
case ARRAY:
case BTREE:
+ case SAMPLE:
return ResourceType.LSM_BTREE;
case RTREE:
return ResourceType.LSM_RTREE;
@@ -296,7 +303,8 @@
public enum IndexCategory {
VALUE,
TEXT,
- ARRAY;
+ ARRAY,
+ SAMPLE;
public static IndexCategory of(IndexType indexType) {
switch (indexType) {
@@ -310,6 +318,8 @@
return TEXT;
case ARRAY:
return ARRAY;
+ case SAMPLE:
+ return SAMPLE;
default:
throw new IllegalArgumentException(String.valueOf(indexType));
}
@@ -533,6 +543,81 @@
}
}
+ public static class SampleIndexDetails extends AbstractIndexDetails {
+
+ private static final long serialVersionUID = 1L;
+
+ private final List<List<String>> keyFieldNames;
+
+ private final List<Integer> keyFieldSourceIndicators;
+
+ private final List<IAType> keyFieldTypes;
+
+ private final int sampleCardinalityTarget;
+
+ private final long sourceCardinality;
+
+ private final int sourceAvgItemSize;
+
+ private final long sampleSeed;
+ private final Map<String, IndexStats> indexesStats;
+
+ public SampleIndexDetails(List<List<String>> keyFieldNames, List<Integer> keyFieldSourceIndicators,
+ List<IAType> keyFieldTypes, int sampleCardinalityTarget, long sourceCardinality, int sourceAvgItemSize,
+ long sampleSeed, Map<String, IndexStats> indexesStats) {
+ this.keyFieldNames = keyFieldNames;
+ this.keyFieldSourceIndicators = keyFieldSourceIndicators;
+ this.keyFieldTypes = keyFieldTypes;
+ this.sampleCardinalityTarget = sampleCardinalityTarget;
+ this.sourceCardinality = sourceCardinality;
+ this.sourceAvgItemSize = sourceAvgItemSize;
+ this.sampleSeed = sampleSeed;
+ this.indexesStats = indexesStats;
+ }
+
+ @Override
+ IndexCategory getIndexCategory() {
+ return IndexCategory.SAMPLE;
+ }
+
+ public List<List<String>> getKeyFieldNames() {
+ return keyFieldNames;
+ }
+
+ public List<Integer> getKeyFieldSourceIndicators() {
+ return keyFieldSourceIndicators;
+ }
+
+ public List<IAType> getKeyFieldTypes() {
+ return keyFieldTypes;
+ }
+
+ @Override
+ public boolean isOverridingKeyFieldTypes() {
+ return false;
+ }
+
+ public int getSampleCardinalityTarget() {
+ return sampleCardinalityTarget;
+ }
+
+ public long getSourceCardinality() {
+ return sourceCardinality;
+ }
+
+ public int getSourceAvgItemSize() {
+ return sourceAvgItemSize;
+ }
+
+ public long getSampleSeed() {
+ return sampleSeed;
+ }
+
+ public Map<String, IndexStats> getIndexesStats() {
+ return indexesStats;
+ }
+ }
+
@Deprecated
private static Index.IIndexDetails createSimpleIndexDetails(IndexType indexType, List<List<String>> keyFieldNames,
List<Integer> keyFieldSourceIndicators, List<IAType> keyFieldTypes, boolean overrideKeyFieldTypes,
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/NodeGroup.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/NodeGroup.java
index a8d027f..a50fd4a 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/NodeGroup.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/NodeGroup.java
@@ -20,6 +20,7 @@
package org.apache.asterix.metadata.entities;
import java.util.ArrayList;
+import java.util.Collection;
import java.util.Collections;
import java.util.List;
@@ -42,7 +43,7 @@
this.nodeNames = nodeNames;
}
- public static NodeGroup createOrdered(String groupName, List<String> nodeNames) {
+ public static NodeGroup createOrdered(String groupName, Collection<String> nodeNames) {
List<String> sortedNodeNames = new ArrayList<>(nodeNames);
Collections.sort(sortedNodeNames);
return new NodeGroup(groupName, sortedNodeNames);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
index bb51748..b00a706 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
@@ -25,7 +25,9 @@
import java.util.ArrayList;
import java.util.Calendar;
import java.util.Collections;
+import java.util.HashMap;
import java.util.List;
+import java.util.Map;
import java.util.stream.Collectors;
import org.apache.asterix.builders.IARecordBuilder;
@@ -47,7 +49,9 @@
import org.apache.asterix.om.base.ABoolean;
import org.apache.asterix.om.base.ACollectionCursor;
import org.apache.asterix.om.base.AInt32;
+import org.apache.asterix.om.base.AInt64;
import org.apache.asterix.om.base.AInt8;
+import org.apache.asterix.om.base.AMutableInt64;
import org.apache.asterix.om.base.AMutableInt8;
import org.apache.asterix.om.base.ANull;
import org.apache.asterix.om.base.AOrderedList;
@@ -66,6 +70,7 @@
import org.apache.hyracks.algebricks.common.utils.Triple;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.job.profiling.IndexStats;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
import org.apache.hyracks.util.OptionalBoolean;
@@ -90,6 +95,13 @@
public static final String INDEX_SEARCHKEY_ELEMENTS_FIELD_NAME = "SearchKeyElements";
public static final String COMPLEXSEARCHKEY_UNNEST_FIELD_NAME = "UnnestList";
public static final String COMPLEXSEARCHKEY_PROJECT_FIELD_NAME = "ProjectList";
+ public static final String SAMPLE_SEED = "SampleSeed";
+ public static final String SAMPLE_CARDINALITY_TARGET = "SampleCardinalityTarget";
+ public static final String SOURCE_CARDINALITY = "SourceCardinality";
+ public static final String SOURCE_AVG_ITEM_SIZE = "SourceAvgItemSize";
+ public static final String INDEXES_STATS = "IndexStats";
+ public static final String STATS_NUM_PAGES = "NumPages";
+ public static final String STATS_INDEX_NAME = "IndexName";
protected final TxnId txnId;
protected final MetadataNode metadataNode;
@@ -100,13 +112,15 @@
protected OrderedListBuilder complexSearchKeyNameListBuilder;
protected IARecordBuilder complexSearchKeyNameRecordBuilder;
protected IARecordBuilder castRecordBuilder;
+ protected OrderedListBuilder indexesStatsListBuilder;
+ protected IARecordBuilder indexStatsRecordBuilder;
protected AOrderedListType stringList;
protected AOrderedListType int8List;
protected ArrayBackedValueStorage nameValue;
protected ArrayBackedValueStorage itemValue;
protected AMutableInt8 aInt8;
+ protected AMutableInt64 aInt64;
protected ISerializerDeserializer<AInt8> int8Serde;
- protected ISerializerDeserializer<ANull> nullSerde;
@SuppressWarnings("unchecked")
protected IndexTupleTranslator(TxnId txnId, MetadataNode metadataNode, boolean getTuple) {
@@ -119,14 +133,16 @@
primaryKeyListBuilder = new OrderedListBuilder();
complexSearchKeyNameRecordBuilder = new RecordBuilder();
castRecordBuilder = new RecordBuilder();
+ indexesStatsListBuilder = new OrderedListBuilder();
+ indexStatsRecordBuilder = new RecordBuilder();
complexSearchKeyNameListBuilder = new OrderedListBuilder();
stringList = new AOrderedListType(BuiltinType.ASTRING, null);
int8List = new AOrderedListType(BuiltinType.AINT8, null);
nameValue = new ArrayBackedValueStorage();
itemValue = new ArrayBackedValueStorage();
aInt8 = new AMutableInt8((byte) 0);
+ aInt64 = new AMutableInt64(0);
int8Serde = SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT8);
- nullSerde = SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ANULL);
}
}
@@ -240,6 +256,9 @@
searchElements.add(searchElement);
}
break;
+ case SAMPLE:
+ searchElements = Collections.emptyList();
+ break;
default:
throw new AsterixException(ErrorCode.METADATA_ERROR, indexType.toString());
}
@@ -452,6 +471,57 @@
}
indexDetails = new Index.ArrayIndexDetails(elementList, isOverridingKeyTypes);
break;
+ case SAMPLE:
+ keyFieldNames =
+ searchElements.stream().map(Pair::getSecond).map(l -> l.get(0)).collect(Collectors.toList());
+ keyFieldTypes = searchKeyType.stream().map(l -> l.get(0)).collect(Collectors.toList());
+
+ int sampleSeedPos = indexRecord.getType().getFieldIndex(SAMPLE_SEED);
+ if (sampleSeedPos < 0) {
+ throw new AsterixException(ErrorCode.METADATA_ERROR, SAMPLE_SEED);
+ }
+ long sampleSeed = ((AInt64) indexRecord.getValueByPos(sampleSeedPos)).getLongValue();
+
+ int sampleCardinalityTargetPos = indexRecord.getType().getFieldIndex(SAMPLE_CARDINALITY_TARGET);
+ if (sampleCardinalityTargetPos < 0) {
+ throw new AsterixException(ErrorCode.METADATA_ERROR, SAMPLE_CARDINALITY_TARGET);
+ }
+ int sampleCardinalityTarget =
+ ((AInt32) indexRecord.getValueByPos(sampleCardinalityTargetPos)).getIntegerValue();
+
+ int sourceCardinalityPos = indexRecord.getType().getFieldIndex(SOURCE_CARDINALITY);
+ if (sourceCardinalityPos < 0) {
+ throw new AsterixException(ErrorCode.METADATA_ERROR, SOURCE_CARDINALITY);
+ }
+ long sourceCardinality = ((AInt64) indexRecord.getValueByPos(sourceCardinalityPos)).getLongValue();
+
+ int sourceAvgItemSizePos = indexRecord.getType().getFieldIndex(SOURCE_AVG_ITEM_SIZE);
+ if (sourceAvgItemSizePos < 0) {
+ throw new AsterixException(ErrorCode.METADATA_ERROR, SOURCE_AVG_ITEM_SIZE);
+ }
+ int sourceAvgItemSize = ((AInt32) indexRecord.getValueByPos(sourceAvgItemSizePos)).getIntegerValue();
+
+ int indexesStatsPos = indexRecord.getType().getFieldIndex(INDEXES_STATS);
+ Map<String, IndexStats> indexesStats;
+ if (indexesStatsPos >= 0) {
+ AOrderedList indexesStatsList = (AOrderedList) indexRecord.getValueByPos(indexesStatsPos);
+ int numIndexes = indexesStatsList.size();
+ indexesStats = numIndexes > 0 ? new HashMap<>() : Collections.emptyMap();
+ for (int i = 0; i < numIndexes; i++) {
+ ARecord stats = (ARecord) indexesStatsList.getItem(i);
+ IAObject numPages = stats.getValueByPos(stats.getType().getFieldIndex(STATS_NUM_PAGES));
+ IAObject idxNameObj = stats.getValueByPos(stats.getType().getFieldIndex(STATS_INDEX_NAME));
+ String idxName = ((AString) idxNameObj).getStringValue();
+ IndexStats idxStats = new IndexStats(idxName, ((AInt64) numPages).getLongValue());
+ indexesStats.put(idxName, idxStats);
+ }
+ } else {
+ indexesStats = Collections.emptyMap();
+ }
+
+ indexDetails = new Index.SampleIndexDetails(keyFieldNames, keyFieldSourceIndicator, keyFieldTypes,
+ sampleCardinalityTarget, sourceCardinality, sourceAvgItemSize, sampleSeed, indexesStats);
+ break;
default:
throw new AsterixException(ErrorCode.METADATA_ERROR, indexType.toString());
}
@@ -527,6 +597,9 @@
// If we have a complex index, we persist all of the names in the complex SK name array instead.
searchKey = Collections.emptyList();
break;
+ case SAMPLE:
+ searchKey = ((Index.SampleIndexDetails) index.getIndexDetails()).getKeyFieldNames();
+ break;
default:
throw new AsterixException(ErrorCode.METADATA_ERROR, indexType.toString());
}
@@ -596,6 +669,7 @@
writeSearchKeySourceIndicator(index);
writeExcludeUnknownKey(index);
writeCast(index);
+ writeSampleDetails(index);
}
private void writeComplexSearchKeys(Index.ArrayIndexDetails indexDetails) throws HyracksDataException {
@@ -767,6 +841,9 @@
keySourceIndicator = ((Index.ArrayIndexDetails) index.getIndexDetails()).getElementList().stream()
.map(Index.ArrayIndexElement::getSourceIndicator).collect(Collectors.toList());
break;
+ case SAMPLE:
+ keySourceIndicator = ((Index.SampleIndexDetails) index.getIndexDetails()).getKeyFieldSourceIndicators();
+ break;
default:
throw new AsterixException(ErrorCode.METADATA_ERROR, index.getIndexType().toString());
}
@@ -855,4 +932,73 @@
}
}
}
+
+ private void writeSampleDetails(Index index) throws HyracksDataException {
+ if (index.getIndexType() == IndexType.SAMPLE) {
+ Index.SampleIndexDetails indexDetails = (Index.SampleIndexDetails) index.getIndexDetails();
+
+ nameValue.reset();
+ fieldValue.reset();
+ aString.setValue(SAMPLE_SEED);
+ stringSerde.serialize(aString, nameValue.getDataOutput());
+ int64Serde.serialize(new AInt64(indexDetails.getSampleSeed()), fieldValue.getDataOutput());
+ recordBuilder.addField(nameValue, fieldValue);
+
+ nameValue.reset();
+ fieldValue.reset();
+ aString.setValue(SAMPLE_CARDINALITY_TARGET);
+ stringSerde.serialize(aString, nameValue.getDataOutput());
+ int32Serde.serialize(new AInt32(indexDetails.getSampleCardinalityTarget()), fieldValue.getDataOutput());
+ recordBuilder.addField(nameValue, fieldValue);
+
+ nameValue.reset();
+ fieldValue.reset();
+ aString.setValue(SOURCE_CARDINALITY);
+ stringSerde.serialize(aString, nameValue.getDataOutput());
+ int64Serde.serialize(new AInt64(indexDetails.getSourceCardinality()), fieldValue.getDataOutput());
+ recordBuilder.addField(nameValue, fieldValue);
+
+ nameValue.reset();
+ fieldValue.reset();
+ aString.setValue(SOURCE_AVG_ITEM_SIZE);
+ stringSerde.serialize(aString, nameValue.getDataOutput());
+ int32Serde.serialize(new AInt32(indexDetails.getSourceAvgItemSize()), fieldValue.getDataOutput());
+ recordBuilder.addField(nameValue, fieldValue);
+
+ Map<String, IndexStats> indexesStats = indexDetails.getIndexesStats();
+ if (!indexesStats.isEmpty()) {
+ indexesStatsListBuilder.reset(AOrderedListType.FULL_OPEN_ORDEREDLIST_TYPE);
+ for (Map.Entry<String, IndexStats> stats : indexesStats.entrySet()) {
+ indexStatsRecordBuilder.reset(RecordUtil.FULLY_OPEN_RECORD_TYPE);
+ // index name
+ nameValue.reset();
+ itemValue.reset();
+ aString.setValue(STATS_INDEX_NAME);
+ stringSerde.serialize(aString, nameValue.getDataOutput());
+ aString.setValue(stats.getKey());
+ stringSerde.serialize(aString, itemValue.getDataOutput());
+ indexStatsRecordBuilder.addField(nameValue, itemValue);
+
+ // index number of pages
+ nameValue.reset();
+ itemValue.reset();
+ aString.setValue(STATS_NUM_PAGES);
+ stringSerde.serialize(aString, nameValue.getDataOutput());
+ aInt64.setValue(stats.getValue().getNumPages());
+ int64Serde.serialize(aInt64, itemValue.getDataOutput());
+ indexStatsRecordBuilder.addField(nameValue, itemValue);
+
+ itemValue.reset();
+ indexStatsRecordBuilder.write(itemValue.getDataOutput(), true);
+ indexesStatsListBuilder.addItem(itemValue);
+ }
+ nameValue.reset();
+ fieldValue.reset();
+ aString.setValue(INDEXES_STATS);
+ stringSerde.serialize(aString, nameValue.getDataOutput());
+ indexesStatsListBuilder.write(fieldValue.getDataOutput(), true);
+ recordBuilder.addField(nameValue, fieldValue);
+ }
+ }
+ }
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedMetadataUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedMetadataUtil.java
index 92390a7..0ee9516 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedMetadataUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedMetadataUtil.java
@@ -30,7 +30,6 @@
import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.common.exceptions.MetadataException;
-import org.apache.asterix.common.exceptions.NoOpWarningCollector;
import org.apache.asterix.common.external.IDataSourceAdapter;
import org.apache.asterix.common.external.IDataSourceAdapter.AdapterType;
import org.apache.asterix.common.functions.ExternalFunctionLanguage;
@@ -62,6 +61,7 @@
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.NoOpWarningCollector;
/**
* A utility class for providing helper functions for feeds TODO: Refactor this
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ArrayIndexUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ArrayIndexUtil.java
index 1abf300..498630a 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ArrayIndexUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ArrayIndexUtil.java
@@ -35,6 +35,7 @@
import org.apache.asterix.om.types.AbstractCollectionType;
import org.apache.asterix.om.types.IAType;
import org.apache.asterix.om.utils.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.RecordUtil;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.common.utils.Pair;
@@ -240,8 +241,9 @@
* Traverse each distinct record path and invoke the appropriate commands for each scenario. Here, we keep track
* of the record/list type at each step and give this to each command.
*/
- public static void walkArrayPath(Index index, ARecordType baseRecordType, List<String> flattenedFieldName,
- List<Boolean> unnestFlags, TypeTrackerCommandExecutor commandExecutor) throws AlgebricksException {
+ public static void walkArrayPath(Index index, Index.ArrayIndexElement workingElement, ARecordType baseRecordType,
+ List<String> flattenedFieldName, List<Boolean> unnestFlags, TypeTrackerCommandExecutor commandExecutor)
+ throws AlgebricksException {
ArrayPath arrayPath = new ArrayPath(flattenedFieldName, unnestFlags).invoke();
List<List<String>> fieldNamesPerArray = arrayPath.fieldNamesPerArray;
List<Boolean> unnestFlagsPerArray = arrayPath.unnestFlagsPerArray;
@@ -253,7 +255,8 @@
IAType workingType = baseRecordType;
for (int i = 0; i < fieldNamesPerArray.size(); i++) {
- ARecordType startingStepRecordType = (isTrackingType) ? (ARecordType) workingType : null;
+ ARecordType startingStepRecordType =
+ (isTrackingType) ? (ARecordType) workingType : RecordUtil.FULLY_OPEN_RECORD_TYPE;
if (isTrackingType) {
if (!workingType.getTypeTag().equals(ATypeTag.OBJECT)) {
throw new AsterixException(ErrorCode.COMPILATION_ERROR, "Mismatched record type to depth-"
@@ -286,15 +289,15 @@
}
}
boolean isFirstArrayStep = i == 0;
- boolean isLastUnnestInIntermediateStep = i < fieldNamesPerArray.size() - 1;
+ boolean isLastUnnestInIntermediateStep = i <= fieldNamesPerArray.size() - 1;
commandExecutor.executeActionOnEachArrayStep(startingStepRecordType, workingType,
fieldNamesPerArray.get(i), isFirstArrayStep, isLastUnnestInIntermediateStep);
}
if (i == fieldNamesPerArray.size() - 1) {
boolean isNonArrayStep = !unnestFlagsPerArray.get(i);
- commandExecutor.executeActionOnFinalArrayStep(startingStepRecordType, fieldNamesPerArray.get(i),
- isNonArrayStep, requiresOnlyOneUnnest);
+ commandExecutor.executeActionOnFinalArrayStep(workingElement, baseRecordType, startingStepRecordType,
+ fieldNamesPerArray.get(i), isNonArrayStep, requiresOnlyOneUnnest);
}
}
}
@@ -341,8 +344,9 @@
List<String> fieldName, boolean isFirstArrayStep, boolean isLastUnnestInIntermediateStep)
throws AlgebricksException;
- void executeActionOnFinalArrayStep(ARecordType startingStepRecordType, List<String> fieldName,
- boolean isNonArrayStep, boolean requiresOnlyOneUnnest) throws AlgebricksException;
+ void executeActionOnFinalArrayStep(Index.ArrayIndexElement workingElement, ARecordType baseRecordType,
+ ARecordType startingStepRecordType, List<String> fieldName, boolean isNonArrayStep,
+ boolean requiresOnlyOneUnnest) throws AlgebricksException;
}
private static class ArrayPath {
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java
index 6529794..b2e6817 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java
@@ -21,7 +21,6 @@
import static org.apache.asterix.common.utils.IdentifierUtil.dataset;
import java.io.DataOutput;
-import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Map;
@@ -613,10 +612,10 @@
appCtx.getMetadataLockManager().acquireNodeGroupWriteLock(metadataProvider.getLocks(), nodeGroup);
NodeGroup ng = MetadataManager.INSTANCE.getNodegroup(mdTxnCtx, nodeGroup);
if (ng != null) {
- nodeGroup = nodeGroup + "_" + UUID.randomUUID().toString();
+ nodeGroup = nodeGroup + "_" + UUID.randomUUID();
appCtx.getMetadataLockManager().acquireNodeGroupWriteLock(metadataProvider.getLocks(), nodeGroup);
}
- MetadataManager.INSTANCE.addNodegroup(mdTxnCtx, NodeGroup.createOrdered(nodeGroup, new ArrayList<>(ncNames)));
+ MetadataManager.INSTANCE.addNodegroup(mdTxnCtx, NodeGroup.createOrdered(nodeGroup, ncNames));
return nodeGroup;
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ExternalIndexingOperations.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ExternalIndexingOperations.java
index b899e16..c6eafe0 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ExternalIndexingOperations.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ExternalIndexingOperations.java
@@ -30,7 +30,6 @@
import org.apache.asterix.common.config.DatasetConfig.ExternalFilePendingOp;
import org.apache.asterix.common.config.DatasetConfig.TransactionState;
import org.apache.asterix.common.context.IStorageComponentProvider;
-import org.apache.asterix.common.exceptions.NoOpWarningCollector;
import org.apache.asterix.external.api.ITypedAdapterFactory;
import org.apache.asterix.external.indexing.ExternalFile;
import org.apache.asterix.external.indexing.IndexingConstants;
@@ -64,6 +63,7 @@
import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.NoOpWarningCollector;
import org.apache.hyracks.api.exceptions.SourceLocation;
import org.apache.hyracks.api.job.JobSpecification;
import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ISecondaryIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ISecondaryIndexOperationsHelper.java
new file mode 100644
index 0000000..13ff19d
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ISecondaryIndexOperationsHelper.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.metadata.utils;
+
+import java.util.Set;
+
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
+import org.apache.hyracks.storage.am.common.dataflow.IndexDropOperatorDescriptor;
+
+public interface ISecondaryIndexOperationsHelper {
+
+ void init() throws AlgebricksException;
+
+ JobSpecification buildCreationJobSpec() throws AlgebricksException;
+
+ JobSpecification buildLoadingJobSpec() throws AlgebricksException;
+
+ JobSpecification buildCompactJobSpec() throws AlgebricksException;
+
+ JobSpecification buildDropJobSpec(Set<IndexDropOperatorDescriptor.DropOption> options) throws AlgebricksException;
+
+ IFileSplitProvider getSecondaryFileSplitProvider();
+
+ RecordDescriptor getSecondaryRecDesc();
+
+ IBinaryComparatorFactory[] getSecondaryComparatorFactories();
+
+ AlgebricksPartitionConstraint getSecondaryPartitionConstraint();
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java
index ad92803..7e9cfc3 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java
@@ -79,6 +79,8 @@
} else if (index.getIndexType() == DatasetConfig.IndexType.ARRAY) {
numSecondaryKeys = ((Index.ArrayIndexDetails) index.getIndexDetails()).getElementList().stream()
.map(e -> e.getProjectList().size()).reduce(0, Integer::sum);
+ } else if (index.getIndexType() == DatasetConfig.IndexType.SAMPLE) {
+ return null;
} else {
throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE, index.getIndexType().toString());
}
@@ -111,6 +113,8 @@
case SINGLE_PARTITION_NGRAM_INVIX:
case SINGLE_PARTITION_WORD_INVIX:
break;
+ case SAMPLE:
+ break;
default:
throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE,
index.getIndexType().toString());
@@ -120,21 +124,21 @@
public static JobSpecification buildDropIndexJobSpec(Index index, MetadataProvider metadataProvider,
Dataset dataset, SourceLocation sourceLoc) throws AlgebricksException {
- SecondaryIndexOperationsHelper secondaryIndexHelper =
+ ISecondaryIndexOperationsHelper secondaryIndexHelper =
SecondaryIndexOperationsHelper.createIndexOperationsHelper(dataset, index, metadataProvider, sourceLoc);
return secondaryIndexHelper.buildDropJobSpec(EnumSet.noneOf(DropOption.class));
}
public static JobSpecification buildDropIndexJobSpec(Index index, MetadataProvider metadataProvider,
Dataset dataset, Set<DropOption> options, SourceLocation sourceLoc) throws AlgebricksException {
- SecondaryIndexOperationsHelper secondaryIndexHelper =
+ ISecondaryIndexOperationsHelper secondaryIndexHelper =
SecondaryIndexOperationsHelper.createIndexOperationsHelper(dataset, index, metadataProvider, sourceLoc);
return secondaryIndexHelper.buildDropJobSpec(options);
}
public static JobSpecification buildSecondaryIndexCreationJobSpec(Dataset dataset, Index index,
MetadataProvider metadataProvider, SourceLocation sourceLoc) throws AlgebricksException {
- SecondaryIndexOperationsHelper secondaryIndexHelper =
+ ISecondaryIndexOperationsHelper secondaryIndexHelper =
SecondaryIndexOperationsHelper.createIndexOperationsHelper(dataset, index, metadataProvider, sourceLoc);
return secondaryIndexHelper.buildCreationJobSpec();
}
@@ -147,8 +151,8 @@
public static JobSpecification buildSecondaryIndexLoadingJobSpec(Dataset dataset, Index index,
MetadataProvider metadataProvider, List<ExternalFile> files, SourceLocation sourceLoc)
throws AlgebricksException {
- SecondaryIndexOperationsHelper secondaryIndexHelper;
- if (dataset.isCorrelated()) {
+ ISecondaryIndexOperationsHelper secondaryIndexHelper;
+ if (dataset.isCorrelated() && supportsCorrelated(index.getIndexType())) { //TODO:REVISIT
secondaryIndexHelper = SecondaryCorrelatedTreeIndexOperationsHelper.createIndexOperationsHelper(dataset,
index, metadataProvider, sourceLoc);
} else {
@@ -156,14 +160,18 @@
metadataProvider, sourceLoc);
}
if (files != null) {
- secondaryIndexHelper.setExternalFiles(files);
+ ((SecondaryIndexOperationsHelper) secondaryIndexHelper).setExternalFiles(files);
}
return secondaryIndexHelper.buildLoadingJobSpec();
}
+ private static boolean supportsCorrelated(DatasetConfig.IndexType indexType) {
+ return indexType != DatasetConfig.IndexType.SAMPLE;
+ }
+
public static JobSpecification buildSecondaryIndexCompactJobSpec(Dataset dataset, Index index,
MetadataProvider metadataProvider, SourceLocation sourceLoc) throws AlgebricksException {
- SecondaryIndexOperationsHelper secondaryIndexHelper =
+ ISecondaryIndexOperationsHelper secondaryIndexHelper =
SecondaryIndexOperationsHelper.createIndexOperationsHelper(dataset, index, metadataProvider, sourceLoc);
return secondaryIndexHelper.buildCompactJobSpec();
}
@@ -240,4 +248,9 @@
public static boolean excludesUnknowns(Index index) {
return !includesUnknowns(index);
}
+
+ public static Pair<String, String> getSampleIndexNames(String datasetName) {
+ return new Pair<>(MetadataConstants.SAMPLE_INDEX_1_PREFIX + datasetName,
+ MetadataConstants.SAMPLE_INDEX_2_PREFIX + datasetName);
+ }
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/KeyFieldTypeUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/KeyFieldTypeUtil.java
index c4cc1de..fd20b3c 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/KeyFieldTypeUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/KeyFieldTypeUtil.java
@@ -128,16 +128,16 @@
* @return a list of IATypes, one for each corresponding index key field.
* @throws AlgebricksException
*/
- public static List<IAType> getBTreeIndexKeyTypes(Index index, ARecordType recordType, ARecordType metaRecordType)
- throws AlgebricksException {
+ public static List<Pair<IAType, Boolean>> getBTreeIndexKeyTypes(Index index, ARecordType recordType,
+ ARecordType metaRecordType) throws AlgebricksException {
Index.ValueIndexDetails indexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
List<Integer> keySourceIndicators = indexDetails.getKeyFieldSourceIndicators();
- List<IAType> indexKeyTypes = new ArrayList<>();
+ List<Pair<IAType, Boolean>> indexKeyTypes = new ArrayList<>();
for (int i = 0; i < indexDetails.getKeyFieldNames().size(); i++) {
Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(index,
indexDetails.getKeyFieldTypes().get(i), indexDetails.getKeyFieldNames().get(i),
chooseSource(keySourceIndicators, i, recordType, metaRecordType));
- indexKeyTypes.add(keyPairType.first);
+ indexKeyTypes.add(keyPairType);
}
return indexKeyTypes;
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataConstants.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataConstants.java
index b52cddd..0f0ea40 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataConstants.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataConstants.java
@@ -57,6 +57,11 @@
public static final String FULL_TEXT_CONFIG_DATASET_NAME = "FullTextConfig";
public static final String FULL_TEXT_FILTER_DATASET_NAME = "FullTextFilter";
+ public static final String PRIMARY_INDEX_PREFIX = "primary_idx_";
+ public static final String SAMPLE_INDEX_PREFIX = "sample_idx_";
+ public static final String SAMPLE_INDEX_1_PREFIX = SAMPLE_INDEX_PREFIX + "1_";
+ public static final String SAMPLE_INDEX_2_PREFIX = SAMPLE_INDEX_PREFIX + "2_";
+
private MetadataConstants() {
}
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataLockUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataLockUtil.java
index 7bb11e5..1de387f 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataLockUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataLockUtil.java
@@ -311,6 +311,20 @@
}
@Override
+ public void analyzeDatasetBegin(IMetadataLockManager lockMgr, LockList locks, DataverseName dataverseName,
+ String datasetName) throws AlgebricksException {
+ lockMgr.acquireDataverseReadLock(locks, dataverseName);
+ lockMgr.acquireDatasetCreateIndexLock(locks, dataverseName, datasetName);
+ }
+
+ @Override
+ public void analyzeDatasetDropBegin(IMetadataLockManager lockMgr, LockList locks, DataverseName dataverseName,
+ String datasetName) throws AlgebricksException {
+ lockMgr.acquireDataverseReadLock(locks, dataverseName);
+ lockMgr.acquireDatasetWriteLock(locks, dataverseName, datasetName);
+ }
+
+ @Override
public void compactBegin(IMetadataLockManager lockMgr, LockList locks, DataverseName dataverseName,
String datasetName) throws AlgebricksException {
lockMgr.acquireDataverseReadLock(locks, dataverseName);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SampleOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SampleOperationsHelper.java
new file mode 100644
index 0000000..056a8c2
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SampleOperationsHelper.java
@@ -0,0 +1,360 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.metadata.utils;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.asterix.common.config.OptimizationConfUtil;
+import org.apache.asterix.common.context.IStorageComponentProvider;
+import org.apache.asterix.common.utils.StorageConstants;
+import org.apache.asterix.formats.base.IDataFormat;
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.metadata.entities.Dataset;
+import org.apache.asterix.metadata.entities.Index;
+import org.apache.asterix.om.base.AInt32;
+import org.apache.asterix.om.constants.AsterixConstantValue;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.runtime.aggregates.collections.FirstElementEvalFactory;
+import org.apache.asterix.runtime.evaluators.comparisons.GreaterThanDescriptor;
+import org.apache.asterix.runtime.operators.DatasetStreamStatsOperatorDescriptor;
+import org.apache.asterix.runtime.operators.LSMIndexBulkLoadOperatorDescriptor;
+import org.apache.asterix.runtime.runningaggregates.std.SampleSlotRunningAggregateFunctionFactory;
+import org.apache.asterix.runtime.runningaggregates.std.TidRunningAggregateDescriptor;
+import org.apache.asterix.runtime.utils.RuntimeUtils;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
+import org.apache.hyracks.algebricks.core.jobgen.impl.ConnectorPolicyAssignmentPolicy;
+import org.apache.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
+import org.apache.hyracks.algebricks.data.INormalizedKeyComputerFactoryProvider;
+import org.apache.hyracks.algebricks.data.ISerializerDeserializerProvider;
+import org.apache.hyracks.algebricks.data.ITypeTraitProvider;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
+import org.apache.hyracks.algebricks.runtime.base.IRunningAggregateEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.evaluators.ColumnAccessEvalFactory;
+import org.apache.hyracks.algebricks.runtime.operators.aggreg.SimpleAlgebricksAccumulatingAggregatorFactory;
+import org.apache.hyracks.algebricks.runtime.operators.aggrun.RunningAggregateRuntimeFactory;
+import org.apache.hyracks.algebricks.runtime.operators.base.SinkRuntimeFactory;
+import org.apache.hyracks.algebricks.runtime.operators.meta.AlgebricksMetaOperatorDescriptor;
+import org.apache.hyracks.algebricks.runtime.operators.std.StreamProjectRuntimeFactory;
+import org.apache.hyracks.algebricks.runtime.operators.std.StreamSelectRuntimeFactory;
+import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
+import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
+import org.apache.hyracks.dataflow.std.group.AbstractAggregatorDescriptorFactory;
+import org.apache.hyracks.dataflow.std.group.sort.SortGroupByOperatorDescriptor;
+import org.apache.hyracks.storage.am.common.api.IIndexBuilderFactory;
+import org.apache.hyracks.storage.am.common.build.IndexBuilderFactory;
+import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import org.apache.hyracks.storage.am.common.dataflow.IndexCreateOperatorDescriptor;
+import org.apache.hyracks.storage.am.common.dataflow.IndexDataflowHelperFactory;
+import org.apache.hyracks.storage.am.common.dataflow.IndexDropOperatorDescriptor;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
+import org.apache.hyracks.storage.common.IResourceFactory;
+import org.apache.hyracks.storage.common.IStorageManager;
+
+/**
+ * Utility class for sampling operations.
+ * <p>
+ * The sampling method described in:
+ * "A Convenient Algorithm for Drawing a Simple Random Sample",
+ * by A. I. McLeod and D. R. Bellhouse
+ */
+public class SampleOperationsHelper implements ISecondaryIndexOperationsHelper {
+
+ public static final String DATASET_STATS_OPERATOR_NAME = "Sample.DatasetStats";
+
+ private final MetadataProvider metadataProvider;
+ private final Dataset dataset;
+ private final Index index;
+ private final SourceLocation sourceLoc;
+
+ private ARecordType itemType;
+ private ARecordType metaType;
+ private RecordDescriptor recordDesc;
+ private IBinaryComparatorFactory[] comparatorFactories;
+ private IFileSplitProvider fileSplitProvider;
+ private AlgebricksPartitionConstraint partitionConstraint;
+ private ILSMMergePolicyFactory mergePolicyFactory;
+ private Map<String, String> mergePolicyProperties;
+ private int groupbyNumFrames;
+
+ protected SampleOperationsHelper(Dataset dataset, Index index, MetadataProvider metadataProvider,
+ SourceLocation sourceLoc) {
+ this.dataset = dataset;
+ this.index = index;
+ this.metadataProvider = metadataProvider;
+ this.sourceLoc = sourceLoc;
+ }
+
+ @Override
+ public void init() throws AlgebricksException {
+ itemType =
+ (ARecordType) metadataProvider.findType(dataset.getItemTypeDataverseName(), dataset.getItemTypeName());
+ metaType = DatasetUtil.getMetaType(metadataProvider, dataset);
+ recordDesc = dataset.getPrimaryRecordDescriptor(metadataProvider);
+ comparatorFactories = dataset.getPrimaryComparatorFactories(metadataProvider, itemType, metaType);
+ groupbyNumFrames = getGroupByNumFrames(metadataProvider, sourceLoc);
+
+ Pair<IFileSplitProvider, AlgebricksPartitionConstraint> secondarySplitsAndConstraint =
+ metadataProvider.getSplitProviderAndConstraints(dataset, index.getIndexName());
+ fileSplitProvider = secondarySplitsAndConstraint.first;
+ partitionConstraint = secondarySplitsAndConstraint.second;
+
+ Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo =
+ DatasetUtil.getMergePolicyFactory(dataset, metadataProvider.getMetadataTxnContext());
+ mergePolicyFactory = compactionInfo.first;
+ mergePolicyProperties = compactionInfo.second;
+ }
+
+ @Override
+ public JobSpecification buildCreationJobSpec() throws AlgebricksException {
+ JobSpecification spec = RuntimeUtils.createJobSpecification(metadataProvider.getApplicationContext());
+ IStorageComponentProvider storageComponentProvider = metadataProvider.getStorageComponentProvider();
+ IResourceFactory resourceFactory = dataset.getResourceFactory(metadataProvider, index, itemType, metaType,
+ mergePolicyFactory, mergePolicyProperties);
+ IIndexBuilderFactory indexBuilderFactory = new IndexBuilderFactory(storageComponentProvider.getStorageManager(),
+ fileSplitProvider, resourceFactory, true);
+ IndexCreateOperatorDescriptor indexCreateOp = new IndexCreateOperatorDescriptor(spec, indexBuilderFactory);
+ indexCreateOp.setSourceLocation(sourceLoc);
+ AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, indexCreateOp, partitionConstraint);
+ spec.addRoot(indexCreateOp);
+ spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
+ return spec;
+ }
+
+ @Override
+ public JobSpecification buildLoadingJobSpec() throws AlgebricksException {
+ Index.SampleIndexDetails indexDetails = (Index.SampleIndexDetails) index.getIndexDetails();
+ int sampleCardinalityTarget = indexDetails.getSampleCardinalityTarget();
+ long sampleSeed = indexDetails.getSampleSeed();
+ IDataFormat format = metadataProvider.getDataFormat();
+ int nFields = recordDesc.getFieldCount();
+ int[] columns = new int[nFields];
+ for (int i = 0; i < nFields; i++) {
+ columns[i] = i;
+ }
+ IStorageManager storageMgr = metadataProvider.getStorageComponentProvider().getStorageManager();
+ JobSpecification spec = RuntimeUtils.createJobSpecification(metadataProvider.getApplicationContext());
+ IIndexDataflowHelperFactory dataflowHelperFactory =
+ new IndexDataflowHelperFactory(storageMgr, fileSplitProvider);
+
+ // job spec:
+ IndexUtil.bindJobEventListener(spec, metadataProvider);
+
+ // dummy key provider ----> primary index scan
+ IOperatorDescriptor sourceOp = DatasetUtil.createDummyKeyProviderOp(spec, dataset, metadataProvider);
+ IOperatorDescriptor targetOp = DatasetUtil.createPrimaryIndexScanOp(spec, metadataProvider, dataset);
+ spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
+ sourceOp = targetOp;
+
+ // primary index scan ----> stream stats op
+ List<Pair<IFileSplitProvider, String>> indexesInfo = metadataProvider.getSplitProviderOfAllIndexes(dataset);
+ IndexDataflowHelperFactory[] indexes = new IndexDataflowHelperFactory[indexesInfo.size()];
+ String[] names = new String[indexesInfo.size()];
+ for (int i = 0; i < indexes.length; i++) {
+ Pair<IFileSplitProvider, String> indexInfo = indexesInfo.get(i);
+ indexes[i] = new IndexDataflowHelperFactory(storageMgr, indexInfo.first);
+ names[i] = indexInfo.second;
+ }
+ targetOp =
+ new DatasetStreamStatsOperatorDescriptor(spec, recordDesc, DATASET_STATS_OPERATOR_NAME, indexes, names);
+ spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
+ sourceOp = targetOp;
+
+ // stream stats op ----> (running agg + select)
+ // ragg produces a slot number and a tuple counter for each tuple
+ // If the slot number is 0 then the tuple is not in the sample and is removed by subsequent select op.
+ // If the slot number is greater than 0 then the tuple is in the sample.
+ // There could be several tuples with the same slot number, the latest one wins
+ // (with the greatest tuple counter). This is accomplished by the group by below
+ BuiltinType raggSlotType = BuiltinType.AINT32;
+ BuiltinType raggCounterType = BuiltinType.AINT64;
+ int[] raggProjectColumns = new int[nFields + 2];
+ raggProjectColumns[0] = nFields;
+ raggProjectColumns[1] = nFields + 1;
+ System.arraycopy(columns, 0, raggProjectColumns, 2, nFields);
+ int[] raggAggColumns = { nFields, nFields + 1 };
+
+ ISerializerDeserializerProvider serdeProvider = format.getSerdeProvider();
+ ISerializerDeserializer[] raggSerdes = new ISerializerDeserializer[nFields + 2];
+ raggSerdes[0] = serdeProvider.getSerializerDeserializer(raggSlotType);
+ raggSerdes[1] = serdeProvider.getSerializerDeserializer(raggCounterType);
+ System.arraycopy(recordDesc.getFields(), 0, raggSerdes, 2, nFields);
+
+ ITypeTraitProvider typeTraitProvider = format.getTypeTraitProvider();
+ ITypeTraits[] raggTraits = new ITypeTraits[nFields + 2];
+ raggTraits[0] = typeTraitProvider.getTypeTrait(raggSlotType);
+ raggTraits[1] = typeTraitProvider.getTypeTrait(raggCounterType);
+ System.arraycopy(recordDesc.getTypeTraits(), 0, raggTraits, 2, nFields);
+
+ RecordDescriptor raggRecordDesc = new RecordDescriptor(raggSerdes, raggTraits);
+
+ IRunningAggregateEvaluatorFactory raggSlotEvalFactory =
+ new SampleSlotRunningAggregateFunctionFactory(sampleCardinalityTarget, sampleSeed);
+ IRunningAggregateEvaluatorFactory raggCounterEvalFactory = TidRunningAggregateDescriptor.FACTORY
+ .createFunctionDescriptor().createRunningAggregateEvaluatorFactory(new IScalarEvaluatorFactory[0]);
+ RunningAggregateRuntimeFactory raggRuntimeFactory =
+ new RunningAggregateRuntimeFactory(raggProjectColumns, raggAggColumns,
+ new IRunningAggregateEvaluatorFactory[] { raggSlotEvalFactory, raggCounterEvalFactory });
+
+ IFunctionDescriptor gtDescriptor = GreaterThanDescriptor.FACTORY.createFunctionDescriptor();
+ gtDescriptor.setImmutableStates(raggSlotType, raggSlotType);
+ IScalarEvaluatorFactory gtFactory =
+ gtDescriptor.createEvaluatorFactory(new IScalarEvaluatorFactory[] { new ColumnAccessEvalFactory(0),
+ format.getConstantEvalFactory(new AsterixConstantValue(new AInt32(0))) });
+ StreamSelectRuntimeFactory selectRuntimeFactory = new StreamSelectRuntimeFactory(gtFactory, null,
+ format.getBinaryBooleanInspectorFactory(), false, -1, null);
+
+ targetOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 1,
+ new IPushRuntimeFactory[] { raggRuntimeFactory, selectRuntimeFactory },
+ new RecordDescriptor[] { raggRecordDesc, raggRecordDesc });
+ spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
+ sourceOp = targetOp;
+
+ // (running agg + select) ---> group-by
+ int[] groupFields = new int[] { 0 }; // [slot]
+ int[] sortFields = new int[] { 0, 1 }; // [slot, counter]
+ OrderOperator.IOrder sortSlotOrder = OrderOperator.ASC_ORDER;
+ OrderOperator.IOrder sortCounterOrder = OrderOperator.DESC_ORDER;
+ IBinaryComparatorFactoryProvider comparatorFactoryProvider = format.getBinaryComparatorFactoryProvider();
+ IBinaryComparatorFactory[] raggCmpFactories = {
+ comparatorFactoryProvider.getBinaryComparatorFactory(raggSlotType,
+ sortSlotOrder.getKind() == OrderOperator.IOrder.OrderKind.ASC),
+ comparatorFactoryProvider.getBinaryComparatorFactory(raggCounterType,
+ sortCounterOrder.getKind() == OrderOperator.IOrder.OrderKind.ASC) };
+
+ INormalizedKeyComputerFactoryProvider normKeyProvider = format.getNormalizedKeyComputerFactoryProvider();
+ INormalizedKeyComputerFactory[] normKeyFactories = {
+ normKeyProvider.getNormalizedKeyComputerFactory(raggSlotType,
+ sortSlotOrder.getKind() == OrderOperator.IOrder.OrderKind.ASC),
+ normKeyProvider.getNormalizedKeyComputerFactory(raggCounterType,
+ sortCounterOrder.getKind() == OrderOperator.IOrder.OrderKind.ASC) };
+
+ // agg = [counter, .. original columns ..]
+ IAggregateEvaluatorFactory[] aggFactories = new IAggregateEvaluatorFactory[nFields + 1];
+ for (int i = 0; i < aggFactories.length; i++) {
+ aggFactories[i] = new FirstElementEvalFactory(
+ new IScalarEvaluatorFactory[] { new ColumnAccessEvalFactory(1 + i) }, false, sourceLoc);
+ }
+ AbstractAggregatorDescriptorFactory aggregatorFactory =
+ new SimpleAlgebricksAccumulatingAggregatorFactory(aggFactories, groupFields);
+
+ targetOp = new SortGroupByOperatorDescriptor(spec, groupbyNumFrames, sortFields, groupFields, normKeyFactories,
+ raggCmpFactories, aggregatorFactory, aggregatorFactory, raggRecordDesc, raggRecordDesc, false);
+ spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
+ sourceOp = targetOp;
+
+ // group by --> project (remove ragg fields)
+ int[] projectColumns = new int[nFields];
+ for (int i = 0; i < nFields; i++) {
+ projectColumns[i] = 2 + i;
+ }
+ StreamProjectRuntimeFactory projectRuntimeFactory = new StreamProjectRuntimeFactory(projectColumns);
+ targetOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 1, new IPushRuntimeFactory[] { projectRuntimeFactory },
+ new RecordDescriptor[] { recordDesc });
+ spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
+ sourceOp = targetOp;
+
+ // project ---> bulk load op
+ targetOp = createTreeIndexBulkLoadOp(spec, columns, dataflowHelperFactory,
+ StorageConstants.DEFAULT_TREE_FILL_FACTOR, sampleCardinalityTarget);
+ spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
+ sourceOp = targetOp;
+
+ // bulk load op ----> sink op
+ SinkRuntimeFactory sinkRuntimeFactory = new SinkRuntimeFactory();
+ sinkRuntimeFactory.setSourceLocation(sourceLoc);
+ targetOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 0, new IPushRuntimeFactory[] { sinkRuntimeFactory },
+ new RecordDescriptor[] { recordDesc });
+ spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
+
+ spec.addRoot(targetOp);
+ spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
+
+ return spec;
+ }
+
+ protected LSMIndexBulkLoadOperatorDescriptor createTreeIndexBulkLoadOp(JobSpecification spec,
+ int[] fieldPermutation, IIndexDataflowHelperFactory dataflowHelperFactory, float fillFactor,
+ long numElementHint) {
+ LSMIndexBulkLoadOperatorDescriptor treeIndexBulkLoadOp = new LSMIndexBulkLoadOperatorDescriptor(spec,
+ recordDesc, fieldPermutation, fillFactor, false, numElementHint, true, dataflowHelperFactory, null,
+ LSMIndexBulkLoadOperatorDescriptor.BulkLoadUsage.LOAD, dataset.getDatasetId(), null);
+ treeIndexBulkLoadOp.setSourceLocation(sourceLoc);
+ AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, treeIndexBulkLoadOp,
+ partitionConstraint);
+ return treeIndexBulkLoadOp;
+ }
+
+ @Override
+ public JobSpecification buildDropJobSpec(Set<IndexDropOperatorDescriptor.DropOption> options)
+ throws AlgebricksException {
+ return SecondaryTreeIndexOperationsHelper.buildDropJobSpecImpl(dataset, index, options, metadataProvider,
+ sourceLoc);
+ }
+
+ @Override
+ public JobSpecification buildCompactJobSpec() {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public IFileSplitProvider getSecondaryFileSplitProvider() {
+ return fileSplitProvider;
+ }
+
+ @Override
+ public RecordDescriptor getSecondaryRecDesc() {
+ return recordDesc;
+ }
+
+ @Override
+ public IBinaryComparatorFactory[] getSecondaryComparatorFactories() {
+ return comparatorFactories;
+ }
+
+ @Override
+ public AlgebricksPartitionConstraint getSecondaryPartitionConstraint() {
+ return partitionConstraint;
+ }
+
+ private static int getGroupByNumFrames(MetadataProvider metadataProvider, SourceLocation sourceLoc)
+ throws AlgebricksException {
+ return OptimizationConfUtil.getGroupByNumFrames(
+ metadataProvider.getApplicationContext().getCompilerProperties(), metadataProvider.getConfig(),
+ sourceLoc);
+ }
+}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryArrayIndexBTreeOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryArrayIndexBTreeOperationsHelper.java
index c8a7ee1..de0ca5a 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryArrayIndexBTreeOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryArrayIndexBTreeOperationsHelper.java
@@ -158,7 +158,7 @@
int flattenedListPos = 0;
for (Index.ArrayIndexElement e : arrayIndexDetails.getElementList()) {
for (int i = 0; i < e.getProjectList().size(); i++) {
- addSKEvalFactories(isOverridingKeyFieldTypes ? enforcedItemType : itemType, flattenedListPos, false);
+ addSKEvalFactories(itemType, flattenedListPos, false, e);
Pair<IAType, Boolean> keyTypePair = ArrayIndexUtil.getNonNullableOpenFieldType(e.getTypeList().get(i),
e.getUnnestList(), e.getProjectList().get(i), itemType);
IAType keyType = keyTypePair.first;
@@ -203,7 +203,7 @@
ARecordType filterItemType =
((InternalDatasetDetails) dataset.getDatasetDetails()).getFilterSourceIndicator() == 0 ? itemType
: metaType;
- addSKEvalFactories(itemType, numSecondaryKeys, true);
+ addSKEvalFactories(itemType, numSecondaryKeys, true, null);
Pair<IAType, Boolean> keyTypePair;
keyTypePair = Index.getNonNullableKeyFieldType(filterFieldName, filterItemType);
IAType type = keyTypePair.first;
@@ -230,8 +230,8 @@
return fieldPermutation;
}
- protected void addSKEvalFactories(ARecordType recordType, int fieldPos, boolean isFilterField)
- throws AlgebricksException {
+ protected void addSKEvalFactories(ARecordType recordType, int fieldPos, boolean isFilterField,
+ Index.ArrayIndexElement workingElement) throws AlgebricksException {
if (isFilterField) {
addFilterFieldToBuilder(recordType);
return;
@@ -245,7 +245,8 @@
} else {
EvalFactoryAndRecDescInvoker commandExecutor =
new EvalFactoryAndRecDescInvoker(!evalFactoryAndRecDescStackBuilder.isUnnestEvalPopulated());
- ArrayIndexUtil.walkArrayPath(index, recordType, flattenedFieldName, workingUnnestFlags, commandExecutor);
+ ArrayIndexUtil.walkArrayPath(index, workingElement, recordType, flattenedFieldName, workingUnnestFlags,
+ commandExecutor);
}
}
@@ -267,12 +268,6 @@
spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
sourceOp = targetOp;
- if (arrayIndexDetails.isOverridingKeyFieldTypes() && !enforcedItemType.equals(itemType)) {
- // If we have an enforced type, insert a "cast" after the primary index scan.
- targetOp = createCastOp(spec, dataset.getDatasetType(), index.isEnforced());
- spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
- sourceOp = targetOp;
- }
// We do not index meta fields. Project away meta fields if they exist.
if (dataset.hasMetaPart()) {
@@ -305,7 +300,15 @@
if (anySecondaryKeyIsNullable || arrayIndexDetails.isOverridingKeyFieldTypes()) {
// If any of the secondary fields are nullable, then we need to filter out the nulls.
- targetOp = createFilterAnyUnknownSelectOp(spec, numTotalSecondaryKeys, secondaryRecDesc);
+ List<IAType> secondaryKeyTypes = new ArrayList<>();
+ if (arrayIndexDetails.isOverridingKeyFieldTypes() && !enforcedItemType.equals(itemType)) {
+ for (Index.ArrayIndexElement arrayIndexElement : arrayIndexDetails.getElementList()) {
+ List<IAType> typeList = arrayIndexElement.getTypeList();
+ secondaryKeyTypes.addAll(typeList);
+ }
+ }
+ targetOp = createCastFilterAnyUnknownSelectOp(spec, numTotalSecondaryKeys, secondaryRecDesc,
+ secondaryKeyTypes);
spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
sourceOp = targetOp;
}
@@ -556,8 +559,9 @@
}
@Override
- public void executeActionOnFinalArrayStep(ARecordType startingStepRecordType, List<String> fieldName,
- boolean isNonArrayStep, boolean requiresOnlyOneUnnest) throws AlgebricksException {
+ public void executeActionOnFinalArrayStep(Index.ArrayIndexElement workingElement, ARecordType baseRecordType,
+ ARecordType startingStepRecordType, List<String> fieldName, boolean isNonArrayStep,
+ boolean requiresOnlyOneUnnest) throws AlgebricksException {
// If the final value is nested inside a record, add this SEF.
if (!isNonArrayStep) {
return;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
index 9e0d650..ab64b18 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
@@ -91,7 +91,7 @@
// TODO: We should eventually have a hierarchy of classes that can create all
// possible index job specs,
// not just for creation.
-public abstract class SecondaryIndexOperationsHelper {
+public abstract class SecondaryIndexOperationsHelper implements ISecondaryIndexOperationsHelper {
protected final MetadataProvider metadataProvider;
protected final Dataset dataset;
protected final Index index;
@@ -160,10 +160,10 @@
metadataProvider.getConfig(), sourceLoc);
}
- public static SecondaryIndexOperationsHelper createIndexOperationsHelper(Dataset dataset, Index index,
+ public static ISecondaryIndexOperationsHelper createIndexOperationsHelper(Dataset dataset, Index index,
MetadataProvider metadataProvider, SourceLocation sourceLoc) throws AlgebricksException {
- SecondaryIndexOperationsHelper indexOperationsHelper;
+ ISecondaryIndexOperationsHelper indexOperationsHelper;
switch (index.getIndexType()) {
case ARRAY:
indexOperationsHelper =
@@ -182,6 +182,9 @@
indexOperationsHelper =
new SecondaryInvertedIndexOperationsHelper(dataset, index, metadataProvider, sourceLoc);
break;
+ case SAMPLE:
+ indexOperationsHelper = new SampleOperationsHelper(dataset, index, metadataProvider, sourceLoc);
+ break;
default:
throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE, sourceLoc,
index.getIndexType());
@@ -202,7 +205,8 @@
protected abstract int getNumSecondaryKeys();
- protected void init() throws AlgebricksException {
+ @Override
+ public void init() throws AlgebricksException {
payloadSerde = SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(itemType);
metaSerde =
metaType == null ? null : SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(metaType);
@@ -474,9 +478,22 @@
return createFilterSelectOp(spec, numSecondaryKeyFields, secondaryRecDesc, AndDescriptor::new);
}
+ public AlgebricksMetaOperatorDescriptor createCastFilterAnyUnknownSelectOp(JobSpecification spec,
+ int numSecondaryKeyFields, RecordDescriptor secondaryRecDesc, List<IAType> castFieldTypes)
+ throws AlgebricksException {
+ return createFilterSelectOp(spec, numSecondaryKeyFields, secondaryRecDesc, AndDescriptor::new, castFieldTypes);
+ }
+
private AlgebricksMetaOperatorDescriptor createFilterSelectOp(JobSpecification spec, int numSecondaryKeyFields,
RecordDescriptor secondaryRecDesc, Supplier<AbstractFunctionDescriptor> predicatesCombinerFuncSupplier)
throws AlgebricksException {
+ return createFilterSelectOp(spec, numSecondaryKeyFields, secondaryRecDesc, predicatesCombinerFuncSupplier,
+ Collections.emptyList());
+ }
+
+ private AlgebricksMetaOperatorDescriptor createFilterSelectOp(JobSpecification spec, int numSecondaryKeyFields,
+ RecordDescriptor secondaryRecDesc, Supplier<AbstractFunctionDescriptor> predicatesCombinerFuncSupplier,
+ List<IAType> castFieldTypes) throws AlgebricksException {
IScalarEvaluatorFactory[] predicateArgsEvalFactories = new IScalarEvaluatorFactory[numSecondaryKeyFields];
NotDescriptor notDesc = new NotDescriptor();
notDesc.setSourceLocation(sourceLoc);
@@ -485,8 +502,14 @@
for (int i = 0; i < numSecondaryKeyFields; i++) {
// Access column i, and apply 'is not null'.
ColumnAccessEvalFactory columnAccessEvalFactory = new ColumnAccessEvalFactory(i);
+ IScalarEvaluatorFactory evalFactory = columnAccessEvalFactory;
+ if (castFieldTypes != null && !castFieldTypes.isEmpty()) {
+ IScalarEvaluatorFactory[] castArg = new IScalarEvaluatorFactory[] { columnAccessEvalFactory };
+ evalFactory = createCastFunction(castFieldTypes.get(i), BuiltinType.ANY, index.isEnforced(), sourceLoc)
+ .createEvaluatorFactory(castArg);
+ }
IScalarEvaluatorFactory isUnknownEvalFactory =
- isUnknownDesc.createEvaluatorFactory(new IScalarEvaluatorFactory[] { columnAccessEvalFactory });
+ isUnknownDesc.createEvaluatorFactory(new IScalarEvaluatorFactory[] { evalFactory });
IScalarEvaluatorFactory notEvalFactory =
notDesc.createEvaluatorFactory(new IScalarEvaluatorFactory[] { isUnknownEvalFactory });
predicateArgsEvalFactories[i] = notEvalFactory;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryTreeIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryTreeIndexOperationsHelper.java
index 0373e32..e8825fc 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryTreeIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryTreeIndexOperationsHelper.java
@@ -72,13 +72,19 @@
@Override
public JobSpecification buildDropJobSpec(Set<DropOption> options) throws AlgebricksException {
+ return buildDropJobSpecImpl(dataset, index, options, metadataProvider, sourceLoc);
+ }
+
+ static JobSpecification buildDropJobSpecImpl(Dataset dataset, Index index, Set<DropOption> dropOptions,
+ MetadataProvider metadataProvider, SourceLocation sourceLoc) throws AlgebricksException {
JobSpecification spec = RuntimeUtils.createJobSpecification(metadataProvider.getApplicationContext());
Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
metadataProvider.getSplitProviderAndConstraints(dataset, index.getIndexName());
IIndexDataflowHelperFactory dataflowHelperFactory = new IndexDataflowHelperFactory(
metadataProvider.getStorageComponentProvider().getStorageManager(), splitsAndConstraint.first);
// The index drop operation should be persistent regardless of temp datasets or permanent dataset.
- IndexDropOperatorDescriptor btreeDrop = new IndexDropOperatorDescriptor(spec, dataflowHelperFactory, options);
+ IndexDropOperatorDescriptor btreeDrop =
+ new IndexDropOperatorDescriptor(spec, dataflowHelperFactory, dropOptions);
btreeDrop.setSourceLocation(sourceLoc);
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, btreeDrop,
splitsAndConstraint.second);
diff --git a/asterixdb/asterix-om/pom.xml b/asterixdb/asterix-om/pom.xml
index 0fed772..40b3e9b 100644
--- a/asterixdb/asterix-om/pom.xml
+++ b/asterixdb/asterix-om/pom.xml
@@ -21,7 +21,7 @@
<parent>
<artifactId>apache-asterixdb</artifactId>
<groupId>org.apache.asterix</groupId>
- <version>0.9.8-SNAPSHOT</version>
+ <version>0.9.8.2-SNAPSHOT</version>
</parent>
<artifactId>asterix-om</artifactId>
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ComparatorUtil.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ComparatorUtil.java
index 4db23b2..0eb30f8 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ComparatorUtil.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ComparatorUtil.java
@@ -84,9 +84,9 @@
// start points to the value; checking left and right are compatible and numbers has to be done before calling this
static int compareNumbers(ATypeTag lTag, byte[] l, int lStart, ATypeTag rTag, byte[] r, int rStart) {
if (lTag == DOUBLE || rTag == DOUBLE) {
- return Double.compare(getDoubleValue(lTag, l, lStart), getDoubleValue(rTag, r, rStart));
+ return compareDoubles(getDoubleValue(lTag, l, lStart), getDoubleValue(rTag, r, rStart));
} else if (lTag == FLOAT || rTag == FLOAT) {
- return Float.compare((float) getDoubleValue(lTag, l, lStart), (float) getDoubleValue(rTag, r, rStart));
+ return compareFloats((float) getDoubleValue(lTag, l, lStart), (float) getDoubleValue(rTag, r, rStart));
} else if (lTag == BIGINT || rTag == BIGINT) {
return Long.compare(getLongValue(lTag, l, lStart), getLongValue(rTag, r, rStart));
} else if (lTag == INTEGER || lTag == SMALLINT || lTag == TINYINT) {
@@ -103,10 +103,10 @@
byte[] leftBytes = left.getByteArray();
int start = left.getStartOffset();
if (leftTag == DOUBLE || rightTag == DOUBLE) {
- return asResult(Double.compare(getDoubleValue(leftTag, leftBytes, start), getConstantDouble(right)));
+ return asResult(compareDoubles(getDoubleValue(leftTag, leftBytes, start), getConstantDouble(right)));
} else if (leftTag == FLOAT || rightTag == FLOAT) {
return asResult(
- Float.compare((float) getDoubleValue(leftTag, leftBytes, start), (float) getConstantDouble(right)));
+ compareFloats((float) getDoubleValue(leftTag, leftBytes, start), (float) getConstantDouble(right)));
} else if (leftTag == BIGINT || rightTag == BIGINT) {
return asResult(Long.compare(getLongValue(leftTag, leftBytes, start), getConstantLong(right)));
} else if (leftTag == INTEGER || leftTag == SMALLINT || leftTag == TINYINT) {
@@ -122,10 +122,10 @@
ATypeTag leftTag = leftConstant.getType().getTypeTag();
ATypeTag rightTag = rightConstant.getType().getTypeTag();
if (leftTag == DOUBLE || rightTag == DOUBLE) {
- return asResult(Double.compare(getConstantDouble(leftConstant), getConstantDouble(rightConstant)));
+ return asResult(compareDoubles(getConstantDouble(leftConstant), getConstantDouble(rightConstant)));
} else if (leftTag == FLOAT || rightTag == FLOAT) {
return asResult(
- Float.compare((float) getConstantDouble(leftConstant), (float) getConstantDouble(rightConstant)));
+ compareFloats((float) getConstantDouble(leftConstant), (float) getConstantDouble(rightConstant)));
} else if (leftTag == BIGINT || rightTag == BIGINT) {
return asResult(Long.compare(getConstantLong(leftConstant), getConstantLong(rightConstant)));
} else if (leftTag == INTEGER || leftTag == SMALLINT || leftTag == TINYINT) {
@@ -208,4 +208,12 @@
throw new UnsupportedOperationException();
}
}
+
+ public static int compareDoubles(double d1, double d2) {
+ return d1 == d2 ? 0 : Double.compare(d1, d2);
+ }
+
+ public static int compareFloats(float f1, float f2) {
+ return f1 == f2 ? 0 : Float.compare(f1, f2);
+ }
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ABoolean.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ABoolean.java
index 5589205..0664fec 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ABoolean.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ABoolean.java
@@ -50,7 +50,7 @@
@Override
public String toString() {
- return Boolean.toString(bVal).toUpperCase();
+ return Boolean.toString(bVal);
}
@Override
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ARecord.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ARecord.java
index d964079..cb6c5fe 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ARecord.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ARecord.java
@@ -20,6 +20,7 @@
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.utils.RecordUtil;
+import org.apache.hyracks.util.JSONUtil;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.node.ArrayNode;
@@ -89,7 +90,7 @@
if (i > 0) {
sb.append(", ");
}
- sb.append(type.getFieldNames()[i]);
+ JSONUtil.quoteAndEscape(sb, type.getFieldNames()[i]);
sb.append(": ");
sb.append(fields[i]);
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/ExceptionUtil.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/ExceptionUtil.java
index ccb3a8d..daa9d83 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/ExceptionUtil.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/ExceptionUtil.java
@@ -19,6 +19,10 @@
package org.apache.asterix.om.exceptions;
+import static org.apache.asterix.common.exceptions.ErrorCode.FUNCTION_EVALUATION_FAILED;
+import static org.apache.hyracks.api.exceptions.ErrorCode.INVALID_STRING_UNICODE;
+import static org.apache.hyracks.api.util.ExceptionUtils.isErrorCode;
+
import java.util.function.Supplier;
import org.apache.asterix.common.exceptions.ErrorCode;
@@ -26,6 +30,7 @@
import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.exceptions.IWarningCollector;
import org.apache.hyracks.api.exceptions.SourceLocation;
import org.apache.hyracks.api.exceptions.Warning;
@@ -143,6 +148,13 @@
warnInvalidValue(ctx, srcLoc, fid, argIdx, argValue, ErrorCode.NEGATIVE_VALUE);
}
+ public static void warnFunctionEvalFailed(IEvaluatorContext ctx, SourceLocation srcLoc, FunctionIdentifier fid,
+ String errMsg) {
+ if (ctx.getWarningCollector().shouldWarn()) {
+ ctx.getWarningCollector().warn(Warning.of(srcLoc, FUNCTION_EVALUATION_FAILED, fid.getName(), errMsg));
+ }
+ }
+
private static void warnInvalidValue(IEvaluatorContext ctx, SourceLocation srcLoc, FunctionIdentifier fid,
int argIdx, double argValue, ErrorCode errorCode) {
IWarningCollector warningCollector = ctx.getWarningCollector();
@@ -151,4 +163,8 @@
Warning.of(srcLoc, errorCode, fid.getName(), indexToPosition(argIdx), Double.toString(argValue)));
}
}
+
+ public static boolean isStringUnicodeError(HyracksDataException throwable) {
+ return isErrorCode(throwable, INVALID_STRING_UNICODE);
+ }
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
index 0c9f0dd..bb05ba6 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
@@ -95,6 +95,7 @@
import org.apache.asterix.om.typecomputer.impl.InjectFailureTypeComputer;
import org.apache.asterix.om.typecomputer.impl.Int64ArrayToStringTypeComputer;
import org.apache.asterix.om.typecomputer.impl.LocalAvgTypeComputer;
+import org.apache.asterix.om.typecomputer.impl.LocalMedianTypeComputer;
import org.apache.asterix.om.typecomputer.impl.LocalSingleVarStatisticsTypeComputer;
import org.apache.asterix.om.typecomputer.impl.MinMaxAggTypeComputer;
import org.apache.asterix.om.typecomputer.impl.MissingIfTypeComputer;
@@ -551,6 +552,8 @@
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-intermediate-avg", 1);
public static final FunctionIdentifier LOCAL_AVG =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-local-avg", 1);
+ public static final FunctionIdentifier MEDIAN =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-median", 1);
public static final FunctionIdentifier FIRST_ELEMENT =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-first-element", 1);
public static final FunctionIdentifier LOCAL_FIRST_ELEMENT =
@@ -628,6 +631,8 @@
public static final FunctionIdentifier SCALAR_SUM = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "sum", 1);
public static final FunctionIdentifier SCALAR_MAX = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "max", 1);
public static final FunctionIdentifier SCALAR_MIN = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "min", 1);
+ public static final FunctionIdentifier SCALAR_MEDIAN =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "median", 1);
public static final FunctionIdentifier SCALAR_FIRST_ELEMENT =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "first-element", 1);
public static final FunctionIdentifier SCALAR_LOCAL_FIRST_ELEMENT =
@@ -802,6 +807,14 @@
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-global-sql-avg", 1);
public static final FunctionIdentifier LOCAL_SQL_AVG =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-local-sql-avg", 1);
+ public static final FunctionIdentifier SQL_MEDIAN =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-sql-median", 1);
+ public static final FunctionIdentifier LOCAL_SQL_MEDIAN =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-local-sql-median", 1);
+ public static final FunctionIdentifier INTERMEDIATE_SQL_MEDIAN =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-intermediate-sql-median", 1);
+ public static final FunctionIdentifier GLOBAL_SQL_MEDIAN =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-global-sql-median", 1);
public static final FunctionIdentifier SQL_STDDEV_SAMP =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-sql-stddev_samp", 1);
public static final FunctionIdentifier INTERMEDIATE_SQL_STDDEV_SAMP =
@@ -869,6 +882,8 @@
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "sql-max", 1);
public static final FunctionIdentifier SCALAR_SQL_MIN =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "sql-min", 1);
+ public static final FunctionIdentifier SCALAR_SQL_MEDIAN =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "sql-median", 1);
public static final FunctionIdentifier SCALAR_SQL_STDDEV_SAMP =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "sql-stddev_samp", 1);
public static final FunctionIdentifier SCALAR_SQL_STDDEV_POP =
@@ -1699,6 +1714,9 @@
public static final FunctionIdentifier DECODE_DATAVERSE_NAME =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "decode-dataverse-name", 1);
+ public static final FunctionIdentifier SERIALIZED_SIZE =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "serialized-size", 1);
+
static {
// first, take care of Algebricks builtin functions
addFunction(IS_MISSING, BooleanOnlyTypeComputer.INSTANCE, true);
@@ -1799,28 +1817,26 @@
addPrivateFunction(MAKE_FIELD_NAME_HANDLE, AnyTypeComputer.INSTANCE, true);
// cast null type constructors
- addPrivateFunction(BOOLEAN_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_BOOLEAN, true);
- addPrivateFunction(INT8_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_INT8, true);
- addPrivateFunction(INT16_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_INT16, true);
- addPrivateFunction(INT32_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_INT32, true);
- addPrivateFunction(INT64_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_INT64, true);
- addPrivateFunction(FLOAT_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_FLOAT, true);
- addPrivateFunction(DOUBLE_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_DOUBLE, true);
- addPrivateFunction(STRING_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_STRING, true);
- addPrivateFunction(DATE_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_DATE, true);
- addPrivateFunction(DATE_DEFAULT_NULL_CONSTRUCTOR_WITH_FORMAT, NullableTypeComputer.INSTANCE_DATE, true);
- addPrivateFunction(TIME_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_TIME, true);
- addPrivateFunction(TIME_DEFAULT_NULL_CONSTRUCTOR_WITH_FORMAT, NullableTypeComputer.INSTANCE_TIME, true);
- addPrivateFunction(DATETIME_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_DATE_TIME, true);
- addPrivateFunction(DATETIME_DEFAULT_NULL_CONSTRUCTOR_WITH_FORMAT, NullableTypeComputer.INSTANCE_DATE_TIME,
+ addFunction(BOOLEAN_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_BOOLEAN, true);
+ addFunction(INT8_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_INT8, true);
+ addFunction(INT16_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_INT16, true);
+ addFunction(INT32_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_INT32, true);
+ addFunction(INT64_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_INT64, true);
+ addFunction(FLOAT_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_FLOAT, true);
+ addFunction(DOUBLE_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_DOUBLE, true);
+ addFunction(STRING_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_STRING, true);
+ addFunction(DATE_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_DATE, true);
+ addFunction(DATE_DEFAULT_NULL_CONSTRUCTOR_WITH_FORMAT, NullableTypeComputer.INSTANCE_DATE, true);
+ addFunction(TIME_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_TIME, true);
+ addFunction(TIME_DEFAULT_NULL_CONSTRUCTOR_WITH_FORMAT, NullableTypeComputer.INSTANCE_TIME, true);
+ addFunction(DATETIME_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_DATE_TIME, true);
+ addFunction(DATETIME_DEFAULT_NULL_CONSTRUCTOR_WITH_FORMAT, NullableTypeComputer.INSTANCE_DATE_TIME, true);
+ addFunction(DURATION_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_DURATION, true);
+ addFunction(DAY_TIME_DURATION_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_DAY_TIME_DURATION, true);
+ addFunction(YEAR_MONTH_DURATION_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_YEAR_MONTH_DURATION,
true);
- addPrivateFunction(DURATION_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_DURATION, true);
- addPrivateFunction(DAY_TIME_DURATION_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_DAY_TIME_DURATION,
- true);
- addPrivateFunction(YEAR_MONTH_DURATION_DEFAULT_NULL_CONSTRUCTOR,
- NullableTypeComputer.INSTANCE_YEAR_MONTH_DURATION, true);
- addPrivateFunction(UUID_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_UUID, true);
- addPrivateFunction(BINARY_BASE64_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_BINARY, true);
+ addFunction(UUID_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_UUID, true);
+ addFunction(BINARY_BASE64_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_BINARY, true);
addPrivateFunction(NUMERIC_UNARY_MINUS, NumericUnaryTypeComputer.INSTANCE, true);
addPrivateFunction(NUMERIC_SUBTRACT, NumericAddSubMulDivTypeComputer.INSTANCE_SUB, true);
@@ -2142,6 +2158,13 @@
addPrivateFunction(GLOBAL_SQL_UNION_MBR, ARectangleTypeComputer.INSTANCE, true);
addFunction(SCALAR_SQL_UNION_MBR, ARectangleTypeComputer.INSTANCE, true);
+ addFunction(SQL_MEDIAN, NullableDoubleTypeComputer.INSTANCE, true);
+ addFunction(SCALAR_MEDIAN, NullableDoubleTypeComputer.INSTANCE, true);
+ addFunction(SCALAR_SQL_MEDIAN, NullableDoubleTypeComputer.INSTANCE, true);
+ addPrivateFunction(LOCAL_SQL_MEDIAN, LocalMedianTypeComputer.INSTANCE, true);
+ addPrivateFunction(INTERMEDIATE_SQL_MEDIAN, LocalMedianTypeComputer.INSTANCE, true);
+ addPrivateFunction(GLOBAL_SQL_MEDIAN, NullableDoubleTypeComputer.INSTANCE, true);
+
addPrivateFunction(SERIAL_AVG, NullableDoubleTypeComputer.INSTANCE, true);
addPrivateFunction(SERIAL_COUNT, AInt64TypeComputer.INSTANCE, true);
addPrivateFunction(SERIAL_GLOBAL_AVG, NullableDoubleTypeComputer.INSTANCE, true);
@@ -2532,6 +2555,7 @@
addFunction(DECODE_DATAVERSE_NAME, OrderedListOfAStringTypeComputer.INSTANCE_NULLABLE, true);
addPrivateFunction(COLLECTION_TO_SEQUENCE, CollectionToSequenceTypeComputer.INSTANCE, true);
+ addPrivateFunction(SERIALIZED_SIZE, AInt64TypeComputer.INSTANCE, true);
// external lookup
addPrivateFunction(EXTERNAL_LOOKUP, AnyTypeComputer.INSTANCE, false);
@@ -3167,6 +3191,25 @@
addDistinctAgg(SQL_SUM_DISTINCT, SQL_SUM);
addScalarAgg(SQL_SUM_DISTINCT, SCALAR_SQL_SUM_DISTINCT);
+ // SQL MEDIAN
+ addAgg(SQL_MEDIAN);
+ addAgg(LOCAL_SQL_MEDIAN);
+ addAgg(GLOBAL_SQL_MEDIAN);
+
+ addLocalAgg(SQL_MEDIAN, LOCAL_SQL_MEDIAN);
+
+ addIntermediateAgg(SQL_MEDIAN, INTERMEDIATE_SQL_MEDIAN);
+ addIntermediateAgg(LOCAL_SQL_MEDIAN, INTERMEDIATE_SQL_MEDIAN);
+ addIntermediateAgg(GLOBAL_SQL_MEDIAN, INTERMEDIATE_SQL_MEDIAN);
+
+ addGlobalAgg(SQL_MEDIAN, GLOBAL_SQL_MEDIAN);
+
+ addScalarAgg(MEDIAN, SCALAR_MEDIAN);
+ addScalarAgg(SQL_MEDIAN, SCALAR_SQL_MEDIAN);
+
+ registerAggFunctionProperties(SCALAR_SQL_MEDIAN, AggregateFunctionProperty.NO_FRAME_CLAUSE,
+ AggregateFunctionProperty.NO_ORDER_CLAUSE);
+
// SPATIAL AGGREGATES
addAgg(ST_UNION_AGG);
@@ -3201,6 +3244,13 @@
interface BuiltinFunctionProperty {
}
+ public enum AggregateFunctionProperty implements BuiltinFunctionProperty {
+ /** Whether the order clause is prohibited */
+ NO_ORDER_CLAUSE,
+ /** Whether the frame clause is prohibited */
+ NO_FRAME_CLAUSE
+ }
+
public enum WindowFunctionProperty implements BuiltinFunctionProperty {
/** Whether the order clause is prohibited */
NO_ORDER_CLAUSE,
@@ -3363,6 +3413,11 @@
registeredFunctions.put(functionInfo.getFunctionIdentifier(), functionInfo);
}
+ private static <T extends Enum<T> & BuiltinFunctionProperty> void registerAggFunctionProperties(
+ FunctionIdentifier fid, AggregateFunctionProperty... properties) {
+ registerFunctionProperties(fid, AggregateFunctionProperty.class, properties);
+ }
+
private static <T extends Enum<T> & BuiltinFunctionProperty> void registerFunctionProperties(FunctionIdentifier fid,
Class<T> propertyClass, T[] properties) {
if (properties == null) {
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/LocalMedianTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/LocalMedianTypeComputer.java
new file mode 100644
index 0000000..3411ef4
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/LocalMedianTypeComputer.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.om.typecomputer.impl;
+
+import org.apache.asterix.om.typecomputer.base.IResultTypeComputer;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+
+public class LocalMedianTypeComputer implements IResultTypeComputer {
+
+ public static final LocalMedianTypeComputer INSTANCE = new LocalMedianTypeComputer();
+
+ public static final ARecordType REC_TYPE = new ARecordType(null,
+ new String[] { "count", "handle", "address", "port" },
+ new IAType[] { BuiltinType.AINT64, BuiltinType.AINT64, BuiltinType.ASTRING, BuiltinType.AINT32 }, false);
+
+ @Override
+ public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
+ IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
+ return REC_TYPE;
+ }
+}
diff --git a/asterixdb/asterix-om/src/test/java/org/apache/asterix/test/om/typecomputer/TypeComputerTest.java b/asterixdb/asterix-om/src/test/java/org/apache/asterix/test/om/typecomputer/TypeComputerTest.java
index a857981..07f2021 100644
--- a/asterixdb/asterix-om/src/test/java/org/apache/asterix/test/om/typecomputer/TypeComputerTest.java
+++ b/asterixdb/asterix-om/src/test/java/org/apache/asterix/test/om/typecomputer/TypeComputerTest.java
@@ -36,6 +36,7 @@
import org.apache.asterix.om.typecomputer.impl.ClosedRecordConstructorResultType;
import org.apache.asterix.om.typecomputer.impl.InjectFailureTypeComputer;
import org.apache.asterix.om.typecomputer.impl.LocalAvgTypeComputer;
+import org.apache.asterix.om.typecomputer.impl.LocalMedianTypeComputer;
import org.apache.asterix.om.typecomputer.impl.LocalSingleVarStatisticsTypeComputer;
import org.apache.asterix.om.typecomputer.impl.NullableDoubleTypeComputer;
import org.apache.asterix.om.typecomputer.impl.OpenRecordConstructorResultType;
@@ -190,6 +191,7 @@
differentBehaviorFunctions.add(RecordRemoveFieldsTypeComputer.class.getSimpleName());
differentBehaviorFunctions.add(ClosedRecordConstructorResultType.class.getSimpleName());
differentBehaviorFunctions.add(LocalAvgTypeComputer.class.getSimpleName());
+ differentBehaviorFunctions.add(LocalMedianTypeComputer.class.getSimpleName());
differentBehaviorFunctions.add(BooleanOnlyTypeComputer.class.getSimpleName());
// differentBehaviorFunctions.add("AMissingTypeComputer"); // TODO What type computer is this?
differentBehaviorFunctions.add(NullableDoubleTypeComputer.class.getSimpleName());
diff --git a/asterixdb/asterix-replication/pom.xml b/asterixdb/asterix-replication/pom.xml
index 7e9b4f8..82fb3e3 100644
--- a/asterixdb/asterix-replication/pom.xml
+++ b/asterixdb/asterix-replication/pom.xml
@@ -3,7 +3,7 @@
<parent>
<groupId>org.apache.asterix</groupId>
<artifactId>apache-asterixdb</artifactId>
- <version>0.9.8-SNAPSHOT</version>
+ <version>0.9.8.2-SNAPSHOT</version>
</parent>
<artifactId>asterix-replication</artifactId>
<licenses>
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/ReplicationDestination.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/ReplicationDestination.java
index d803756..f7a739b 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/ReplicationDestination.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/ReplicationDestination.java
@@ -20,7 +20,10 @@
import java.io.IOException;
import java.net.InetSocketAddress;
+import java.util.ArrayDeque;
+import java.util.HashMap;
import java.util.HashSet;
+import java.util.Map;
import java.util.Objects;
import java.util.Optional;
import java.util.Set;
@@ -29,6 +32,7 @@
import org.apache.asterix.common.exceptions.ReplicationException;
import org.apache.asterix.common.replication.IPartitionReplica;
import org.apache.asterix.common.replication.IReplicationDestination;
+import org.apache.asterix.common.storage.ReplicaIdentifier;
import org.apache.asterix.replication.management.NetworkingUtil;
import org.apache.asterix.replication.messaging.ReplicationProtocol;
import org.apache.hyracks.api.network.ISocketChannel;
@@ -41,6 +45,7 @@
private static final Logger LOGGER = LogManager.getLogger();
private final Set<IPartitionReplica> replicas = new HashSet<>();
private final InetSocketAddress inputLocation;
+ private final Map<ReplicaIdentifier, ArrayDeque<PartitionReplica>> replicasConnPool = new HashMap<>();
private InetSocketAddress resolvedLocation;
private ISocketChannel logRepChannel;
@@ -64,6 +69,11 @@
@Override
public synchronized void remove(IPartitionReplica replica) {
replicas.remove(replica);
+ ArrayDeque<PartitionReplica> partitionConnections = replicasConnPool.remove(replica.getIdentifier());
+ if (partitionConnections != null) {
+ partitionConnections.forEach(PartitionReplica::close);
+ partitionConnections.clear();
+ }
}
@Override
@@ -138,4 +148,26 @@
public int hashCode() {
return Objects.hash(inputLocation);
}
+
+ public synchronized PartitionReplica getPartitionReplicaConnection(ReplicaIdentifier identifier,
+ INcApplicationContext appCtx) {
+ ArrayDeque<PartitionReplica> partitionReplicas =
+ replicasConnPool.computeIfAbsent(identifier, k -> new ArrayDeque<>());
+ if (!partitionReplicas.isEmpty()) {
+ return partitionReplicas.remove();
+ }
+ return new PartitionReplica(identifier, appCtx);
+ }
+
+ public synchronized void recycleConnection(PartitionReplica partitionReplica) {
+ ArrayDeque<PartitionReplica> partitionReplicas = replicasConnPool.get(partitionReplica.getIdentifier());
+ if (partitionReplicas != null) {
+ partitionReplicas.add(partitionReplica);
+ }
+ }
+
+ public synchronized void closeConnections() {
+ replicasConnPool
+ .forEach(((identifier, partitionReplicas) -> partitionReplicas.forEach(PartitionReplica::close)));
+ }
}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/RemoteLogsNotifier.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/RemoteLogsNotifier.java
index 80bb3c8..42a3c0b 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/RemoteLogsNotifier.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/RemoteLogsNotifier.java
@@ -96,7 +96,9 @@
for (DatasetResourceReference replicaIndexRef : replicaIndexesRef) {
final IIndexCheckpointManager indexCheckpointManager = indexCheckpointManagerProvider.get(replicaIndexRef);
synchronized (indexCheckpointManager) {
- indexCheckpointManager.masterFlush(remoteLogMapping.getMasterLsn(), remoteLogMapping.getLSN());
+ if (indexCheckpointManager.isValidIndex()) {
+ indexCheckpointManager.masterFlush(remoteLogMapping.getMasterLsn(), remoteLogMapping.getLSN());
+ }
}
}
}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/IndexReplicationManager.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/IndexReplicationManager.java
index 063709a..8c514bd 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/IndexReplicationManager.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/IndexReplicationManager.java
@@ -21,22 +21,21 @@
import java.util.HashSet;
import java.util.Optional;
import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicInteger;
import org.apache.asterix.common.api.INcApplicationContext;
import org.apache.asterix.common.exceptions.ReplicationException;
-import org.apache.asterix.common.replication.IPartitionReplica;
import org.apache.asterix.common.replication.IReplicationDestination;
import org.apache.asterix.common.replication.IReplicationManager;
import org.apache.asterix.common.replication.IReplicationStrategy;
import org.apache.asterix.common.storage.DatasetResourceReference;
-import org.apache.asterix.common.storage.ResourceReference;
-import org.apache.asterix.replication.api.PartitionReplica;
import org.apache.asterix.replication.api.ReplicationDestination;
-import org.apache.asterix.replication.sync.IndexSynchronizer;
import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.replication.IReplicationJob;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexReplicationJob;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
@@ -45,13 +44,15 @@
private static final Logger LOGGER = LogManager.getLogger();
private final IReplicationManager replicationManager;
- private final Set<ReplicationDestination> destinations = new HashSet<>();
+ private final Set<ReplicationDestination> destinations = ConcurrentHashMap.newKeySet();
private final LinkedBlockingQueue<IReplicationJob> replicationJobsQ = new LinkedBlockingQueue<>();
private final IReplicationStrategy replicationStrategy;
private final PersistentLocalResourceRepository resourceRepository;
private final INcApplicationContext appCtx;
+ private final ILSMIOOperationScheduler ioScheduler;
private final Object transferLock = new Object();
private final Set<ReplicationDestination> failedDest = new HashSet<>();
+ private final AtomicInteger pendingRepOpsCount = new AtomicInteger();
public IndexReplicationManager(INcApplicationContext appCtx, IReplicationManager replicationManager) {
this.appCtx = appCtx;
@@ -59,6 +60,8 @@
this.resourceRepository = (PersistentLocalResourceRepository) appCtx.getLocalResourceRepository();
replicationStrategy = replicationManager.getReplicationStrategy();
appCtx.getThreadExecutor().execute(new ReplicationJobsProcessor());
+ ioScheduler = appCtx.getStorageComponentProvider().getIoOperationSchedulerProvider()
+ .getIoScheduler(appCtx.getServiceContext());
}
public void register(ReplicationDestination dest) {
@@ -72,12 +75,18 @@
public void unregister(IReplicationDestination dest) {
synchronized (transferLock) {
LOGGER.info(() -> "unregister " + dest);
+ for (ReplicationDestination existingDest : destinations) {
+ if (existingDest.equals(dest)) {
+ existingDest.closeConnections();
+ break;
+ }
+ }
destinations.remove(dest);
failedDest.remove(dest);
}
}
- private void handleFailure(ReplicationDestination dest, Exception e) {
+ public void handleFailure(ReplicationDestination dest, Exception e) {
synchronized (transferLock) {
if (failedDest.contains(dest)) {
return;
@@ -87,6 +96,7 @@
LOGGER.error("replica at {} failed", dest);
failedDest.add(dest);
}
+ dest.closeConnections();
replicationManager.notifyFailure(dest, e);
}
}
@@ -99,71 +109,62 @@
process(job);
}
- private void process(IReplicationJob job) {
- try {
- if (skip(job)) {
- return;
- }
- synchronized (transferLock) {
- if (destinations.isEmpty()) {
- return;
- }
- final IndexSynchronizer synchronizer = new IndexSynchronizer(job, appCtx);
- final int indexPartition = getJobPartition(job);
- for (ReplicationDestination dest : destinations) {
- try {
- Optional<IPartitionReplica> partitionReplica = dest.getPartitionReplica(indexPartition);
- if (!partitionReplica.isPresent()) {
- continue;
- }
- PartitionReplica replica = (PartitionReplica) partitionReplica.get();
- synchronizer.sync(replica);
- } catch (Exception e) {
- handleFailure(dest, e);
- }
- }
- closeChannels();
- }
- } finally {
- afterReplication(job);
+ public Set<ReplicationDestination> getDestinations() {
+ synchronized (transferLock) {
+ return destinations;
}
}
- private boolean skip(IReplicationJob job) {
- try {
- final String fileToReplicate = job.getAnyFile();
- final Optional<DatasetResourceReference> indexFileRefOpt =
- resourceRepository.getLocalResourceReference(fileToReplicate);
- if (!indexFileRefOpt.isPresent()) {
- LOGGER.warn("skipping replication of {} due to missing dataset resource reference", fileToReplicate);
- return true;
+ private void process(IReplicationJob job) {
+ pendingRepOpsCount.incrementAndGet();
+ Optional<DatasetResourceReference> jobIndexRefOpt = getJobIndexRef(job);
+ if (jobIndexRefOpt.isEmpty()) {
+ LOGGER.warn("skipping replication of {} due to missing dataset resource reference", job.getAnyFile());
+ afterReplication(job);
+ return;
+ }
+ ReplicationOperation rp = new ReplicationOperation(appCtx, jobIndexRefOpt.get(), job, this);
+ if (job.getExecutionType() == IReplicationJob.ReplicationExecutionType.SYNC) {
+ rp.call();
+ } else {
+ try {
+ ioScheduler.scheduleOperation(rp);
+ } catch (HyracksDataException e) {
+ throw new ReplicationException(e);
}
- return !replicationStrategy.isMatch(indexFileRefOpt.get().getDatasetId());
+ }
+ }
+
+ public boolean skip(DatasetResourceReference indexRef) {
+ return !replicationStrategy.isMatch(indexRef.getDatasetId());
+ }
+
+ public Optional<DatasetResourceReference> getJobIndexRef(IReplicationJob job) {
+ final String fileToReplicate = job.getAnyFile();
+ try {
+ return resourceRepository.getLocalResourceReference(fileToReplicate);
} catch (HyracksDataException e) {
throw new IllegalStateException("Couldn't find resource for " + job.getAnyFile(), e);
}
}
- private int getJobPartition(IReplicationJob job) {
- return ResourceReference.of(job.getAnyFile()).getPartitionNum();
- }
-
private void closeChannels() {
- if (!replicationJobsQ.isEmpty()) {
- return;
- }
LOGGER.trace("no pending replication jobs; closing connections to replicas");
for (ReplicationDestination dest : destinations) {
- dest.getReplicas().stream().map(PartitionReplica.class::cast).forEach(PartitionReplica::close);
+ dest.closeConnections();
}
}
- private static void afterReplication(IReplicationJob job) {
+ public void afterReplication(IReplicationJob job) {
try {
+ int pendingOps = pendingRepOpsCount.decrementAndGet();
if (job.getOperation() == IReplicationJob.ReplicationOperation.REPLICATE
&& job instanceof ILSMIndexReplicationJob) {
((ILSMIndexReplicationJob) job).endReplication();
}
+ if (pendingOps == 0 && replicationJobsQ.isEmpty()) {
+ closeChannels();
+ }
} catch (HyracksDataException e) {
throw new ReplicationException(e);
}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationOperation.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationOperation.java
new file mode 100644
index 0000000..258f24a
--- /dev/null
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationOperation.java
@@ -0,0 +1,106 @@
+/*
+ * 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.replication.management;
+
+import java.util.Optional;
+import java.util.Set;
+
+import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.common.replication.IPartitionReplica;
+import org.apache.asterix.common.storage.DatasetResourceReference;
+import org.apache.asterix.replication.api.PartitionReplica;
+import org.apache.asterix.replication.api.ReplicationDestination;
+import org.apache.asterix.replication.sync.IndexSynchronizer;
+import org.apache.hyracks.api.replication.IReplicationJob;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractIoOperation;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
+import org.apache.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallbackFactory;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+public class ReplicationOperation extends AbstractIoOperation {
+
+ private static final Logger LOGGER = LogManager.getLogger();
+
+ private static final ILSMIOOperationCallback INSTANCE =
+ NoOpIOOperationCallbackFactory.INSTANCE.createIoOpCallback(null);
+ private final INcApplicationContext appCtx;
+ private final DatasetResourceReference indexRef;
+ private final IReplicationJob job;
+ private final IndexReplicationManager indexReplicationManager;
+
+ public ReplicationOperation(INcApplicationContext appCtx, DatasetResourceReference indexRef, IReplicationJob job,
+ IndexReplicationManager indexReplicationManager) {
+ super(null, null, INSTANCE, indexRef.getRelativePath().toString());
+ this.appCtx = appCtx;
+ this.indexRef = indexRef;
+ this.job = job;
+ this.indexReplicationManager = indexReplicationManager;
+ }
+
+ @Override
+ public LSMIOOperationType getIOOpertionType() {
+ return LSMIOOperationType.REPLICATE;
+ }
+
+ @Override
+ public LSMIOOperationStatus call() {
+ try {
+ Set<ReplicationDestination> destinations = indexReplicationManager.getDestinations();
+ if (destinations.isEmpty() || indexReplicationManager.skip(indexRef)) {
+ return LSMIOOperationStatus.SUCCESS;
+ }
+ LOGGER.debug("started replicate operation on index {}", indexRef);
+ final IndexSynchronizer synchronizer = new IndexSynchronizer(job, appCtx);
+ final int indexPartition = indexRef.getPartitionId();
+ for (ReplicationDestination dest : destinations) {
+ Optional<IPartitionReplica> partitionReplica = dest.getPartitionReplica(indexPartition);
+ if (partitionReplica.isEmpty()) {
+ continue;
+ }
+ PartitionReplica destReplica = null;
+ try {
+ destReplica = dest.getPartitionReplicaConnection(partitionReplica.get().getIdentifier(), appCtx);
+ synchronizer.sync(destReplica);
+ dest.recycleConnection(destReplica);
+ } catch (Exception e) {
+ if (destReplica != null) {
+ destReplica.close();
+ }
+ indexReplicationManager.handleFailure(dest, e);
+ }
+ }
+ LOGGER.debug("completed replicate operation on index {}", indexRef);
+ return LSMIOOperationStatus.SUCCESS;
+ } finally {
+ indexReplicationManager.afterReplication(job);
+ }
+ }
+
+ @Override
+ protected LSMComponentFileReferences getComponentFiles() {
+ return null;
+ }
+
+ @Override
+ public long getRemainingPages() {
+ return 0;
+ }
+}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/PartitionResourcesListTask.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/PartitionResourcesListTask.java
index 0f5949e..97d03e6 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/PartitionResourcesListTask.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/PartitionResourcesListTask.java
@@ -51,6 +51,11 @@
@Override
public void perform(INcApplicationContext appCtx, IReplicationWorker worker) throws HyracksDataException {
LOGGER.debug("processing {}", this);
+ if (appCtx.getMetadataPartitionId().isPresent() && appCtx.getMetadataPartitionId().getAsInt() == partition
+ && appCtx.getReplicaManager().getPartitions().contains(partition)) {
+ LOGGER.warn("received request to get metadata files from non-master {}", worker.getRemoteAddress());
+ throw new IllegalStateException();
+ }
final PersistentLocalResourceRepository localResourceRepository =
(PersistentLocalResourceRepository) appCtx.getLocalResourceRepository();
localResourceRepository.cleanup(partition);
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/ReplicaSynchronizer.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/ReplicaSynchronizer.java
index 459ff01..68ccd54 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/ReplicaSynchronizer.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/ReplicaSynchronizer.java
@@ -103,6 +103,6 @@
private void waitForReplicatedDatasetsIO() throws HyracksDataException {
// wait for IO operations to ensure replicated datasets files won't change during replica sync
final IReplicationStrategy replStrategy = appCtx.getReplicationManager().getReplicationStrategy();
- appCtx.getDatasetLifecycleManager().waitForIO(replStrategy);
+ appCtx.getDatasetLifecycleManager().waitForIO(replStrategy, replica.getIdentifier().getPartition());
}
}
diff --git a/asterixdb/asterix-runtime/pom.xml b/asterixdb/asterix-runtime/pom.xml
index 22958ac..870bbcc 100644
--- a/asterixdb/asterix-runtime/pom.xml
+++ b/asterixdb/asterix-runtime/pom.xml
@@ -21,7 +21,7 @@
<parent>
<artifactId>apache-asterixdb</artifactId>
<groupId>org.apache.asterix</groupId>
- <version>0.9.8-SNAPSHOT</version>
+ <version>0.9.8.2-SNAPSHOT</version>
</parent>
<artifactId>asterix-runtime</artifactId>
<properties>
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/FirstElementEvalFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/FirstElementEvalFactory.java
index b787e26..6c5c0c7 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/FirstElementEvalFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/FirstElementEvalFactory.java
@@ -32,14 +32,14 @@
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
-class FirstElementEvalFactory implements IAggregateEvaluatorFactory {
+public class FirstElementEvalFactory implements IAggregateEvaluatorFactory {
private static final long serialVersionUID = 1L;
private final IScalarEvaluatorFactory[] args;
private final boolean isLocal;
private final SourceLocation sourceLoc;
- FirstElementEvalFactory(IScalarEvaluatorFactory[] args, boolean isLocal, SourceLocation sourceLoc) {
+ public FirstElementEvalFactory(IScalarEvaluatorFactory[] args, boolean isLocal, SourceLocation sourceLoc) {
this.args = args;
this.isLocal = isLocal;
this.sourceLoc = sourceLoc;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/AbstractScalarAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/AbstractScalarAggregateDescriptor.java
index 064c861..2fd065d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/AbstractScalarAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/AbstractScalarAggregateDescriptor.java
@@ -80,7 +80,7 @@
public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
// Use ScanCollection to iterate over list items.
ScanCollectionUnnestingFunctionFactory scanCollectionFactory =
- new ScanCollectionUnnestingFunctionFactory(args[0], sourceLoc);
+ new ScanCollectionUnnestingFunctionFactory(args[0], sourceLoc, getIdentifier());
return createScalarAggregateEvaluator(aggFuncFactory.createAggregateEvaluator(ctx),
scanCollectionFactory, ctx);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlMedianAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlMedianAggregateDescriptor.java
new file mode 100644
index 0000000..5c83a5e
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlMedianAggregateDescriptor.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.aggregates.scalar;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.runtime.aggregates.std.SqlMedianAggregateDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+
+public class ScalarSqlMedianAggregateDescriptor extends AbstractScalarAggregateDescriptor {
+
+ private static final long serialVersionUID = 1L;
+
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new ScalarSqlMedianAggregateDescriptor();
+ }
+
+ @Override
+ public IFunctionTypeInferer createFunctionTypeInferer() {
+ return FunctionTypeInferers.MEDIAN_MEMORY;
+ }
+ };
+
+ private ScalarSqlMedianAggregateDescriptor() {
+ super(SqlMedianAggregateDescriptor.FACTORY);
+ }
+
+ @Override
+ public void setImmutableStates(Object... states) {
+ super.setImmutableStates(states);
+ aggFuncDesc.setImmutableStates(states);
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.SCALAR_SQL_MEDIAN;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractLocalMedianAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractLocalMedianAggregateFunction.java
new file mode 100644
index 0000000..8ff7ed5
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractLocalMedianAggregateFunction.java
@@ -0,0 +1,176 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.aggregates.std;
+
+import java.util.List;
+
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.comm.IFrameTupleAppender;
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.comm.NetworkAddress;
+import org.apache.hyracks.api.context.IHyracksJobletContext;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputer;
+import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.control.nc.NodeControllerService;
+import org.apache.hyracks.control.nc.partitions.JobFileState;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+import org.apache.hyracks.dataflow.common.io.GeneratedRunFileReader;
+import org.apache.hyracks.dataflow.common.io.RunFileWriter;
+import org.apache.hyracks.dataflow.std.buffermanager.EnumFreeSlotPolicy;
+import org.apache.hyracks.dataflow.std.sort.Algorithm;
+import org.apache.hyracks.dataflow.std.sort.ExternalSortRunGenerator;
+import org.apache.hyracks.dataflow.std.sort.ExternalSortRunMerger;
+import org.apache.hyracks.dataflow.std.sort.ISorter;
+
+public abstract class AbstractLocalMedianAggregateFunction extends AbstractMedianAggregateFunction {
+
+ protected final ExternalSortRunGenerator runsGenerator;
+ protected final IFrameTupleAppender appender;
+ private final ArrayTupleBuilder tupleBuilder;
+ private final int numFrames;
+ private ExternalSortRunMerger runsMerger;
+
+ public AbstractLocalMedianAggregateFunction(IScalarEvaluatorFactory[] args, IEvaluatorContext context,
+ SourceLocation sourceLoc, int numFrames) throws HyracksDataException {
+ super(args, context, sourceLoc);
+ this.numFrames = numFrames;
+ appender = new FrameTupleAppender(frame);
+ tupleBuilder = new ArrayTupleBuilder(1);
+ runsGenerator = new ExternalSortRunGenerator(context.getTaskContext(), new int[] { 0 },
+ new INormalizedKeyComputerFactory[] { doubleNkComputerFactory },
+ new IBinaryComparatorFactory[] { doubleComparatorFactory }, recordDesc, Algorithm.MERGE_SORT,
+ EnumFreeSlotPolicy.LAST_FIT, numFrames, Integer.MAX_VALUE);
+
+ }
+
+ @Override
+ public void init() throws HyracksDataException {
+ super.init();
+ appender.reset(frame, true);
+ runsGenerator.open();
+ runsGenerator.getSorter().reset();
+ }
+
+ protected void processDataValue(IFrameTupleReference tuple) throws HyracksDataException {
+ eval.evaluate(tuple, inputVal);
+ byte[] data = inputVal.getByteArray();
+ int start = inputVal.getStartOffset();
+ ATypeTag tag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(data[start]);
+ if (ATypeHierarchy.getTypeDomain(tag) == ATypeHierarchy.Domain.NUMERIC) {
+ count++;
+ aDouble.setValue(ATypeHierarchy.getDoubleValue(MEDIAN, 0, data, start));
+ tupleBuilder.reset();
+ tupleBuilder.addField(doubleSerde, aDouble);
+ FrameUtils.appendToWriter(runsGenerator, appender, tupleBuilder.getFieldEndOffsets(),
+ tupleBuilder.getByteArray(), 0, tupleBuilder.getSize());
+ }
+ }
+
+ protected void finishLocalPartial(IPointable result) throws HyracksDataException {
+ if (count == 0) {
+ setPartialResult(result, -1, "", -1);
+ return;
+ }
+ if (appender.getTupleCount() > 0) {
+ appender.write(runsGenerator, true);
+ }
+ // close to sort the in-memory data or write out sorted data to run files
+ runsGenerator.close();
+
+ IHyracksTaskContext taskCtx = ctx.getTaskContext();
+ IHyracksJobletContext jobletCtx = taskCtx.getJobletContext();
+ NetworkAddress netAddress = ((NodeControllerService) jobletCtx.getServiceContext().getControllerService())
+ .getNetworkManager().getPublicNetworkAddress();
+ FileReference fileRef = writeRunFile(taskCtx, jobletCtx);
+ long fileId = jobletCtx.nextUniqueId();
+ taskCtx.setStateObject(new JobFileState(fileRef, jobletCtx.getJobId(), fileId));
+ setPartialResult(result, fileId, netAddress.getAddress(), netAddress.getPort());
+ }
+
+ private FileReference writeRunFile(IHyracksTaskContext taskCtx, IHyracksJobletContext jobletCtx)
+ throws HyracksDataException {
+ List<GeneratedRunFileReader> runs = runsGenerator.getRuns();
+ FileReference managedFile;
+ if (runs.isEmpty()) {
+ managedFile = jobletCtx.createManagedWorkspaceFile(MEDIAN);
+ writeMemoryDataToRunFile(managedFile, taskCtx);
+ } else if (runs.size() == 1) {
+ managedFile = runs.get(0).getFile();
+ } else {
+ managedFile = jobletCtx.createManagedWorkspaceFile(MEDIAN);
+ mergeRunsToRunFile(managedFile, taskCtx, runs);
+ }
+ return managedFile;
+ }
+
+ private void mergeRunsToRunFile(FileReference managedFile, IHyracksTaskContext taskCtx,
+ List<GeneratedRunFileReader> runs) throws HyracksDataException {
+ createOrResetRunsMerger(runs);
+ RunFileWriter runFileWriter = new RunFileWriter(managedFile, taskCtx.getIoManager());
+ IFrameWriter wrappingWriter = runsMerger.prepareFinalMergeResultWriter(runFileWriter);
+ try {
+ wrappingWriter.open();
+ runsMerger.process(wrappingWriter);
+ } finally {
+ wrappingWriter.close();
+ }
+ }
+
+ protected RunFileWriter writeMemoryDataToRunFile(FileReference managedFile, IHyracksTaskContext taskCtx)
+ throws HyracksDataException {
+ RunFileWriter runFileWriter = new RunFileWriter(managedFile, taskCtx.getIoManager());
+ try {
+ runFileWriter.open();
+ ISorter sorter = runsGenerator.getSorter();
+ if (sorter.hasRemaining()) {
+ sorter.flush(runFileWriter);
+ }
+ } finally {
+ runFileWriter.close();
+ }
+ return runFileWriter;
+ }
+
+ private void createOrResetRunsMerger(List<GeneratedRunFileReader> runs) {
+ if (runsMerger == null) {
+ IBinaryComparator[] comparators =
+ new IBinaryComparator[] { doubleComparatorFactory.createBinaryComparator() };
+ INormalizedKeyComputer nmkComputer = doubleNkComputerFactory.createNormalizedKeyComputer();
+ runsMerger = new ExternalSortRunMerger(ctx.getTaskContext(), runs, new int[] { 0 }, comparators,
+ nmkComputer, recordDesc, numFrames, Integer.MAX_VALUE);
+ } else {
+ runsMerger.reset(runs);
+ }
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractMedianAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractMedianAggregateFunction.java
new file mode 100644
index 0000000..207eaa3
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractMedianAggregateFunction.java
@@ -0,0 +1,380 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.aggregates.std;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.asterix.builders.RecordBuilder;
+import org.apache.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.ARecordSerializerDeserializer;
+import org.apache.asterix.formats.nontagged.BinaryComparatorFactoryProvider;
+import org.apache.asterix.formats.nontagged.NormalizedKeyComputerFactoryProvider;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.formats.nontagged.TypeTraitProvider;
+import org.apache.asterix.om.base.ADouble;
+import org.apache.asterix.om.base.AInt32;
+import org.apache.asterix.om.base.AInt64;
+import org.apache.asterix.om.base.AMutableDouble;
+import org.apache.asterix.om.base.AMutableInt32;
+import org.apache.asterix.om.base.AMutableInt64;
+import org.apache.asterix.om.base.AMutableString;
+import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.typecomputer.impl.LocalMedianTypeComputer;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.asterix.runtime.exceptions.UnsupportedItemTypeException;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.application.INCServiceContext;
+import org.apache.hyracks.api.comm.IFrame;
+import org.apache.hyracks.api.comm.IFrameReader;
+import org.apache.hyracks.api.comm.NetworkAddress;
+import org.apache.hyracks.api.comm.VSizeFrame;
+import org.apache.hyracks.api.context.IHyracksJobletContext;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.comm.channels.FileNetworkInputChannel;
+import org.apache.hyracks.control.nc.NodeControllerService;
+import org.apache.hyracks.control.nc.net.NetworkManager;
+import org.apache.hyracks.control.nc.partitions.JobFileState;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import org.apache.hyracks.dataflow.common.data.accessors.FrameTupleReference;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+import org.apache.hyracks.dataflow.common.io.RunFileWriter;
+import org.apache.hyracks.dataflow.std.collectors.InputChannelFrameReader;
+import org.apache.hyracks.dataflow.std.sort.RunMergingFrameReader;
+import org.apache.hyracks.util.string.UTF8StringUtil;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+public abstract class AbstractMedianAggregateFunction extends AbstractAggregateFunction {
+
+ private static final Logger LOGGER = LogManager.getLogger();
+
+ protected static final String MEDIAN = "median";
+ private static final int COUNT_FIELD_ID = 0;
+ private static final int HANDLE_FIELD_ID = 1;
+ private static final int ADDRESS_FIELD_ID = 2;
+ private static final int PORT_FIELD_ID = 3;
+
+ private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+ @SuppressWarnings("unchecked")
+ private final ISerializerDeserializer<AString> stringSerde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING);
+ @SuppressWarnings("unchecked")
+ private final ISerializerDeserializer<AInt64> longSerde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT64);
+ @SuppressWarnings("unchecked")
+ private final ISerializerDeserializer<AInt32> intSerde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32);
+ @SuppressWarnings("unchecked")
+ protected final ISerializerDeserializer<ADouble> doubleSerde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADOUBLE);
+ protected final IBinaryComparatorFactory doubleComparatorFactory =
+ BinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(ATypeTag.DOUBLE, true);
+ protected final INormalizedKeyComputerFactory doubleNkComputerFactory =
+ NormalizedKeyComputerFactoryProvider.INSTANCE.getNormalizedKeyComputerFactory(BuiltinType.ADOUBLE, true);
+ protected final RecordDescriptor recordDesc = new RecordDescriptor(new ISerializerDeserializer[] { doubleSerde },
+ new ITypeTraits[] { TypeTraitProvider.INSTANCE.getTypeTrait(BuiltinType.ADOUBLE) });
+
+ protected final AMutableString aString = new AMutableString("");
+ protected final AMutableInt64 aInt64 = new AMutableInt64(0);
+ protected final AMutableInt32 aInt32 = new AMutableInt32(0);
+ protected final AMutableDouble aDouble = new AMutableDouble(0);
+ protected final IPointable inputVal = new VoidPointable();
+ private final FrameTupleReference ftr = new FrameTupleReference();
+ private final FrameTupleAccessor fta = new FrameTupleAccessor(recordDesc);
+ protected final List<IFrameReader> readers = new ArrayList<>();
+
+ protected final IScalarEvaluator eval;
+ protected final IEvaluatorContext ctx;
+ protected final IFrame frame;
+ protected long count;
+ private List<IFrame> inFrames;
+ private List<PartialResult> partialResults;
+ private RecordBuilder recBuilder;
+
+ public AbstractMedianAggregateFunction(IScalarEvaluatorFactory[] args, IEvaluatorContext context,
+ SourceLocation sourceLoc) throws HyracksDataException {
+ super(sourceLoc);
+ ctx = context;
+ eval = args[0].createScalarEvaluator(context);
+ frame = new VSizeFrame(context.getTaskContext());
+ }
+
+ @Override
+ public void init() throws HyracksDataException {
+ if (partialResults == null) {
+ partialResults = new ArrayList<>();
+ }
+ if (recBuilder == null) {
+ recBuilder = new RecordBuilder();
+ recBuilder.reset(LocalMedianTypeComputer.REC_TYPE);
+ }
+ count = 0;
+ partialResults.clear();
+ recBuilder.init();
+ }
+
+ protected void processPartialResults(IFrameTupleReference tuple) throws HyracksDataException {
+ eval.evaluate(tuple, inputVal);
+ byte[] serBytes = inputVal.getByteArray();
+ int offset = inputVal.getStartOffset();
+ ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(serBytes[offset]);
+ if (typeTag == ATypeTag.OBJECT) {
+ long handleCount = AInt64SerializerDeserializer.getLong(serBytes,
+ ARecordSerializerDeserializer.getFieldOffsetById(serBytes, offset, COUNT_FIELD_ID, 0, false));
+ if (handleCount == 0) {
+ return;
+ }
+ count += handleCount;
+
+ long fileId = AInt64SerializerDeserializer.getLong(serBytes,
+ ARecordSerializerDeserializer.getFieldOffsetById(serBytes, offset, HANDLE_FIELD_ID, 0, false));
+
+ String address = UTF8StringUtil.toString(serBytes,
+ ARecordSerializerDeserializer.getFieldOffsetById(serBytes, offset, ADDRESS_FIELD_ID, 0, false));
+
+ int port = AInt32SerializerDeserializer.getInt(serBytes,
+ ARecordSerializerDeserializer.getFieldOffsetById(serBytes, offset, PORT_FIELD_ID, 0, false));
+
+ partialResults.add(new PartialResult(fileId, handleCount, address, port));
+ } else {
+ throw new UnsupportedItemTypeException(sourceLoc, MEDIAN, serBytes[offset]);
+ }
+ }
+
+ protected void finishPartialResult(IPointable result) throws HyracksDataException {
+ if (count == 0) {
+ setPartialResult(result, -1, "", -1);
+ return;
+ }
+
+ IHyracksTaskContext taskCtx = ctx.getTaskContext();
+ IHyracksJobletContext jobletCtx = taskCtx.getJobletContext();
+ RunMergingFrameReader merger = createRunsMergingFrameReader();
+ FileReference managedFile = jobletCtx.createManagedWorkspaceFile(MEDIAN);
+ RunFileWriter runFileWriter = new RunFileWriter(managedFile, taskCtx.getIoManager());
+ merger.open();
+ runFileWriter.open();
+ try {
+ while (merger.nextFrame(frame)) {
+ runFileWriter.nextFrame(frame.getBuffer());
+ }
+ } finally {
+ runFileWriter.close();
+ merger.close();
+ }
+
+ NetworkAddress netAddress = ((NodeControllerService) jobletCtx.getServiceContext().getControllerService())
+ .getNetworkManager().getPublicNetworkAddress();
+
+ long fileId = jobletCtx.nextUniqueId();
+ taskCtx.setStateObject(new JobFileState(managedFile, jobletCtx.getJobId(), fileId));
+ setPartialResult(result, fileId, netAddress.getAddress(), netAddress.getPort());
+ }
+
+ protected void finishFinalResult(IPointable result) throws HyracksDataException {
+ if (count == 0) {
+ PointableHelper.setNull(result);
+ return;
+ }
+ try {
+ boolean medianFound = findMedian();
+ resultStorage.reset();
+ if (medianFound) {
+ doubleSerde.serialize(aDouble, resultStorage.getDataOutput());
+ result.set(resultStorage);
+ } else {
+ PointableHelper.setNull(result);
+ LOGGER.warn("median was not found");
+ }
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ }
+
+ private boolean findMedian() throws HyracksDataException {
+ RunMergingFrameReader merger = createRunsMergingFrameReader();
+ return getMedian(merger);
+ }
+
+ protected RunMergingFrameReader createRunsMergingFrameReader() throws HyracksDataException {
+ IHyracksTaskContext taskCtx = ctx.getTaskContext();
+ IHyracksJobletContext jobletCtx = taskCtx.getJobletContext();
+ INCServiceContext serviceCtx = jobletCtx.getServiceContext();
+ NetworkManager netManager = ((NodeControllerService) serviceCtx.getControllerService()).getNetworkManager();
+ List<IFrame> inFrames = getInFrames(partialResults.size(), taskCtx);
+ readers.clear();
+ for (PartialResult partialResult : partialResults) {
+ IFrameReader inputChannelReader = createInputChannel(netManager, taskCtx,
+ new NetworkAddress(partialResult.address, partialResult.port), jobletCtx.getJobId().getId(),
+ partialResult.fileId);
+ readers.add(inputChannelReader);
+ }
+ return new RunMergingFrameReader(taskCtx, readers, inFrames, new int[] { 0 },
+ new IBinaryComparator[] { doubleComparatorFactory.createBinaryComparator() },
+ doubleNkComputerFactory.createNormalizedKeyComputer(), recordDesc);
+ }
+
+ private boolean getMedian(RunMergingFrameReader merger) throws HyracksDataException {
+ boolean isOdd = count % 2 != 0;
+ long medianPosition = isOdd ? count / 2 : (count - 1) / 2;
+ long currentTupleCount = 0;
+ boolean found = false;
+ merger.open();
+ try {
+ while (merger.nextFrame(frame)) {
+ fta.reset(frame.getBuffer());
+ int tupleCount = fta.getTupleCount();
+ if (currentTupleCount + tupleCount > medianPosition) {
+ int firstMedian = (int) (medianPosition - currentTupleCount);
+ ftr.reset(fta, firstMedian);
+ double medianVal =
+ ADoubleSerializerDeserializer.getDouble(ftr.getFieldData(0), ftr.getFieldStart(0) + 1);
+ if (!isOdd) {
+ if (firstMedian + 1 < tupleCount) {
+ // second median is in the same frame
+ ftr.reset(fta, firstMedian + 1);
+ } else {
+ // second median is in the next frame
+ merger.nextFrame(frame);
+ fta.reset(frame.getBuffer());
+ ftr.reset(fta, 0);
+ }
+ medianVal =
+ (ADoubleSerializerDeserializer.getDouble(ftr.getFieldData(0), ftr.getFieldStart(0) + 1)
+ + medianVal) / 2;
+ }
+ aDouble.setValue(medianVal);
+ found = true;
+ break;
+ }
+ currentTupleCount += tupleCount;
+ }
+ while (merger.nextFrame(frame)) {
+ // consume the remaining frames to close the network channels gracefully
+ }
+ } finally {
+ merger.close();
+ }
+ return found;
+ }
+
+ protected void setPartialResult(IPointable result, long fileId, String address, int port)
+ throws HyracksDataException {
+ try {
+ resultStorage.reset();
+ aInt64.setValue(count);
+ longSerde.serialize(aInt64, resultStorage.getDataOutput());
+ recBuilder.addField(COUNT_FIELD_ID, resultStorage);
+
+ resultStorage.reset();
+ aInt64.setValue(fileId);
+ longSerde.serialize(aInt64, resultStorage.getDataOutput());
+ recBuilder.addField(HANDLE_FIELD_ID, resultStorage);
+
+ resultStorage.reset();
+ aString.setValue(address);
+ stringSerde.serialize(aString, resultStorage.getDataOutput());
+ recBuilder.addField(ADDRESS_FIELD_ID, resultStorage);
+
+ resultStorage.reset();
+ aInt32.setValue(port);
+ intSerde.serialize(aInt32, resultStorage.getDataOutput());
+ recBuilder.addField(PORT_FIELD_ID, resultStorage);
+
+ resultStorage.reset();
+ recBuilder.write(resultStorage.getDataOutput(), true);
+ result.set(resultStorage);
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ }
+
+ protected List<IFrame> getInFrames(int size, IHyracksTaskContext taskCtx) throws HyracksDataException {
+ if (inFrames == null) {
+ inFrames = new ArrayList<>(size);
+ }
+ int k = 0;
+ for (int inFramesSize = inFrames.size(); k < size && k < inFramesSize; k++) {
+ inFrames.get(k).reset();
+ }
+ for (; k < size; k++) {
+ inFrames.add(new VSizeFrame(taskCtx));
+ }
+ return inFrames;
+ }
+
+ private IFrameReader createInputChannel(NetworkManager netManager, IHyracksTaskContext taskContext,
+ NetworkAddress networkAddress, long jobId, long fileId) throws HyracksDataException {
+ FileNetworkInputChannel FileNetworkInputChannel =
+ new FileNetworkInputChannel(netManager, getSocketAddress(networkAddress), jobId, fileId);
+ InputChannelFrameReader channelFrameReader = new InputChannelFrameReader(FileNetworkInputChannel);
+ FileNetworkInputChannel.registerMonitor(channelFrameReader);
+ FileNetworkInputChannel.open(taskContext);
+ return channelFrameReader;
+ }
+
+ private static SocketAddress getSocketAddress(NetworkAddress netAddress) throws HyracksDataException {
+ try {
+ return new InetSocketAddress(InetAddress.getByAddress(netAddress.lookupIpAddress()), netAddress.getPort());
+ } catch (UnknownHostException e) {
+ throw HyracksDataException.create(e);
+ }
+ }
+
+ public static class PartialResult {
+
+ long fileId;
+ String address;
+ int port;
+ long count;
+
+ PartialResult(long fileId, long count, String address, int port) {
+ this.fileId = fileId;
+ this.count = count;
+ this.address = address;
+ this.port = port;
+ }
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlMedianAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlMedianAggregateDescriptor.java
new file mode 100644
index 0000000..faf5272
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlMedianAggregateDescriptor.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.aggregates.std;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class GlobalSqlMedianAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
+
+ private static final long serialVersionUID = 1L;
+
+ public static final IFunctionDescriptorFactory FACTORY = GlobalSqlMedianAggregateDescriptor::new;
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.GLOBAL_SQL_MEDIAN;
+ }
+
+ @Override
+ public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IAggregateEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IAggregateEvaluator createAggregateEvaluator(final IEvaluatorContext ctx)
+ throws HyracksDataException {
+ return new GlobalSqlMedianAggregateFunction(args, ctx, sourceLoc);
+ }
+ };
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlMedianAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlMedianAggregateFunction.java
new file mode 100644
index 0000000..c9ee8d8
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlMedianAggregateFunction.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.aggregates.std;
+
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class GlobalSqlMedianAggregateFunction extends AbstractMedianAggregateFunction {
+
+ public GlobalSqlMedianAggregateFunction(IScalarEvaluatorFactory[] args, IEvaluatorContext context,
+ SourceLocation sourceLoc) throws HyracksDataException {
+ super(args, context, sourceLoc);
+
+ }
+
+ @Override
+ public void step(IFrameTupleReference tuple) throws HyracksDataException {
+ processPartialResults(tuple);
+ }
+
+ @Override
+ public void finish(IPointable result) throws HyracksDataException {
+ finishFinalResult(result);
+ }
+
+ @Override
+ public void finishPartial(IPointable result) throws HyracksDataException {
+ finishPartialResult(result);
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlMedianAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlMedianAggregateDescriptor.java
new file mode 100644
index 0000000..532cb5f
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlMedianAggregateDescriptor.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.aggregates.std;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class IntermediateSqlMedianAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
+
+ private static final long serialVersionUID = 1L;
+
+ public static final IFunctionDescriptorFactory FACTORY = IntermediateSqlMedianAggregateDescriptor::new;
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.INTERMEDIATE_SQL_MEDIAN;
+ }
+
+ @Override
+ public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IAggregateEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IAggregateEvaluator createAggregateEvaluator(final IEvaluatorContext ctx)
+ throws HyracksDataException {
+ return new IntermediateSqlMedianAggregateFunction(args, ctx, sourceLoc);
+ }
+ };
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlMedianAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlMedianAggregateFunction.java
new file mode 100644
index 0000000..d78c17f
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlMedianAggregateFunction.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.aggregates.std;
+
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class IntermediateSqlMedianAggregateFunction extends AbstractMedianAggregateFunction {
+
+ public IntermediateSqlMedianAggregateFunction(IScalarEvaluatorFactory[] args, IEvaluatorContext context,
+ SourceLocation sourceLoc) throws HyracksDataException {
+ super(args, context, sourceLoc);
+ }
+
+ @Override
+ public void step(IFrameTupleReference tuple) throws HyracksDataException {
+ processPartialResults(tuple);
+ }
+
+ @Override
+ public void finish(IPointable result) throws HyracksDataException {
+ finishPartialResult(result);
+ }
+
+ @Override
+ public void finishPartial(IPointable result) throws HyracksDataException {
+ finishPartialResult(result);
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlMedianAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlMedianAggregateDescriptor.java
new file mode 100644
index 0000000..50f36ce
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlMedianAggregateDescriptor.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.aggregates.std;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class LocalSqlMedianAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
+
+ private static final long serialVersionUID = 1L;
+
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new LocalSqlMedianAggregateDescriptor();
+ }
+
+ @Override
+ public IFunctionTypeInferer createFunctionTypeInferer() {
+ return FunctionTypeInferers.MEDIAN_MEMORY;
+ }
+ };
+
+ private int numFrames = 0;
+
+ @Override
+ public void setImmutableStates(Object... states) {
+ this.numFrames = (int) states[0];
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.LOCAL_SQL_MEDIAN;
+ }
+
+ @Override
+ public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IAggregateEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IAggregateEvaluator createAggregateEvaluator(final IEvaluatorContext ctx)
+ throws HyracksDataException {
+ return new LocalSqlMedianAggregateFunction(args, ctx, sourceLoc, numFrames);
+ }
+ };
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlMedianAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlMedianAggregateFunction.java
new file mode 100644
index 0000000..5400ca2
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlMedianAggregateFunction.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.aggregates.std;
+
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class LocalSqlMedianAggregateFunction extends AbstractLocalMedianAggregateFunction {
+
+ public LocalSqlMedianAggregateFunction(IScalarEvaluatorFactory[] args, IEvaluatorContext context,
+ SourceLocation sourceLoc, int numFrames) throws HyracksDataException {
+ super(args, context, sourceLoc, numFrames);
+ }
+
+ @Override
+ public void step(IFrameTupleReference tuple) throws HyracksDataException {
+ processDataValue(tuple);
+ }
+
+ @Override
+ public void finishPartial(IPointable result) throws HyracksDataException {
+ finishLocalPartial(result);
+ }
+
+ @Override
+ public void finish(IPointable result) throws HyracksDataException {
+ finishLocalPartial(result);
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlMedianAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlMedianAggregateDescriptor.java
new file mode 100644
index 0000000..b98e389
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlMedianAggregateDescriptor.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.aggregates.std;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class SqlMedianAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
+
+ private static final long serialVersionUID = 1L;
+
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new SqlMedianAggregateDescriptor();
+ }
+
+ @Override
+ public IFunctionTypeInferer createFunctionTypeInferer() {
+ return FunctionTypeInferers.MEDIAN_MEMORY;
+ }
+ };
+
+ private int numFrames = 0;
+
+ @Override
+ public void setImmutableStates(Object... states) {
+ this.numFrames = (int) states[0];
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.SQL_MEDIAN;
+ }
+
+ @Override
+ public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IAggregateEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IAggregateEvaluator createAggregateEvaluator(final IEvaluatorContext ctx)
+ throws HyracksDataException {
+ return new SqlMedianAggregateFunction(args, ctx, sourceLoc, numFrames);
+ }
+ };
+ }
+
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlMedianAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlMedianAggregateFunction.java
new file mode 100644
index 0000000..1dbbfe6
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlMedianAggregateFunction.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.aggregates.std;
+
+import java.util.List;
+
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.comm.IFrame;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+import org.apache.hyracks.dataflow.common.io.GeneratedRunFileReader;
+import org.apache.hyracks.dataflow.common.io.RunFileWriter;
+import org.apache.hyracks.dataflow.std.sort.RunMergingFrameReader;
+
+public class SqlMedianAggregateFunction extends AbstractLocalMedianAggregateFunction {
+
+ public SqlMedianAggregateFunction(IScalarEvaluatorFactory[] args, IEvaluatorContext context,
+ SourceLocation sourceLoc, int numFrames) throws HyracksDataException {
+ super(args, context, sourceLoc, numFrames);
+ }
+
+ @Override
+ public void step(IFrameTupleReference tuple) throws HyracksDataException {
+ processDataValue(tuple);
+ }
+
+ @Override
+ public void finish(IPointable result) throws HyracksDataException {
+ if (appender.getTupleCount() > 0) {
+ appender.write(runsGenerator, true);
+ }
+ // close to sort the in-memory data or write out sorted data to run files
+ runsGenerator.close();
+ super.finishFinalResult(result);
+ }
+
+ @Override
+ public void finishPartial(IPointable result) throws HyracksDataException {
+ finishLocalPartial(result);
+ }
+
+ @Override
+ protected RunMergingFrameReader createRunsMergingFrameReader() throws HyracksDataException {
+ IHyracksTaskContext taskCtx = ctx.getTaskContext();
+ List<GeneratedRunFileReader> runs = runsGenerator.getRuns();
+ readers.clear();
+ if (runs.isEmpty()) {
+ //TODO: no need to write memory to run file, should just read the sorted data out of the sorter
+ FileReference managedFile = taskCtx.createManagedWorkspaceFile(MEDIAN);
+ RunFileWriter runFileWriter = writeMemoryDataToRunFile(managedFile, taskCtx);
+ GeneratedRunFileReader deleteOnCloseReader = runFileWriter.createDeleteOnCloseReader();
+ readers.add(deleteOnCloseReader);
+ } else {
+ readers.addAll(runs);
+ }
+
+ List<IFrame> inFrames = getInFrames(readers.size(), taskCtx);
+ return new RunMergingFrameReader(taskCtx, readers, inFrames, new int[] { 0 },
+ new IBinaryComparator[] { doubleComparatorFactory.createBinaryComparator() },
+ doubleNkComputerFactory.createNormalizedKeyComputer(), recordDesc);
+ }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/IntArray.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/IntArray.java
new file mode 100644
index 0000000..627ed89
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/IntArray.java
@@ -0,0 +1,80 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.evaluators.common;
+
+import java.util.Arrays;
+
+public class IntArray {
+ private static final int SIZE = 128;
+
+ private int[] data;
+ private int length;
+
+ public IntArray() {
+ data = new int[SIZE];
+ length = 0;
+ }
+
+ public void add(int d) {
+ if (length == data.length) {
+ data = Arrays.copyOf(data, data.length << 1);
+ }
+ data[length++] = d;
+ }
+
+ public int[] get() {
+ return data;
+ }
+
+ public int get(int i) {
+ return data[i];
+ }
+
+ public int length() {
+ return length;
+ }
+
+ public void reset() {
+ length = 0;
+ }
+
+ public void sort() {
+ sort(0, length);
+ }
+
+ public void sort(int start, int end) {
+ Arrays.sort(data, start, end);
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder out = new StringBuilder();
+ out.append('[');
+ for (int i = 0; i < length; ++i) {
+ out.append(data[i]);
+ if (i < length - 1) {
+ out.append(',');
+ out.append(' ');
+ }
+ }
+ out.append(']');
+ return out.toString();
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractBinaryStringEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractBinaryStringEval.java
index 2fc8654..204020e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractBinaryStringEval.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractBinaryStringEval.java
@@ -106,6 +106,13 @@
// The actual processing.
try {
process(leftStringPointable, rightStringPointable, resultPointable);
+ } catch (HyracksDataException ex) {
+ if (ExceptionUtil.isStringUnicodeError(ex)) {
+ PointableHelper.setNull(resultPointable);
+ ExceptionUtil.warnFunctionEvalFailed(ctx, sourceLoc, funcID, ex.getMessageNoCode());
+ return;
+ }
+ throw ex;
} catch (IOException e) {
throw HyracksDataException.create(e);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractUnaryStringStringEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractUnaryStringStringEval.java
index 5efe529..d92c9e9 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractUnaryStringStringEval.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractUnaryStringStringEval.java
@@ -84,6 +84,13 @@
try {
process(stringPtr, resultPointable);
writeResult(resultPointable);
+ } catch (HyracksDataException ex) {
+ if (ExceptionUtil.isStringUnicodeError(ex)) {
+ PointableHelper.setNull(resultPointable);
+ ExceptionUtil.warnFunctionEvalFailed(ctx, sourceLoc, funcID, ex.getMessageNoCode());
+ return;
+ }
+ throw ex;
} catch (IOException e) {
throw HyracksDataException.create(e);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SerializedSizeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SerializedSizeDescriptor.java
new file mode 100644
index 0000000..2669fd7
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SerializedSizeDescriptor.java
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.evaluators.functions;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AInt64;
+import org.apache.asterix.om.base.AMutableInt64;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public final class SerializedSizeDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+ private static final long serialVersionUID = 1L;
+
+ public static final IFunctionDescriptorFactory FACTORY = SerializedSizeDescriptor::new;
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
+ return new AbstractScalarEval(sourceLoc, getIdentifier()) {
+ private final IScalarEvaluator eval0 = args[0].createScalarEvaluator(ctx);
+ private final VoidPointable arg0 = VoidPointable.FACTORY.createPointable();
+ private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+ private final DataOutput resultOut = resultStorage.getDataOutput();
+
+ @SuppressWarnings("unchecked")
+ private final ISerializerDeserializer<AInt64> int64Serde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT64);
+ private final AMutableInt64 int64 = new AMutableInt64(0);
+
+ @Override
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+ resultStorage.reset();
+ eval0.evaluate(tuple, arg0);
+ int64.setValue(arg0.getLength());
+ try {
+ int64Serde.serialize(int64, resultOut);
+ result.set(resultStorage);
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ }
+ };
+ }
+ };
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.SERIALIZED_SIZE;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SleepDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SleepDescriptor.java
index ac87f7e..ef348fe 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SleepDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SleepDescriptor.java
@@ -64,16 +64,16 @@
final long time = ATypeHierarchy.getLongValue(getIdentifier().getName(), 1, bytes, offset);
try {
- if (LOGGER.isInfoEnabled()) {
- LOGGER.log(Level.INFO,
+ if (LOGGER.isTraceEnabled()) {
+ LOGGER.log(Level.TRACE,
ctx.getTaskContext().getTaskAttemptId() + " sleeping for " + time + " ms");
}
Thread.sleep(time);
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
} finally {
- if (LOGGER.isInfoEnabled()) {
- LOGGER.log(Level.INFO,
+ if (LOGGER.isTraceEnabled()) {
+ LOGGER.log(Level.TRACE,
ctx.getTaskContext().getTaskAttemptId() + " done sleeping for " + time + " ms");
}
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java
new file mode 100644
index 0000000..4e69b43
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java
@@ -0,0 +1,1102 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.evaluators.functions;
+
+import java.io.DataOutput;
+
+import org.apache.asterix.common.annotations.MissingNullInOutFunction;
+import org.apache.asterix.dataflow.data.nontagged.Coordinate;
+import org.apache.asterix.dataflow.data.nontagged.serde.ACircleSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt16SerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.ALineSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AObjectSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.APointSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.APolygonSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.ARectangleSerializerDeserializer;
+import org.apache.asterix.formats.nontagged.BinaryComparatorFactoryProvider;
+import org.apache.asterix.om.base.ABoolean;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.common.DoubleArray;
+import org.apache.asterix.runtime.evaluators.common.IntArray;
+import org.apache.asterix.runtime.evaluators.common.SpatialUtils;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+@MissingNullInOutFunction
+public class SpatialIntersectDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ private static final long serialVersionUID = 1L;
+
+ public static final IFunctionDescriptorFactory FACTORY = SpatialIntersectDescriptor::new;
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(final IEvaluatorContext ctx) throws HyracksDataException {
+ return new IScalarEvaluator() {
+
+ private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+ private final DataOutput out = resultStorage.getDataOutput();
+ private final IPointable inputArg0 = new VoidPointable();
+ private final IPointable inputArg1 = new VoidPointable();
+ private final IScalarEvaluator eval0 = args[0].createScalarEvaluator(ctx);
+ private final IScalarEvaluator eval1 = args[1].createScalarEvaluator(ctx);
+ private final IBinaryComparator ascDoubleComp =
+ BinaryComparatorFactoryProvider.DOUBLE_POINTABLE_INSTANCE.createBinaryComparator();
+ private final SpatialUtils spatialUtils = new SpatialUtils();
+ private final IntArray pointsOffsets0 = new IntArray();
+ private final IntArray pointsOffsets1 = new IntArray();
+ private final DoubleArray trianglesX0 = new DoubleArray();
+ private final DoubleArray trianglesY0 = new DoubleArray();
+ private final DoubleArray trianglesX1 = new DoubleArray();
+ private final DoubleArray trianglesY1 = new DoubleArray();
+ private final AObjectSerializerDeserializer aBooleanSerDer = AObjectSerializerDeserializer.INSTANCE;
+
+ private boolean pointOnLine(double pX, double pY, double startX, double startY, double endX,
+ double endY) throws HyracksDataException {
+ double crossProduct =
+ SpatialUtils.crossProduct(pY - startY, pX - startX, endY - startY, endX - startX);
+ if (Math.abs(crossProduct) > SpatialUtils.doubleEpsilon()) { // crossProduct != 0
+ return false;
+ }
+
+ double dotProduct =
+ SpatialUtils.dotProduct((pX - startX), (pY - startY), (endX - startX), (endY - startY));
+ if (dotProduct < 0.0) {
+ return false;
+ }
+
+ double squaredLengthBA = (endX - startX) * (endX - startX) + (endY - startY) * (endY - startY);
+ if (dotProduct > squaredLengthBA) {
+ return false;
+ }
+ return true;
+ }
+
+ private boolean pointInPolygon(byte[] bytes0, int offset0, byte[] bytes1, int offset1)
+ throws HyracksDataException { // ray casting
+
+ double pX = ADoubleSerializerDeserializer.getDouble(bytes0,
+ offset0 + 1 + APointSerializerDeserializer.getCoordinateOffset(Coordinate.X));
+ double pY = ADoubleSerializerDeserializer.getDouble(bytes0,
+ offset0 + 1 + APointSerializerDeserializer.getCoordinateOffset(Coordinate.Y));
+ int numOfPoints1 = AInt16SerializerDeserializer.getShort(bytes1,
+ offset1 + 1 + APolygonSerializerDeserializer.getNumberOfPointsOffset());
+
+ if (numOfPoints1 < 3) {
+ throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
+ ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
+ }
+
+ int counter = 0;
+ double xInters;
+ double x1, x2, y1, y2;
+ x1 = ADoubleSerializerDeserializer.getDouble(bytes1,
+ offset1 + 1 + APolygonSerializerDeserializer.getCoordinateOffset(0, Coordinate.X));
+ y1 = ADoubleSerializerDeserializer.getDouble(bytes1,
+ offset1 + 1 + APolygonSerializerDeserializer.getCoordinateOffset(0, Coordinate.Y));
+
+ for (int i = 1; i <= numOfPoints1; i++) {
+ if (i == numOfPoints1) {
+ x2 = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
+ + APolygonSerializerDeserializer.getCoordinateOffset(0, Coordinate.X));
+ y2 = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
+ + APolygonSerializerDeserializer.getCoordinateOffset(0, Coordinate.Y));
+ } else {
+ x2 = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
+ + APolygonSerializerDeserializer.getCoordinateOffset(i, Coordinate.X));
+ y2 = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
+ + APolygonSerializerDeserializer.getCoordinateOffset(i, Coordinate.Y));
+ }
+
+ if (!pointOnLine(pX, pY, x1, y1, x2, y2)) {
+ if (pY > Math.min(y1, y2)) {
+ if (pY <= Math.max(y1, y2)) {
+ if (pX <= Math.max(x1, x2)) {
+ if (y1 != y2) {
+ xInters = (pY - y1) * (x2 - x1) / (y2 - y1) + x1;
+ if (x1 == x2 || pX <= xInters) {
+ counter++;
+ }
+ }
+ }
+ }
+ }
+ }
+ x1 = x2;
+ y1 = y2;
+ }
+ if (counter % 2 == 1) {
+ return true;
+ } else {
+ return false;
+ }
+ }
+
+ private boolean pointInCircle(byte[] bytes0, int offset0, byte[] bytes1, int offset1)
+ throws HyracksDataException {
+ double x = ADoubleSerializerDeserializer.getDouble(bytes0,
+ offset0 + 1 + APointSerializerDeserializer.getCoordinateOffset(Coordinate.X));
+ double y = ADoubleSerializerDeserializer.getDouble(bytes0,
+ offset0 + 1 + APointSerializerDeserializer.getCoordinateOffset(Coordinate.Y));
+
+ double cX = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
+ + ACircleSerializerDeserializer.getCenterPointCoordinateOffset(Coordinate.X));
+ double cY = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
+ + ACircleSerializerDeserializer.getCenterPointCoordinateOffset(Coordinate.Y));
+ double radius = ADoubleSerializerDeserializer.getDouble(bytes1,
+ offset1 + 1 + ACircleSerializerDeserializer.getRadiusOffset());
+
+ if ((x - cX) * (x - cX) + (y - cY) * (y - cY) <= (radius * radius)) {
+ return true;
+ }
+ return false;
+ }
+
+ private boolean lineLineIntersection(double startX1, double startY1, double endX1, double endY1,
+ double startX2, double startY2, double endX2, double endY2) {
+ double A1 = endY1 - startY1;
+ double B1 = startX1 - endX1;
+ double C1 = A1 * startX1 + B1 * startY1;
+
+ double A2 = endY2 - startY2;
+ double B2 = startX2 - endX2;
+ double C2 = A2 * startX2 + B2 * startY2;
+
+ double det = (A1 * B2) - (A2 * B1);
+ if (Math.abs(det) > SpatialUtils.doubleEpsilon()) { // det != 0
+ double x = (B2 * C1 - B1 * C2) / det;
+ double y = (A1 * C2 - A2 * C1) / det;
+
+ if ((x >= Math.min(startX1, endX1) && x <= Math.max(startX1, endX1))
+ && (y >= Math.min(startY1, endY1) && y <= Math.max(startY1, endY1))) {
+ if ((x >= Math.min(startX2, endX2) && x <= Math.max(startX2, endX2))
+ && (y >= Math.min(startY2, endY2) && y <= Math.max(startY2, endY2))) {
+ return true;
+ }
+ }
+ }
+ return false;
+ }
+
+ private boolean linePolygonIntersection(byte[] bytes0, int offset0, byte[] bytes1, int offset1)
+ throws HyracksDataException {
+ double startX1 = ADoubleSerializerDeserializer.getDouble(bytes0,
+ offset0 + 1 + ALineSerializerDeserializer.getStartPointCoordinateOffset(Coordinate.X));
+ double startY1 = ADoubleSerializerDeserializer.getDouble(bytes0,
+ offset0 + 1 + ALineSerializerDeserializer.getStartPointCoordinateOffset(Coordinate.Y));
+ double endX1 = ADoubleSerializerDeserializer.getDouble(bytes0,
+ offset0 + 1 + ALineSerializerDeserializer.getEndPointCoordinateOffset(Coordinate.X));
+ double endY1 = ADoubleSerializerDeserializer.getDouble(bytes0,
+ offset0 + 1 + ALineSerializerDeserializer.getEndPointCoordinateOffset(Coordinate.Y));
+
+ int numOfPoints1 = AInt16SerializerDeserializer.getShort(bytes1,
+ offset1 + 1 + APolygonSerializerDeserializer.getNumberOfPointsOffset());
+
+ if (numOfPoints1 < 3) {
+ throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
+ ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
+ }
+ for (int i = 0; i < numOfPoints1; i++) {
+ double startX2 = ADoubleSerializerDeserializer.getDouble(bytes1,
+ offset1 + 1 + APolygonSerializerDeserializer.getCoordinateOffset(i, Coordinate.X));
+ double startY2 = ADoubleSerializerDeserializer.getDouble(bytes1,
+ offset1 + 1 + APolygonSerializerDeserializer.getCoordinateOffset(i, Coordinate.Y));
+
+ double endX2;
+ double endY2;
+ if (i + 1 == numOfPoints1) {
+ endX2 = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
+ + APolygonSerializerDeserializer.getCoordinateOffset(0, Coordinate.X));
+ endY2 = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
+ + APolygonSerializerDeserializer.getCoordinateOffset(0, Coordinate.Y));
+ } else {
+ endX2 = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
+ + APolygonSerializerDeserializer.getCoordinateOffset(i + 1, Coordinate.X));
+ endY2 = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
+ + APolygonSerializerDeserializer.getCoordinateOffset(i + 1, Coordinate.Y));
+ }
+
+ boolean intersect = lineLineIntersection(startX1, startY1, endX1, endY1, startX2, startY2,
+ endX2, endY2);
+ if (intersect) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ private boolean lineRectangleIntersection(byte[] bytes0, int offset0, byte[] bytes1, int offset1)
+ throws HyracksDataException {
+ double startX1 = ADoubleSerializerDeserializer.getDouble(bytes0,
+ offset0 + 1 + ALineSerializerDeserializer.getStartPointCoordinateOffset(Coordinate.X));
+ double startY1 = ADoubleSerializerDeserializer.getDouble(bytes0,
+ offset0 + 1 + ALineSerializerDeserializer.getStartPointCoordinateOffset(Coordinate.Y));
+ double endX1 = ADoubleSerializerDeserializer.getDouble(bytes0,
+ offset0 + 1 + ALineSerializerDeserializer.getEndPointCoordinateOffset(Coordinate.X));
+ double endY1 = ADoubleSerializerDeserializer.getDouble(bytes0,
+ offset0 + 1 + ALineSerializerDeserializer.getEndPointCoordinateOffset(Coordinate.Y));
+
+ double x1 = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
+ + ARectangleSerializerDeserializer.getBottomLeftCoordinateOffset(Coordinate.X));
+ double y1 = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
+ + ARectangleSerializerDeserializer.getBottomLeftCoordinateOffset(Coordinate.Y));
+
+ double x2 = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
+ + ARectangleSerializerDeserializer.getUpperRightCoordinateOffset(Coordinate.X));
+ double y2 = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
+ + ARectangleSerializerDeserializer.getUpperRightCoordinateOffset(Coordinate.Y));
+
+ if (lineLineIntersection(startX1, startY1, endX1, endY1, x1, y1, x1, y2)
+ || lineLineIntersection(startX1, startY1, endX1, endY1, x1, y2, x2, y2)
+ || lineLineIntersection(startX1, startY1, endX1, endY1, x2, y2, x2, y1)
+ || lineLineIntersection(startX1, startY1, endX1, endY1, x2, y1, x1, y1)) {
+ return true;
+ }
+ return false;
+
+ }
+
+ private boolean lineCircleIntersection(byte[] bytes0, int offset0, byte[] bytes1, int offset1)
+ throws HyracksDataException {
+ double startX = ADoubleSerializerDeserializer.getDouble(bytes0,
+ offset0 + 1 + ALineSerializerDeserializer.getStartPointCoordinateOffset(Coordinate.X));
+ double startY = ADoubleSerializerDeserializer.getDouble(bytes0,
+ offset0 + 1 + ALineSerializerDeserializer.getStartPointCoordinateOffset(Coordinate.Y));
+ double endX = ADoubleSerializerDeserializer.getDouble(bytes0,
+ offset0 + 1 + ALineSerializerDeserializer.getEndPointCoordinateOffset(Coordinate.X));
+ double endY = ADoubleSerializerDeserializer.getDouble(bytes0,
+ offset0 + 1 + ALineSerializerDeserializer.getEndPointCoordinateOffset(Coordinate.Y));
+
+ double cX = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
+ + ACircleSerializerDeserializer.getCenterPointCoordinateOffset(Coordinate.X));
+ double cY = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
+ + ACircleSerializerDeserializer.getCenterPointCoordinateOffset(Coordinate.Y));
+ double radius = ADoubleSerializerDeserializer.getDouble(bytes1,
+ offset1 + 1 + ACircleSerializerDeserializer.getRadiusOffset());
+
+ double dx = endX - startX;
+ double dy = endY - startY;
+ double t = -((startX - cX) * dx + (startY - cY) * dy) / ((dx * dx) + (dy * dy));
+
+ if (t < 0.0) {
+ t = 0.0;
+ } else if (t > 1.0) {
+ t = 1.0;
+ }
+
+ dx = (startX + t * (endX - startX)) - cX;
+ dy = (startY + t * (endY - startY)) - cY;
+ double rt = (dx * dx) + (dy * dy);
+ if (rt <= (radius * radius)) {
+ return true;
+ }
+ return false;
+ }
+
+ private boolean findEar(byte[] bytes, int offset, int u, int v, int w, int n,
+ IntArray pointsOffsets) throws HyracksDataException {
+ int p;
+ double Ax, Ay, Bx, By, Cx, Cy, Px, Py;
+
+ Ax = ADoubleSerializerDeserializer.getDouble(bytes, offset + 1 + APolygonSerializerDeserializer
+ .getCoordinateOffset(pointsOffsets.get(u), Coordinate.X));
+ Ay = ADoubleSerializerDeserializer.getDouble(bytes, offset + 1 + APolygonSerializerDeserializer
+ .getCoordinateOffset(pointsOffsets.get(u), Coordinate.Y));
+
+ Bx = ADoubleSerializerDeserializer.getDouble(bytes, offset + 1 + APolygonSerializerDeserializer
+ .getCoordinateOffset(pointsOffsets.get(v), Coordinate.X));
+ By = ADoubleSerializerDeserializer.getDouble(bytes, offset + 1 + APolygonSerializerDeserializer
+ .getCoordinateOffset(pointsOffsets.get(v), Coordinate.Y));
+
+ Cx = ADoubleSerializerDeserializer.getDouble(bytes, offset + 1 + APolygonSerializerDeserializer
+ .getCoordinateOffset(pointsOffsets.get(w), Coordinate.X));
+ Cy = ADoubleSerializerDeserializer.getDouble(bytes, offset + 1 + APolygonSerializerDeserializer
+ .getCoordinateOffset(pointsOffsets.get(w), Coordinate.Y));
+
+ if (SpatialUtils.doubleEpsilon() > (((Bx - Ax) * (Cy - Ay)) - ((By - Ay) * (Cx - Ax)))) {
+
+ return false;
+ }
+
+ for (p = 0; p < n; p++) {
+ if ((p == u) || (p == v) || (p == w)) {
+ continue;
+ }
+ Px = ADoubleSerializerDeserializer.getDouble(bytes,
+ offset + 1 + APolygonSerializerDeserializer
+ .getCoordinateOffset(pointsOffsets.get(p), Coordinate.X));
+ Py = ADoubleSerializerDeserializer.getDouble(bytes,
+ offset + 1 + APolygonSerializerDeserializer
+ .getCoordinateOffset(pointsOffsets.get(p), Coordinate.Y));
+ if (pointInsideTriangle(Ax, Ay, Bx, By, Cx, Cy, Px, Py)) {
+ return false;
+ }
+ }
+
+ return true;
+ }
+
+ private int triangulatePolygon(byte[] bytes, int offset, int numOfPoints, IntArray pointsOffsets,
+ DoubleArray trianglesX, DoubleArray trianglesY, int triangleId,
+ int nonSimplePolygonDetection, int middleVertex) throws HyracksDataException { // Ear clipping
+
+ if (numOfPoints < 3) {
+ return -1;
+ }
+
+ boolean foundEar = false;
+ int v = middleVertex;
+ while (!foundEar) {
+ if (0 >= (nonSimplePolygonDetection--)) {
+ throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
+ ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
+ }
+ int u = v;
+ if (numOfPoints <= u) {
+ u = 0;
+ }
+ v = u + 1;
+ if (numOfPoints <= v) {
+ v = 0;
+ }
+ int w = v + 1;
+ if (numOfPoints <= w) {
+ w = 0;
+ }
+
+ if (findEar(bytes, offset, u, v, w, numOfPoints, pointsOffsets)) {
+ int s, t;
+
+ addRectangle(trianglesX, trianglesY);
+
+ SpatialUtils.setTriangleXCoordinate(trianglesX, triangleId, 0,
+ ADoubleSerializerDeserializer.getDouble(bytes,
+ offset + 1 + APolygonSerializerDeserializer
+ .getCoordinateOffset(pointsOffsets.get(u), Coordinate.X)));
+
+ SpatialUtils.setTriangleYCoordinate(trianglesY, triangleId, 0,
+ ADoubleSerializerDeserializer.getDouble(bytes,
+ offset + 1 + APolygonSerializerDeserializer
+ .getCoordinateOffset(pointsOffsets.get(u), Coordinate.Y)));
+
+ SpatialUtils.setTriangleXCoordinate(trianglesX, triangleId, 1,
+ ADoubleSerializerDeserializer.getDouble(bytes,
+ offset + 1 + APolygonSerializerDeserializer
+ .getCoordinateOffset(pointsOffsets.get(v), Coordinate.X)));
+
+ SpatialUtils.setTriangleYCoordinate(trianglesY, triangleId, 1,
+ ADoubleSerializerDeserializer.getDouble(bytes,
+ offset + 1 + APolygonSerializerDeserializer
+ .getCoordinateOffset(pointsOffsets.get(v), Coordinate.Y)));
+
+ SpatialUtils.setTriangleXCoordinate(trianglesX, triangleId, 2,
+ ADoubleSerializerDeserializer.getDouble(bytes,
+ offset + 1 + APolygonSerializerDeserializer
+ .getCoordinateOffset(pointsOffsets.get(w), Coordinate.X)));
+
+ SpatialUtils.setTriangleYCoordinate(trianglesY, triangleId, 2,
+ ADoubleSerializerDeserializer.getDouble(bytes,
+ offset + 1 + APolygonSerializerDeserializer
+ .getCoordinateOffset(pointsOffsets.get(w), Coordinate.Y)));
+
+ // remove v from polygon
+ for (s = v, t = v + 1; t < numOfPoints; s++, t++) {
+ pointsOffsets.get()[s] = pointsOffsets.get(t);
+ }
+ foundEar = true;
+ }
+ }
+
+ return v;
+ }
+
+ private boolean triangleTriangleIntersection(DoubleArray trianglesX0, DoubleArray trianglesY0,
+ int triangleId0, DoubleArray trianglesX1, DoubleArray trianglesY1, int triangleId1)
+ throws HyracksDataException { // separating axis theorem
+
+ for (int side = 0; side < 3; side++) {
+ spatialUtils.findNormals(trianglesX0, trianglesY0, triangleId0, side);
+ spatialUtils.projectPolygon(trianglesX0, trianglesY0, triangleId0, spatialUtils.getXAxis(),
+ spatialUtils.getYAxis());
+ double min1 = spatialUtils.getMinProjection();
+ double max1 = spatialUtils.getMaxProjection();
+ spatialUtils.projectPolygon(trianglesX1, trianglesY1, triangleId1, spatialUtils.getXAxis(),
+ spatialUtils.getYAxis());
+ double min2 = spatialUtils.getMinProjection();
+ double max2 = spatialUtils.getMaxProjection();
+
+ if (max1 < min2 || min1 > max2) {
+ return false;
+ }
+ }
+ return true;
+ }
+
+ private boolean pointInsideTriangle(double x1, double y1, double x2, double y2, double x3,
+ double y3, double pX, double pY) {
+ return pointsOnSameSide(pX, pY, x1, y1, x2, y2, x3, y3)
+ && pointsOnSameSide(pX, pY, x2, y2, x1, y1, x3, y3)
+ && pointsOnSameSide(pX, pY, x3, y3, x1, y1, x2, y2);
+ }
+
+ private boolean pointsOnSameSide(double pX, double pY, double x1, double y1, double x2, double y2,
+ double x3, double y3) {
+ double cp1 = SpatialUtils.crossProduct(x3 - x2, y3 - y2, pX - x2, pY - y2);
+ double cp2 = SpatialUtils.crossProduct(x3 - x2, y3 - y2, x1 - x2, y1 - y2);
+ return (cp1 * cp2) >= 0.0;
+ }
+
+ private boolean circleTriangleIntersection(byte[] bytes0, int offset0, DoubleArray trianglesX,
+ DoubleArray trianglesY, int triangleId) throws HyracksDataException { // separating axis theorem
+
+ double cX = ADoubleSerializerDeserializer.getDouble(bytes0, offset0 + 1
+ + ACircleSerializerDeserializer.getCenterPointCoordinateOffset(Coordinate.X));
+ double cY = ADoubleSerializerDeserializer.getDouble(bytes0, offset0 + 1
+ + ACircleSerializerDeserializer.getCenterPointCoordinateOffset(Coordinate.Y));
+ double radius = ADoubleSerializerDeserializer.getDouble(bytes0,
+ offset0 + 1 + ACircleSerializerDeserializer.getRadiusOffset());
+
+ double distance = Double.MAX_VALUE;
+ double distanceSquared;
+
+ double temp;
+ double closestPointX = 0.0;
+ double closestPointY = 0.0;
+ for (int i = 0; i < 3; i++) {
+ double pX = SpatialUtils.getTriangleXCoordinate(trianglesX, triangleId, i);
+ double pY = SpatialUtils.getTriangleXCoordinate(trianglesY, triangleId, i);
+
+ distanceSquared = (cX - pX) * (cX - pX) + (cY - pY) * (cY - pY);
+ if (distanceSquared < distance) {
+ distance = distanceSquared;
+ closestPointX = pX;
+ closestPointY = pY;
+ }
+ }
+
+ double x = Math.abs(cX - closestPointX);
+ double y = Math.abs(cY - closestPointY);
+
+ temp = Math.sqrt(SpatialUtils.dotProduct(x, y, x, y));
+ x /= temp;
+ y /= temp;
+
+ spatialUtils.projectPolygon(trianglesX, trianglesY, triangleId, x, y);
+
+ double min1 = spatialUtils.getMinProjection();
+ double max1 = spatialUtils.getMaxProjection();
+
+ double dotProduct = SpatialUtils.dotProduct(x, y, cX, cY);
+ double max2 = dotProduct + radius;
+ double min2 = dotProduct - radius;
+
+ if (max1 < min2 || min1 > max2) {
+ return false;
+ }
+
+ for (int side = 0; side < 3; side++) {
+ spatialUtils.findNormals(trianglesX, trianglesY, triangleId, side);
+ spatialUtils.projectPolygon(trianglesX, trianglesY, triangleId, spatialUtils.getXAxis(),
+ spatialUtils.getYAxis());
+ min1 = spatialUtils.getMinProjection();
+ max1 = spatialUtils.getMaxProjection();
+
+ dotProduct =
+ SpatialUtils.dotProduct(spatialUtils.getXAxis(), spatialUtils.getYAxis(), cX, cY);
+ max2 = dotProduct + radius;
+ min2 = dotProduct - radius;
+
+ if (max1 < min2 || min1 > max2) {
+ return false;
+ }
+ }
+ return true;
+ }
+
+ private boolean circleCircleIntersection(byte[] bytes0, int offset0, byte[] bytes1, int offset1)
+ throws HyracksDataException {
+ double cX0 = ADoubleSerializerDeserializer.getDouble(bytes0, offset0 + 1
+ + ACircleSerializerDeserializer.getCenterPointCoordinateOffset(Coordinate.X));
+ double cY0 = ADoubleSerializerDeserializer.getDouble(bytes0, offset0 + 1
+ + ACircleSerializerDeserializer.getCenterPointCoordinateOffset(Coordinate.Y));
+ double radius0 = ADoubleSerializerDeserializer.getDouble(bytes0,
+ offset0 + 1 + ACircleSerializerDeserializer.getRadiusOffset());
+
+ double cX1 = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
+ + ACircleSerializerDeserializer.getCenterPointCoordinateOffset(Coordinate.X));
+ double cY1 = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
+ + ACircleSerializerDeserializer.getCenterPointCoordinateOffset(Coordinate.Y));
+ double radius1 = ADoubleSerializerDeserializer.getDouble(bytes1,
+ offset1 + 1 + ACircleSerializerDeserializer.getRadiusOffset());
+
+ double distanceSquared = SpatialUtils.dotProduct(cX0 - cX1, cY0 - cY1, cX0 - cX1, cY0 - cY1);
+ double radiusDistanceSquared = (radius0 + radius1) * (radius0 + radius1);
+ if (distanceSquared <= radiusDistanceSquared) {
+ return true;
+ }
+ return false;
+ }
+
+ private void getCounterClockWisePolygon(byte[] bytes, int offset, IntArray pointsOffsets,
+ int numOfPoints) throws HyracksDataException {
+ pointsOffsets.reset();
+ if (SpatialUtils.polygonArea(bytes, offset, numOfPoints) > 0.0) {
+ for (int i = 0; i < numOfPoints; i++) {
+ pointsOffsets.add(i);
+ }
+ } else {
+ for (int i = 0; i < numOfPoints; i++) {
+ pointsOffsets.add((numOfPoints - 1) - i);
+ }
+ }
+ }
+
+ private boolean pointInRectangle(byte[] bytes0, int offset0, byte[] bytes1, int offset1)
+ throws HyracksDataException {
+
+ double pX = ADoubleSerializerDeserializer.getDouble(bytes0,
+ offset0 + 1 + APointSerializerDeserializer.getCoordinateOffset(Coordinate.X));
+ double pY = ADoubleSerializerDeserializer.getDouble(bytes0,
+ offset0 + 1 + APointSerializerDeserializer.getCoordinateOffset(Coordinate.Y));
+
+ double x1 = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
+ + ARectangleSerializerDeserializer.getBottomLeftCoordinateOffset(Coordinate.X));
+ double y1 = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
+ + ARectangleSerializerDeserializer.getBottomLeftCoordinateOffset(Coordinate.Y));
+
+ double x2 = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
+ + ARectangleSerializerDeserializer.getUpperRightCoordinateOffset(Coordinate.X));
+ double y2 = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
+ + ARectangleSerializerDeserializer.getUpperRightCoordinateOffset(Coordinate.Y));
+
+ if (pointInsideTriangle(x1, y1, x1, y2, x2, y2, pX, pY)
+ || pointInsideTriangle(x1, y1, x2, y1, x2, y2, pX, pY)) {
+ return true;
+ }
+ return false;
+
+ }
+
+ private void addRectangle(DoubleArray trianglesX, DoubleArray trianglesY) {
+ for (int i = 0; i < 3; i++) {
+ double temp = 0;
+ trianglesX.add(temp);
+ trianglesY.add(temp);
+ }
+ }
+
+ private boolean rectangleCircleIntersection(byte[] bytes0, int offset0, byte[] bytes1, int offset1)
+ throws HyracksDataException {
+ triangulateRectangle(bytes0, offset0, trianglesX0, trianglesY0);
+ boolean res = false;
+ // 2 triangles in a rectangle
+ for (int i = 0; i < 2; i++) {
+ res = circleTriangleIntersection(bytes1, offset1, trianglesX0, trianglesY0, i);
+ if (res) {
+ break;
+ }
+ }
+ return res;
+ }
+
+ private void triangulateRectangle(byte[] bytes, int offset, DoubleArray trianglesX,
+ DoubleArray trianglesY) throws HyracksDataException {
+ double x1 = ADoubleSerializerDeserializer.getDouble(bytes, offset + 1
+ + ARectangleSerializerDeserializer.getBottomLeftCoordinateOffset(Coordinate.X));
+ double y1 = ADoubleSerializerDeserializer.getDouble(bytes, offset + 1
+ + ARectangleSerializerDeserializer.getBottomLeftCoordinateOffset(Coordinate.Y));
+
+ double x2 = ADoubleSerializerDeserializer.getDouble(bytes, offset + 1
+ + ARectangleSerializerDeserializer.getUpperRightCoordinateOffset(Coordinate.X));
+ double y2 = ADoubleSerializerDeserializer.getDouble(bytes, offset + 1
+ + ARectangleSerializerDeserializer.getUpperRightCoordinateOffset(Coordinate.Y));
+ trianglesX.reset();
+ trianglesY.reset();
+
+ addRectangle(trianglesX, trianglesY);
+ addRectangle(trianglesX, trianglesY);
+
+ SpatialUtils.setTriangleXCoordinate(trianglesX, 0, 0, x1);
+ SpatialUtils.setTriangleYCoordinate(trianglesY, 0, 0, y1);
+
+ SpatialUtils.setTriangleXCoordinate(trianglesX, 0, 1, x2);
+ SpatialUtils.setTriangleYCoordinate(trianglesY, 0, 1, y1);
+
+ SpatialUtils.setTriangleXCoordinate(trianglesX, 0, 2, x2);
+ SpatialUtils.setTriangleYCoordinate(trianglesY, 0, 2, y2);
+
+ SpatialUtils.setTriangleXCoordinate(trianglesX, 1, 0, x2);
+ SpatialUtils.setTriangleYCoordinate(trianglesY, 1, 0, y2);
+
+ SpatialUtils.setTriangleXCoordinate(trianglesX, 1, 1, x1);
+ SpatialUtils.setTriangleYCoordinate(trianglesY, 1, 1, y2);
+
+ SpatialUtils.setTriangleXCoordinate(trianglesX, 1, 2, x1);
+ SpatialUtils.setTriangleYCoordinate(trianglesY, 1, 2, y1);
+ }
+
+ private boolean rectanglePolygonIntersection(byte[] bytes0, int offset0, byte[] bytes1, int offset1)
+ throws HyracksDataException {
+ int numOfPoints1 = AInt16SerializerDeserializer.getShort(bytes1,
+ offset1 + 1 + APolygonSerializerDeserializer.getNumberOfPointsOffset());
+
+ if (numOfPoints1 < 3) {
+ throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
+ ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
+ }
+
+ getCounterClockWisePolygon(bytes1, offset1, pointsOffsets1, numOfPoints1);
+ int nonSimplePolygonDetection1 = 2 * numOfPoints1;
+ int middleVertex1 = numOfPoints1 - 1;
+ int numOfTriangles1 = 0;
+
+ trianglesX1.reset();
+ trianglesY1.reset();
+ while (true) {
+ middleVertex1 =
+ triangulatePolygon(bytes1, offset1, numOfPoints1, pointsOffsets1, trianglesX1,
+ trianglesY1, numOfTriangles1, nonSimplePolygonDetection1, middleVertex1);
+
+ if (middleVertex1 == -1) {
+ break;
+ }
+
+ numOfPoints1--;
+ nonSimplePolygonDetection1 = 2 * numOfPoints1;
+ numOfTriangles1++;
+ }
+
+ triangulateRectangle(bytes0, offset0, trianglesX0, trianglesY0);
+ boolean res = false;
+ // 2 triangles in a rectangle
+ for (int j = 0; j < 2; j++) {
+ for (int i = 0; i < numOfTriangles1; i++) {
+
+ res = triangleTriangleIntersection(trianglesX1, trianglesY1, i, trianglesX0,
+ trianglesY0, j);
+
+ if (res) {
+ res = triangleTriangleIntersection(trianglesX0, trianglesY0, j, trianglesX1,
+ trianglesY1, i);
+
+ if (res) {
+ return true;
+ }
+ }
+ }
+ }
+ return false;
+ }
+
+ private boolean polygonCircleIntersection(byte[] bytes0, int offset0, byte[] bytes1, int offset1)
+ throws HyracksDataException {
+ int numOfPoints = AInt16SerializerDeserializer.getShort(bytes0,
+ offset0 + 1 + APolygonSerializerDeserializer.getNumberOfPointsOffset());
+
+ if (numOfPoints < 3) {
+ throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
+ ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
+ }
+
+ getCounterClockWisePolygon(bytes0, offset0, pointsOffsets0, numOfPoints);
+ int nonSimplePolygonDetection = 2 * numOfPoints;
+ int middleVertex = numOfPoints - 1;
+ int numOfTriangles = 0;
+
+ trianglesX0.reset();
+ trianglesY0.reset();
+ boolean res = false;
+ while (true) {
+ middleVertex = triangulatePolygon(bytes0, offset0, numOfPoints, pointsOffsets0, trianglesX0,
+ trianglesY0, numOfTriangles, nonSimplePolygonDetection, middleVertex);
+
+ if (middleVertex == -1) {
+ break;
+ }
+ numOfPoints--;
+ nonSimplePolygonDetection = 2 * numOfPoints;
+ numOfTriangles++;
+ int lastTriangle = (trianglesX0.length() / 3) - 1;
+
+ res = circleTriangleIntersection(bytes1, offset1, trianglesX0, trianglesY0, lastTriangle);
+ if (res) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ @Override
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+ resultStorage.reset();
+ eval0.evaluate(tuple, inputArg0);
+ eval1.evaluate(tuple, inputArg1);
+
+ if (PointableHelper.checkAndSetMissingOrNull(result, inputArg0, inputArg1)) {
+ return;
+ }
+
+ byte[] bytes0 = inputArg0.getByteArray();
+ byte[] bytes1 = inputArg1.getByteArray();
+ int offset0 = inputArg0.getStartOffset();
+ int offset1 = inputArg1.getStartOffset();
+
+ boolean res = false;
+ ATypeTag tag0 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes0[offset0]);
+ ATypeTag tag1 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]);
+
+ switch (tag0) {
+ case POINT:
+ switch (tag1) {
+ case POINT:
+ if (ascDoubleComp.compare(bytes0,
+ offset0 + 1
+ + APointSerializerDeserializer
+ .getCoordinateOffset(Coordinate.X),
+ 8, bytes1, offset1 + 1 + APointSerializerDeserializer
+ .getCoordinateOffset(Coordinate.X),
+ 8) == 0) {
+ if (ascDoubleComp
+ .compare(bytes0,
+ offset0 + 1
+ + APointSerializerDeserializer
+ .getCoordinateOffset(Coordinate.Y),
+ 8, bytes1,
+ offset1 + 1
+ + APointSerializerDeserializer
+ .getCoordinateOffset(Coordinate.Y),
+ 8) == 0) {
+ res = true;
+ }
+ }
+ break;
+ case LINE:
+ double pX = ADoubleSerializerDeserializer.getDouble(bytes0, offset0 + 1
+ + APointSerializerDeserializer.getCoordinateOffset(Coordinate.X));
+ double pY = ADoubleSerializerDeserializer.getDouble(bytes0, offset0 + 1
+ + APointSerializerDeserializer.getCoordinateOffset(Coordinate.Y));
+
+ double startX = ADoubleSerializerDeserializer.getDouble(bytes1,
+ offset1 + 1 + ALineSerializerDeserializer
+ .getStartPointCoordinateOffset(Coordinate.X));
+ double startY = ADoubleSerializerDeserializer.getDouble(bytes1,
+ offset1 + 1 + ALineSerializerDeserializer
+ .getStartPointCoordinateOffset(Coordinate.Y));
+ double endX = ADoubleSerializerDeserializer.getDouble(bytes1,
+ offset1 + 1 + ALineSerializerDeserializer
+ .getEndPointCoordinateOffset(Coordinate.X));
+ double endY = ADoubleSerializerDeserializer.getDouble(bytes1,
+ offset1 + 1 + ALineSerializerDeserializer
+ .getEndPointCoordinateOffset(Coordinate.Y));
+
+ res = pointOnLine(pX, pY, startX, startY, endX, endY);
+ break;
+ case POLYGON:
+ res = pointInPolygon(bytes0, offset0, bytes1, offset1);
+ break;
+ case CIRCLE:
+ res = pointInCircle(bytes0, offset0, bytes1, offset1);
+ break;
+ case RECTANGLE:
+ res = pointInRectangle(bytes0, offset0, bytes1, offset1);
+ break;
+ default:
+ throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
+ ATypeTag.SERIALIZED_POINT_TYPE_TAG, ATypeTag.SERIALIZED_LINE_TYPE_TAG,
+ ATypeTag.SERIALIZED_POLYGON_TYPE_TAG,
+ ATypeTag.SERIALIZED_CIRCLE_TYPE_TAG,
+ ATypeTag.SERIALIZED_RECTANGLE_TYPE_TAG);
+ }
+ break;
+ case LINE:
+ switch (tag1) {
+ case POINT:
+ double pX = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
+ + APointSerializerDeserializer.getCoordinateOffset(Coordinate.X));
+ double pY = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
+ + APointSerializerDeserializer.getCoordinateOffset(Coordinate.Y));
+
+ double startX = ADoubleSerializerDeserializer.getDouble(bytes0,
+ offset0 + 1 + ALineSerializerDeserializer
+ .getStartPointCoordinateOffset(Coordinate.X));
+ double startY = ADoubleSerializerDeserializer.getDouble(bytes0,
+ offset0 + 1 + ALineSerializerDeserializer
+ .getStartPointCoordinateOffset(Coordinate.Y));
+ double endX = ADoubleSerializerDeserializer.getDouble(bytes0,
+ offset0 + 1 + ALineSerializerDeserializer
+ .getEndPointCoordinateOffset(Coordinate.X));
+ double endY = ADoubleSerializerDeserializer.getDouble(bytes0,
+ offset0 + 1 + ALineSerializerDeserializer
+ .getEndPointCoordinateOffset(Coordinate.Y));
+
+ res = pointOnLine(pX, pY, startX, startY, endX, endY);
+ break;
+ case LINE:
+ double startX1 = ADoubleSerializerDeserializer.getDouble(bytes0,
+ offset0 + 1 + ALineSerializerDeserializer
+ .getStartPointCoordinateOffset(Coordinate.X));
+ double startY1 = ADoubleSerializerDeserializer.getDouble(bytes0,
+ offset0 + 1 + ALineSerializerDeserializer
+ .getStartPointCoordinateOffset(Coordinate.Y));
+ double endX1 = ADoubleSerializerDeserializer.getDouble(bytes0,
+ offset0 + 1 + ALineSerializerDeserializer
+ .getEndPointCoordinateOffset(Coordinate.X));
+ double endY1 = ADoubleSerializerDeserializer.getDouble(bytes0,
+ offset0 + 1 + ALineSerializerDeserializer
+ .getEndPointCoordinateOffset(Coordinate.Y));
+
+ double startX2 = ADoubleSerializerDeserializer.getDouble(bytes1,
+ offset1 + 1 + ALineSerializerDeserializer
+ .getStartPointCoordinateOffset(Coordinate.X));
+ double startY2 = ADoubleSerializerDeserializer.getDouble(bytes1,
+ offset1 + 1 + ALineSerializerDeserializer
+ .getStartPointCoordinateOffset(Coordinate.Y));
+ double endX2 = ADoubleSerializerDeserializer.getDouble(bytes1,
+ offset1 + 1 + ALineSerializerDeserializer
+ .getEndPointCoordinateOffset(Coordinate.X));
+ double endY2 = ADoubleSerializerDeserializer.getDouble(bytes1,
+ offset1 + 1 + ALineSerializerDeserializer
+ .getEndPointCoordinateOffset(Coordinate.Y));
+ res = lineLineIntersection(startX1, startY1, endX1, endY1, startX2, startY2,
+ endX2, endY2);
+ break;
+ case POLYGON:
+ res = linePolygonIntersection(bytes0, offset0, bytes1, offset1);
+ break;
+ case CIRCLE:
+ res = lineCircleIntersection(bytes0, offset0, bytes1, offset1);
+ break;
+ case RECTANGLE:
+ res = lineRectangleIntersection(bytes0, offset0, bytes1, offset1);
+ break;
+ default:
+ throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
+ ATypeTag.SERIALIZED_POINT_TYPE_TAG, ATypeTag.SERIALIZED_LINE_TYPE_TAG,
+ ATypeTag.SERIALIZED_POLYGON_TYPE_TAG,
+ ATypeTag.SERIALIZED_CIRCLE_TYPE_TAG,
+ ATypeTag.SERIALIZED_RECTANGLE_TYPE_TAG);
+ }
+ break;
+ case POLYGON:
+ switch (tag1) {
+ case POINT:
+ res = pointInPolygon(bytes1, offset1, bytes0, offset0);
+ break;
+ case LINE:
+ res = linePolygonIntersection(bytes1, offset1, bytes0, offset0);
+ break;
+ case POLYGON:
+ int numOfPoints0 = AInt16SerializerDeserializer.getShort(bytes0,
+ offset0 + 1 + APolygonSerializerDeserializer.getNumberOfPointsOffset());
+ int numOfPoints1 = AInt16SerializerDeserializer.getShort(bytes1,
+ offset1 + 1 + APolygonSerializerDeserializer.getNumberOfPointsOffset());
+
+ if (numOfPoints0 < 3 || numOfPoints1 < 3) {
+ throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
+ ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
+ }
+
+ getCounterClockWisePolygon(bytes0, offset0, pointsOffsets0, numOfPoints0);
+ getCounterClockWisePolygon(bytes1, offset1, pointsOffsets1, numOfPoints1);
+ int nonSimplePolygonDetection0 = 2 * numOfPoints0;
+ int nonSimplePolygonDetection1 = 2 * numOfPoints1;
+ boolean intersect = false;
+ int middleVertex0 = numOfPoints0 - 1;
+
+ int numOfTriangles1 = 0;
+ int middleVertex1 = numOfPoints1 - 1;
+ trianglesX1.reset();
+ trianglesY1.reset();
+ while (true) {
+ middleVertex1 = triangulatePolygon(bytes1, offset1, numOfPoints1,
+ pointsOffsets1, trianglesX1, trianglesY1, numOfTriangles1,
+ nonSimplePolygonDetection1, middleVertex1);
+
+ if (middleVertex1 == -1) {
+ break;
+ }
+
+ numOfPoints1--;
+ nonSimplePolygonDetection1 = 2 * numOfPoints1;
+ numOfTriangles1++;
+ }
+ int numOfTriangles0 = 0;
+ trianglesX0.reset();
+ trianglesY0.reset();
+ while (true) {
+ middleVertex0 = triangulatePolygon(bytes0, offset0, numOfPoints0,
+ pointsOffsets0, trianglesX0, trianglesY0, numOfTriangles0,
+ nonSimplePolygonDetection0, middleVertex0);
+
+ if (middleVertex0 == -1) {
+ break;
+ }
+ numOfPoints0--;
+ nonSimplePolygonDetection0 = 2 * numOfPoints0;
+ numOfTriangles0++;
+ int lastTriangle = (trianglesX0.length() / 3) - 1;
+
+ for (int i = 0; i < numOfTriangles1; i++) {
+
+ res = triangleTriangleIntersection(trianglesX0, trianglesY0,
+ lastTriangle, trianglesX1, trianglesY1, i);
+
+ if (res) {
+ res = triangleTriangleIntersection(trianglesX1, trianglesY1, i,
+ trianglesX0, trianglesY0, lastTriangle);
+
+ if (res) {
+ intersect = true;
+ break;
+ }
+ }
+ }
+ if (intersect) {
+ break;
+ }
+ }
+ break;
+ case CIRCLE:
+ res = polygonCircleIntersection(bytes0, offset0, bytes1, offset1);
+ break;
+ case RECTANGLE:
+ res = rectanglePolygonIntersection(bytes1, offset1, bytes0, offset0);
+ break;
+ default:
+ throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
+ ATypeTag.SERIALIZED_POINT_TYPE_TAG, ATypeTag.SERIALIZED_LINE_TYPE_TAG,
+ ATypeTag.SERIALIZED_POLYGON_TYPE_TAG,
+ ATypeTag.SERIALIZED_CIRCLE_TYPE_TAG,
+ ATypeTag.SERIALIZED_RECTANGLE_TYPE_TAG);
+ }
+ break;
+ case CIRCLE:
+ switch (tag1) {
+ case POINT:
+ res = pointInCircle(bytes1, offset1, bytes0, offset0);
+ break;
+ case LINE:
+ res = lineCircleIntersection(bytes1, offset1, bytes0, offset0);
+ break;
+ case POLYGON:
+ res = polygonCircleIntersection(bytes1, offset1, bytes0, offset0);
+ break;
+ case CIRCLE:
+ res = circleCircleIntersection(bytes0, offset0, bytes1, offset1);
+ break;
+ case RECTANGLE:
+ res = rectangleCircleIntersection(bytes1, offset1, bytes0, offset0);
+ break;
+ default:
+ throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
+ ATypeTag.SERIALIZED_POINT_TYPE_TAG, ATypeTag.SERIALIZED_LINE_TYPE_TAG,
+ ATypeTag.SERIALIZED_POLYGON_TYPE_TAG,
+ ATypeTag.SERIALIZED_CIRCLE_TYPE_TAG,
+ ATypeTag.SERIALIZED_RECTANGLE_TYPE_TAG);
+ }
+ break;
+ case RECTANGLE:
+ switch (tag1) {
+ case POINT:
+ res = pointInRectangle(bytes1, offset1, bytes0, offset0);
+ break;
+ case LINE:
+ res = lineRectangleIntersection(bytes1, offset1, bytes0, offset0);
+ break;
+ case POLYGON:
+ res = rectanglePolygonIntersection(bytes0, offset0, bytes1, offset1);
+ break;
+ case CIRCLE:
+ res = rectangleCircleIntersection(bytes0, offset0, bytes1, offset1);
+ break;
+ case RECTANGLE:
+ triangulateRectangle(bytes0, offset0, trianglesX0, trianglesY0);
+ triangulateRectangle(bytes1, offset1, trianglesX1, trianglesY1);
+
+ boolean intersect = false;
+ // 2 triangles in a rectangle
+ for (int j = 0; j < 2; j++) {
+ for (int i = 0; i < 2; i++) {
+
+ res = triangleTriangleIntersection(trianglesX1, trianglesY1, i,
+ trianglesX0, trianglesY0, j);
+
+ if (res) {
+ res = triangleTriangleIntersection(trianglesX0, trianglesY0, j,
+ trianglesX1, trianglesY1, i);
+
+ if (res) {
+ intersect = true;
+ break;
+ }
+ }
+ }
+ if (intersect) {
+ break;
+ }
+ }
+ break;
+ default:
+ throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
+ ATypeTag.SERIALIZED_POINT_TYPE_TAG, ATypeTag.SERIALIZED_LINE_TYPE_TAG,
+ ATypeTag.SERIALIZED_POLYGON_TYPE_TAG,
+ ATypeTag.SERIALIZED_CIRCLE_TYPE_TAG,
+ ATypeTag.SERIALIZED_RECTANGLE_TYPE_TAG);
+ }
+ break;
+ default:
+ throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
+ ATypeTag.SERIALIZED_POINT_TYPE_TAG, ATypeTag.SERIALIZED_LINE_TYPE_TAG,
+ ATypeTag.SERIALIZED_POLYGON_TYPE_TAG, ATypeTag.SERIALIZED_CIRCLE_TYPE_TAG,
+ ATypeTag.SERIALIZED_RECTANGLE_TYPE_TAG);
+ }
+
+ ABoolean aResult = res ? ABoolean.TRUE : ABoolean.FALSE;
+ aBooleanSerDer.serialize(aResult, out);
+ result.set(resultStorage);
+ }
+ };
+ }
+ };
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.SPATIAL_INTERSECT;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringContainsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringContainsDescriptor.java
index eeec70f..f6b2a2d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringContainsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringContainsDescriptor.java
@@ -46,7 +46,8 @@
return new AbstractBinaryStringBoolEval(ctx, args[0], args[1], BuiltinFunctions.STRING_CONTAINS,
sourceLoc) {
@Override
- protected boolean compute(UTF8StringPointable left, UTF8StringPointable right) {
+ protected boolean compute(UTF8StringPointable left, UTF8StringPointable right)
+ throws HyracksDataException {
return UTF8StringPointable.contains(left, right, false);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLengthDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLengthDescriptor.java
index 47caf14..da23ae5 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLengthDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLengthDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.runtime.evaluators.functions;
import java.io.DataOutput;
-import java.io.IOException;
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
@@ -56,13 +55,13 @@
@Override
public IScalarEvaluator createScalarEvaluator(final IEvaluatorContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
- private AMutableInt64 result = new AMutableInt64(0);
- private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
- private DataOutput out = resultStorage.getDataOutput();
- private IPointable inputArg = new VoidPointable();
- private IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
+ private final AMutableInt64 result = new AMutableInt64(0);
+ private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+ private final DataOutput out = resultStorage.getDataOutput();
+ private final IPointable inputArg = new VoidPointable();
+ private final IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
@SuppressWarnings("unchecked")
- private ISerializerDeserializer<AInt64> int64Serde =
+ private final ISerializerDeserializer<AInt64> int64Serde =
SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT64);
@Override
@@ -89,8 +88,14 @@
result.setValue(len);
int64Serde.serialize(result, out);
resultPointable.set(resultStorage);
- } catch (IOException e1) {
- throw HyracksDataException.create(e1);
+ } catch (HyracksDataException ex) {
+ if (ExceptionUtil.isStringUnicodeError(ex)) {
+ PointableHelper.setNull(resultPointable);
+ ExceptionUtil.warnFunctionEvalFailed(ctx, sourceLoc, getIdentifier(),
+ ex.getMessageNoCode());
+ return;
+ }
+ throw ex;
}
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringPositionDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringPositionDescriptor.java
index 6c06056..051083f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringPositionDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringPositionDescriptor.java
@@ -47,7 +47,8 @@
StringPositionDescriptor.this.getIdentifier(), sourceLoc) {
@Override
- protected int compute(UTF8StringPointable left, UTF8StringPointable right) {
+ protected int compute(UTF8StringPointable left, UTF8StringPointable right)
+ throws HyracksDataException {
return UTF8StringPointable.findInCodePoint(left, right, false);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringPositionOffset1Descriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringPositionOffset1Descriptor.java
index 93ada0f..668e03c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringPositionOffset1Descriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringPositionOffset1Descriptor.java
@@ -47,7 +47,8 @@
StringPositionOffset1Descriptor.this.getIdentifier(), sourceLoc) {
@Override
- protected int compute(UTF8StringPointable left, UTF8StringPointable right) {
+ protected int compute(UTF8StringPointable left, UTF8StringPointable right)
+ throws HyracksDataException {
int pos = UTF8StringPointable.findInCodePoint(left, right, false);
return pos < 0 ? pos : pos + 1;
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringToCodePointDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringToCodePointDescriptor.java
index 2f6a223..3320c2e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringToCodePointDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringToCodePointDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.runtime.evaluators.functions;
import java.io.DataOutput;
-import java.io.IOException;
import org.apache.asterix.builders.OrderedListBuilder;
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
@@ -60,14 +59,14 @@
@Override
public IScalarEvaluator createScalarEvaluator(final IEvaluatorContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
- protected final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
- protected final DataOutput out = resultStorage.getDataOutput();
- protected final IPointable argPtr = new VoidPointable();
- protected final IScalarEvaluator stringEval = args[0].createScalarEvaluator(ctx);
- protected final AOrderedListType intListType = new AOrderedListType(BuiltinType.AINT64, null);
+ private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+ private final DataOutput out = resultStorage.getDataOutput();
+ private final IPointable argPtr = new VoidPointable();
+ private final IScalarEvaluator stringEval = args[0].createScalarEvaluator(ctx);
+ private final AOrderedListType intListType = new AOrderedListType(BuiltinType.AINT64, null);
- private OrderedListBuilder listBuilder = new OrderedListBuilder();
- private ArrayBackedValueStorage inputVal = new ArrayBackedValueStorage();
+ private final OrderedListBuilder listBuilder = new OrderedListBuilder();
+ private final ArrayBackedValueStorage inputVal = new ArrayBackedValueStorage();
@SuppressWarnings("unchecked")
private final ISerializerDeserializer<AInt64> int64Serde =
@@ -109,8 +108,14 @@
}
listBuilder.write(out, true);
result.set(resultStorage);
- } catch (IOException e1) {
- throw HyracksDataException.create(e1);
+ } catch (HyracksDataException ex) {
+ if (ExceptionUtil.isStringUnicodeError(ex)) {
+ PointableHelper.setNull(result);
+ ExceptionUtil.warnFunctionEvalFailed(ctx, sourceLoc, getIdentifier(),
+ ex.getMessageNoCode());
+ return;
+ }
+ throw ex;
}
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/utils/StringTrimmer.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/utils/StringTrimmer.java
index 8dc41f5..0ddf459 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/utils/StringTrimmer.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/utils/StringTrimmer.java
@@ -22,6 +22,7 @@
import java.io.IOException;
import org.apache.asterix.runtime.evaluators.functions.StringEvaluatorUtils;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
import org.apache.hyracks.data.std.util.ByteArrayAccessibleOutputStream;
@@ -51,7 +52,7 @@
* @param resultArray
* , the byte array to hold results.
*/
- public StringTrimmer(UTF8StringBuilder resultBuilder, GrowableArray resultArray) {
+ public StringTrimmer(UTF8StringBuilder resultBuilder, GrowableArray resultArray) throws HyracksDataException {
this(resultBuilder, resultArray, null);
}
@@ -63,7 +64,8 @@
* @param pattern
* , the string that is used to construct the charset for trimming.
*/
- public StringTrimmer(UTF8StringBuilder resultBuilder, GrowableArray resultArray, UTF8StringPointable pattern) {
+ public StringTrimmer(UTF8StringBuilder resultBuilder, GrowableArray resultArray, UTF8StringPointable pattern)
+ throws HyracksDataException {
this.resultBuilder = resultBuilder;
this.resultArray = resultArray;
if (pattern != null) {
@@ -78,7 +80,7 @@
* @param patternPtr
* , a pattern string.
*/
- public void build(UTF8StringPointable patternPtr) {
+ public void build(UTF8StringPointable patternPtr) throws HyracksDataException {
final boolean newPattern = (codePointSet.size() == 0) || lastPatternPtr.compareTo(patternPtr) != 0;
if (newPattern) {
StringEvaluatorUtils.copyResetUTF8Pointable(patternPtr, lastPatternStorage, lastPatternPtr);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
index 35993cf..2b8d4e2 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
@@ -54,6 +54,7 @@
import org.apache.asterix.runtime.aggregates.scalar.ScalarSqlKurtosisDistinctAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.scalar.ScalarSqlMaxAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.scalar.ScalarSqlMaxDistinctAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.scalar.ScalarSqlMedianAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.scalar.ScalarSqlMinAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.scalar.ScalarSqlMinDistinctAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.scalar.ScalarSqlSkewnessAggregateDescriptor;
@@ -156,6 +157,7 @@
import org.apache.asterix.runtime.aggregates.std.GlobalSqlAvgAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.GlobalSqlKurtosisAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.GlobalSqlMaxAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.std.GlobalSqlMedianAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.GlobalSqlMinAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.GlobalSqlSkewnessAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.GlobalSqlStddevAggregateDescriptor;
@@ -178,6 +180,7 @@
import org.apache.asterix.runtime.aggregates.std.IntermediateSqlAvgAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.IntermediateSqlKurtosisAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.IntermediateSqlMaxAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.std.IntermediateSqlMedianAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.IntermediateSqlMinAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.IntermediateSqlSkewnessAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.IntermediateSqlStddevAggregateDescriptor;
@@ -202,6 +205,7 @@
import org.apache.asterix.runtime.aggregates.std.LocalSqlAvgAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.LocalSqlKurtosisAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.LocalSqlMaxAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.std.LocalSqlMedianAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.LocalSqlMinAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.LocalSqlSkewnessAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.LocalSqlStddevAggregateDescriptor;
@@ -224,6 +228,7 @@
import org.apache.asterix.runtime.aggregates.std.SqlCountAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.SqlKurtosisAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.SqlMaxAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.std.SqlMedianAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.SqlMinAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.SqlSkewnessAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.SqlStddevAggregateDescriptor;
@@ -446,10 +451,12 @@
import org.apache.asterix.runtime.evaluators.functions.RandomDescriptor;
import org.apache.asterix.runtime.evaluators.functions.RandomWithSeedDescriptor;
import org.apache.asterix.runtime.evaluators.functions.ReferenceTileDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.SerializedSizeDescriptor;
import org.apache.asterix.runtime.evaluators.functions.SleepDescriptor;
import org.apache.asterix.runtime.evaluators.functions.SpatialAreaDescriptor;
import org.apache.asterix.runtime.evaluators.functions.SpatialCellDescriptor;
import org.apache.asterix.runtime.evaluators.functions.SpatialDistanceDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.SpatialIntersectDescriptor;
import org.apache.asterix.runtime.evaluators.functions.StringConcatDescriptor;
import org.apache.asterix.runtime.evaluators.functions.StringContainsDescriptor;
import org.apache.asterix.runtime.evaluators.functions.StringEndsWithDescriptor;
@@ -814,6 +821,10 @@
fc.add(LocalSqlMinAggregateDescriptor.FACTORY);
fc.add(IntermediateSqlMinAggregateDescriptor.FACTORY);
fc.add(GlobalSqlMinAggregateDescriptor.FACTORY);
+ fc.add(SqlMedianAggregateDescriptor.FACTORY);
+ fc.add(LocalSqlMedianAggregateDescriptor.FACTORY);
+ fc.add(IntermediateSqlMedianAggregateDescriptor.FACTORY);
+ fc.add(GlobalSqlMedianAggregateDescriptor.FACTORY);
fc.add(SqlStddevAggregateDescriptor.FACTORY);
fc.add(LocalSqlStddevAggregateDescriptor.FACTORY);
fc.add(IntermediateSqlStddevAggregateDescriptor.FACTORY);
@@ -889,6 +900,7 @@
fc.add(ScalarSqlMaxDistinctAggregateDescriptor.FACTORY);
fc.add(ScalarSqlMinAggregateDescriptor.FACTORY);
fc.add(ScalarSqlMinDistinctAggregateDescriptor.FACTORY);
+ fc.add(ScalarSqlMedianAggregateDescriptor.FACTORY);
fc.add(ScalarSqlStddevAggregateDescriptor.FACTORY);
fc.add(ScalarSqlStddevDistinctAggregateDescriptor.FACTORY);
fc.add(ScalarSqlStddevPopAggregateDescriptor.FACTORY);
@@ -1169,6 +1181,7 @@
fc.add(LineRectanglePolygonAccessor.FACTORY);
fc.add(ReferenceTileDescriptor.FACTORY);
fc.add(GetIntersectionDescriptor.FACTORY);
+ fc.add(SpatialIntersectDescriptor.FACTORY);
// full-text function
fc.add(FullTextContainsFunctionDescriptor.FACTORY);
@@ -1315,6 +1328,7 @@
// Other functions
fc.add(DecodeDataverseNameDescriptor.FACTORY);
fc.add(RandomWithSeedDescriptor.FACTORY);
+ fc.add(SerializedSizeDescriptor.FACTORY);
ServiceLoader.load(IFunctionRegistrant.class).iterator().forEachRemaining(c -> c.register(fc));
return fc;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java
index bc763bd..7d484b6 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java
@@ -116,6 +116,9 @@
}
};
+ public static final IFunctionTypeInferer MEDIAN_MEMORY =
+ (expr, fd, context, compilerProps) -> fd.setImmutableStates(compilerProps.getSortMemoryFrames());
+
public static final class CastTypeInferer implements IFunctionTypeInferer {
@Override
public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context,
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/job/listener/NodeJobTracker.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/job/listener/NodeJobTracker.java
index a2ebd9a..02f20f8 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/job/listener/NodeJobTracker.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/job/listener/NodeJobTracker.java
@@ -36,6 +36,7 @@
import org.apache.hyracks.api.job.JobId;
import org.apache.hyracks.api.job.JobSpecification;
import org.apache.hyracks.api.job.JobStatus;
+import org.apache.hyracks.api.job.resource.IJobCapacityController;
import org.apache.hyracks.util.annotations.ThreadSafe;
@ThreadSafe
@@ -44,17 +45,19 @@
private final Map<String, Set<JobId>> nodeJobs = new HashMap<>();
@Override
- public synchronized void notifyJobCreation(JobId jobId, JobSpecification spec) {
+ public synchronized void notifyJobCreation(JobId jobId, JobSpecification spec,
+ IJobCapacityController.JobSubmissionStatus status) {
getJobParticipatingNodes(spec).stream().map(nodeJobs::get).forEach(jobsSet -> jobsSet.add(jobId));
}
@Override
- public synchronized void notifyJobStart(JobId jobId) {
+ public synchronized void notifyJobStart(JobId jobId, JobSpecification spec) {
// nothing to do
}
@Override
- public synchronized void notifyJobFinish(JobId jobId, JobStatus jobStatus, List<Exception> exceptions) {
+ public synchronized void notifyJobFinish(JobId jobId, JobSpecification spec, JobStatus jobStatus,
+ List<Exception> exceptions) {
nodeJobs.values().forEach(jobsSet -> jobsSet.remove(jobId));
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/job/resource/JobCapacityController.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/job/resource/JobCapacityController.java
index b123a5e..ae903d1 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/job/resource/JobCapacityController.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/job/resource/JobCapacityController.java
@@ -77,6 +77,11 @@
ensureMaxCapacity();
}
+ @Override
+ public IReadOnlyClusterCapacity getClusterCapacity() {
+ return resourceManager.getCurrentCapacity();
+ }
+
private void ensureMaxCapacity() {
final IClusterCapacity currentCapacity = resourceManager.getCurrentCapacity();
final IReadOnlyClusterCapacity maximumCapacity = resourceManager.getMaximumCapacity();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/DatasetStreamStats.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/DatasetStreamStats.java
new file mode 100644
index 0000000..8ea267a
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/DatasetStreamStats.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.operators;
+
+import java.util.Map;
+
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
+import org.apache.hyracks.api.job.profiling.IndexStats;
+
+/**
+ * Helper method to access stats produced by {@link DatasetStreamStatsOperatorDescriptor}
+ */
+public final class DatasetStreamStats {
+
+ private final long cardinality;
+
+ private final int avgTupleSize;
+
+ private final Map<String, IndexStats> indexesStats;
+
+ public DatasetStreamStats(IOperatorStats opStats) {
+ this.cardinality = opStats.getTupleCounter().get();
+ long totalTupleSize = opStats.getDiskIoCounter().get();
+ this.avgTupleSize = cardinality > 0 ? (int) (totalTupleSize / cardinality) : 0;
+ this.indexesStats = opStats.getIndexesStats();
+ }
+
+ static void update(IOperatorStats opStats, long tupleCount, long tupleSize, Map<String, IndexStats> indexStats) {
+ opStats.getTupleCounter().update(tupleCount);
+ opStats.getDiskIoCounter().update(tupleSize);
+ opStats.updateIndexesStats(indexStats);
+ }
+
+ public long getCardinality() {
+ return cardinality;
+ }
+
+ public int getAvgTupleSize() {
+ return avgTupleSize;
+ }
+
+ public Map<String, IndexStats> getIndexesStats() {
+ return indexesStats;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/DatasetStreamStatsOperatorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/DatasetStreamStatsOperatorDescriptor.java
new file mode 100644
index 0000000..4f2e419
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/DatasetStreamStatsOperatorDescriptor.java
@@ -0,0 +1,154 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.operators;
+
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hyracks.api.application.INCServiceContext;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
+import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
+import org.apache.hyracks.api.job.profiling.IStatsCollector;
+import org.apache.hyracks.api.job.profiling.IndexStats;
+import org.apache.hyracks.api.job.profiling.OperatorStats;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
+import org.apache.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+import org.apache.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
+import org.apache.hyracks.storage.am.common.api.IIndexDataflowHelper;
+import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.AbstractLSMWithBloomFilterDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
+
+/**
+ * Computes total tuple count and total tuple length for all input tuples,
+ * and emits these values as operator stats.
+ */
+public final class DatasetStreamStatsOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+
+ private static final long serialVersionUID = 1L;
+
+ private final String operatorName;
+ private final IIndexDataflowHelperFactory[] indexes;
+ private final String[] indexesNames;
+
+ public DatasetStreamStatsOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor rDesc,
+ String operatorName, IIndexDataflowHelperFactory[] indexes, String[] indexesNames) {
+ super(spec, 1, 1);
+ outRecDescs[0] = rDesc;
+ this.operatorName = operatorName;
+ this.indexes = indexes;
+ this.indexesNames = indexesNames;
+ }
+
+ @Override
+ public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+ IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
+
+ return new AbstractUnaryInputUnaryOutputOperatorNodePushable() {
+
+ private FrameTupleAccessor fta;
+ private long totalTupleCount;
+ private long totalTupleLength;
+ private Map<String, IndexStats> indexStats;
+
+ @Override
+ public void open() throws HyracksDataException {
+ fta = new FrameTupleAccessor(outRecDescs[0]);
+ totalTupleCount = 0;
+ writer.open();
+ IStatsCollector coll = ctx.getStatsCollector();
+ if (coll != null) {
+ coll.add(new OperatorStats(operatorName));
+ }
+ INCServiceContext serviceCtx = ctx.getJobletContext().getServiceContext();
+ indexStats = new HashMap<>();
+ for (int i = 0; i < indexes.length; i++) {
+ IIndexDataflowHelper idxFlowHelper = indexes[i].create(serviceCtx, partition);
+ try {
+ idxFlowHelper.open();
+ ILSMIndex indexInstance = (ILSMIndex) idxFlowHelper.getIndexInstance();
+ long numPages = 0;
+ synchronized (indexInstance.getOperationTracker()) {
+ for (ILSMDiskComponent component : indexInstance.getDiskComponents()) {
+ long componentSize = component.getComponentSize();
+ if (component instanceof AbstractLSMWithBloomFilterDiskComponent) {
+ componentSize -= ((AbstractLSMWithBloomFilterDiskComponent) component)
+ .getBloomFilter().getFileReference().getFile().length();
+ }
+ numPages += componentSize / indexInstance.getBufferCache().getPageSize();
+ }
+ }
+ indexStats.put(indexesNames[i], new IndexStats(indexesNames[i], numPages));
+ } finally {
+ idxFlowHelper.close();
+ }
+ }
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ fta.reset(buffer);
+ computeStats();
+ FrameUtils.flushFrame(buffer, writer);
+ }
+
+ private void computeStats() {
+ int n = fta.getTupleCount();
+ totalTupleCount += n;
+ for (int i = 0; i < n; i++) {
+ totalTupleLength += fta.getTupleLength(i);
+ }
+ }
+
+ @Override
+ public void fail() throws HyracksDataException {
+ writer.fail();
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ IStatsCollector statsCollector = ctx.getStatsCollector();
+ if (statsCollector != null) {
+ IOperatorStats stats = statsCollector.getOrAddOperatorStats(operatorName);
+ DatasetStreamStats.update(stats, totalTupleCount, totalTupleLength, indexStats);
+ }
+ writer.close();
+ }
+
+ @Override
+ public void flush() throws HyracksDataException {
+ writer.flush();
+ }
+
+ @Override
+ public String getDisplayName() {
+ return operatorName;
+ }
+ };
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMPrimaryInsertOperatorNodePushable.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMPrimaryInsertOperatorNodePushable.java
index eadb614..5c87994 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMPrimaryInsertOperatorNodePushable.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMPrimaryInsertOperatorNodePushable.java
@@ -18,6 +18,8 @@
*/
package org.apache.asterix.runtime.operators;
+import static org.apache.hyracks.storage.am.lsm.common.api.IBatchController.KEY_BATCH_CONTROLLER;
+
import java.nio.ByteBuffer;
import org.apache.asterix.common.api.INcApplicationContext;
@@ -33,6 +35,7 @@
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.exceptions.SourceLocation;
import org.apache.hyracks.api.util.CleanupUtils;
+import org.apache.hyracks.api.util.HyracksThrowingAction;
import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
@@ -50,6 +53,7 @@
import org.apache.hyracks.storage.am.common.impls.IndexAccessParameters;
import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.IBatchController;
import org.apache.hyracks.storage.am.lsm.common.api.IFrameOperationCallback;
import org.apache.hyracks.storage.am.lsm.common.api.IFrameTupleProcessor;
import org.apache.hyracks.storage.am.lsm.common.dataflow.LSMIndexInsertUpdateDeleteOperatorNodePushable;
@@ -77,6 +81,7 @@
private boolean flushedPartialTuples;
private int currentTupleIdx;
private int lastFlushedTupleIdx;
+ private IBatchController batchController;
private final PermutingFrameTupleReference keyTuple;
@@ -116,6 +121,8 @@
protected IFrameTupleProcessor createTupleProcessor(SourceLocation sourceLoc) {
return new IFrameTupleProcessor() {
+ private HyracksThrowingAction exitAction;
+
@Override
public void process(ITupleReference tuple, int index) throws HyracksDataException {
if (index < currentTupleIdx) {
@@ -219,6 +226,7 @@
(INcApplicationContext) ctx.getJobletContext().getServiceContext().getApplicationContext();
LSMIndexUtil.checkAndSetFirstLSN((AbstractLSMIndex) index,
appCtx.getTransactionSubsystem().getLogManager());
+ batchController = TaskUtil.getOrDefault(KEY_BATCH_CONTROLLER, ctx, StandardBatchController.INSTANCE);
} catch (Throwable e) { // NOSONAR: Re-thrown
throw HyracksDataException.create(e);
}
@@ -227,7 +235,7 @@
@Override
public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
accessor.reset(buffer);
- lsmAccessor.batchOperate(accessor, tuple, processor, frameOpCallback);
+ lsmAccessor.batchOperate(accessor, tuple, processor, frameOpCallback, batchController);
writeBuffer.ensureFrameSize(buffer.capacity());
if (flushedPartialTuples) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMPrimaryUpsertOperatorNodePushable.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMPrimaryUpsertOperatorNodePushable.java
index 1e7f4b7..d2dc3cf 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMPrimaryUpsertOperatorNodePushable.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMPrimaryUpsertOperatorNodePushable.java
@@ -18,6 +18,8 @@
*/
package org.apache.asterix.runtime.operators;
+import static org.apache.hyracks.storage.am.lsm.common.api.IBatchController.KEY_BATCH_CONTROLLER;
+
import java.io.DataOutput;
import java.io.IOException;
import java.nio.ByteBuffer;
@@ -62,6 +64,7 @@
import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
import org.apache.hyracks.storage.am.common.impls.IndexAccessParameters;
import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.IBatchController;
import org.apache.hyracks.storage.am.lsm.common.api.IFrameOperationCallback;
import org.apache.hyracks.storage.am.lsm.common.api.IFrameOperationCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.IFrameTupleProcessor;
@@ -117,6 +120,7 @@
private final ITracer tracer;
private final long traceCategory;
private long lastRecordInTimeStamp = 0L;
+ private IBatchController batchController;
public LSMPrimaryUpsertOperatorNodePushable(IHyracksTaskContext ctx, int partition,
IIndexDataflowHelperFactory indexHelperFactory, int[] fieldPermutation, RecordDescriptor inputRecDesc,
@@ -282,11 +286,18 @@
@Override
public void frameCompleted() throws HyracksDataException {
- appender.write(writer, true);
+ if (appender.getTupleCount() > 0) {
+ appender.write(writer, true);
+ }
callback.frameCompleted();
}
@Override
+ public void beforeExit(boolean success) throws HyracksDataException {
+ callback.beforeExit(success);
+ }
+
+ @Override
public void close() throws IOException {
callback.close();
}
@@ -302,6 +313,7 @@
}
};
frameOpCallback.open();
+ batchController = TaskUtil.getOrDefault(KEY_BATCH_CONTROLLER, ctx, StandardBatchController.INSTANCE);
} catch (Throwable e) { // NOSONAR: Re-thrown
throw HyracksDataException.create(e);
}
@@ -342,7 +354,7 @@
public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
accessor.reset(buffer);
int itemCount = accessor.getTupleCount();
- lsmAccessor.batchOperate(accessor, tuple, processor, frameOpCallback);
+ lsmAccessor.batchOperate(accessor, tuple, processor, frameOpCallback, batchController);
if (itemCount > 0) {
lastRecordInTimeStamp = System.currentTimeMillis();
}
@@ -404,7 +416,9 @@
*/
@Override
public void flushPartialFrame() throws HyracksDataException {
- appender.write(writer, true);
+ if (appender.getTupleCount() > 0) {
+ appender.write(writer, true);
+ }
}
protected void appendFilterToPrevTuple() throws IOException {
@@ -480,4 +494,5 @@
public void flush() throws HyracksDataException {
// No op since nextFrame flushes by default
}
+
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertOperatorNodePushable.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertOperatorNodePushable.java
index 955d5aa..eb4e46a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertOperatorNodePushable.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertOperatorNodePushable.java
@@ -152,4 +152,9 @@
}
return false;
}
+
+ @Override
+ public void flush() throws HyracksDataException {
+ // No op since nextFrame flushes by default
+ }
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/StandardBatchController.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/StandardBatchController.java
new file mode 100644
index 0000000..40465c6
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/StandardBatchController.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.common.api.IBatchController;
+import org.apache.hyracks.storage.am.lsm.common.api.IFrameOperationCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMHarness;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
+
+class StandardBatchController implements IBatchController {
+ static final IBatchController INSTANCE = new StandardBatchController();
+
+ private StandardBatchController() {
+ }
+
+ @Override
+ public void batchEnter(ILSMIndexOperationContext ctx, ILSMHarness lsmHarness, IFrameOperationCallback callback)
+ throws HyracksDataException {
+ lsmHarness.enter(ctx, LSMOperationType.MODIFICATION);
+ }
+
+ @Override
+ public void batchExit(ILSMIndexOperationContext ctx, ILSMHarness lsmHarness, IFrameOperationCallback callback,
+ boolean batchSuccessful) throws HyracksDataException {
+ lsmHarness.exit(ctx, callback, batchSuccessful, LSMOperationType.MODIFICATION);
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/SampleSlotRunningAggregateFunctionFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/SampleSlotRunningAggregateFunctionFactory.java
new file mode 100644
index 0000000..a4bda44
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/SampleSlotRunningAggregateFunctionFactory.java
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.runningaggregates.std;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Random;
+
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AInt32;
+import org.apache.asterix.om.base.AMutableInt32;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IRunningAggregateEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IRunningAggregateEvaluatorFactory;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+/**
+ * See {@code SampleOperationsHelper} for the sampling algorithm details.
+ */
+public class SampleSlotRunningAggregateFunctionFactory implements IRunningAggregateEvaluatorFactory {
+
+ private static final long serialVersionUID = 2L;
+
+ private final int sampleCardinalityTarget;
+
+ private final long sampleSeed;
+
+ public SampleSlotRunningAggregateFunctionFactory(int sampleCardinalityTarget, long sampleSeed) {
+ this.sampleCardinalityTarget = sampleCardinalityTarget;
+ this.sampleSeed = sampleSeed;
+ }
+
+ @Override
+ public IRunningAggregateEvaluator createRunningAggregateEvaluator(IEvaluatorContext ctx)
+ throws HyracksDataException {
+
+ int sampleCardinalityTargetPerPartition = getSampleCardinalityTargetPerPartition(sampleCardinalityTarget,
+ ctx.getTaskContext().getPartitionCount());
+
+ return new IRunningAggregateEvaluator() {
+
+ private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+ private final DataOutput resultOutput = resultStorage.getDataOutput();
+ @SuppressWarnings("unchecked")
+ private final ISerializerDeserializer<AInt32> int32Serde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32);
+ private final AMutableInt32 aInt32 = new AMutableInt32(0);
+
+ private final Random rnd = new Random(sampleSeed);
+ private long counter;
+
+ @Override
+ public void init() {
+ counter = 0;
+ }
+
+ @Override
+ public void step(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+ try {
+ counter++;
+ int outValue = evaluate();
+
+ resultStorage.reset();
+ aInt32.setValue(outValue);
+ int32Serde.serialize(aInt32, resultOutput);
+ result.set(resultStorage);
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ }
+
+ private int evaluate() {
+ if (counter <= sampleCardinalityTargetPerPartition) {
+ return (int) counter;
+ } else {
+ long v = 1 + (long) (rnd.nextDouble() * counter);
+ return v <= sampleCardinalityTargetPerPartition ? (int) v : 0;
+ }
+ }
+ };
+ }
+
+ private static int getSampleCardinalityTargetPerPartition(int sampleCardinalityTarget, int nPartitions) {
+ return Math.max(1, sampleCardinalityTarget / nPartitions + Math.min(sampleCardinalityTarget % nPartitions, 1));
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/TidRunningAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/TidRunningAggregateDescriptor.java
index 95de235..c39db40 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/TidRunningAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/TidRunningAggregateDescriptor.java
@@ -48,8 +48,7 @@
@SuppressWarnings("unchecked")
@Override
- public IRunningAggregateEvaluator createRunningAggregateEvaluator(IEvaluatorContext ctx)
- throws HyracksDataException {
+ public IRunningAggregateEvaluator createRunningAggregateEvaluator(IEvaluatorContext ctx) {
return new IRunningAggregateEvaluator() {
@@ -57,7 +56,7 @@
private final ISerializerDeserializer<AInt64> serde =
SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT64);
private final AMutableInt64 m = new AMutableInt64(0);
- private int cnt;
+ private long cnt;
@Override
public void step(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
@@ -69,7 +68,7 @@
}
@Override
- public void init() throws HyracksDataException {
+ public void init() {
cnt = 1;
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/unnestingfunctions/std/ScanCollectionDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/unnestingfunctions/std/ScanCollectionDescriptor.java
index ff6dd89..6f9dd5c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/unnestingfunctions/std/ScanCollectionDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/unnestingfunctions/std/ScanCollectionDescriptor.java
@@ -21,11 +21,11 @@
import java.io.IOException;
+import org.apache.asterix.om.exceptions.ExceptionUtil;
import org.apache.asterix.om.functions.BuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.runtime.evaluators.common.ListAccessor;
-import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.runtime.unnestingfunctions.base.AbstractUnnestingFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
@@ -52,7 +52,7 @@
@Override
public IUnnestingEvaluatorFactory createUnnestingEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
- return new ScanCollectionUnnestingFunctionFactory(args[0], sourceLoc);
+ return new ScanCollectionUnnestingFunctionFactory(args[0], sourceLoc, getIdentifier());
}
public static class ScanCollectionUnnestingFunctionFactory implements IUnnestingEvaluatorFactory {
@@ -60,10 +60,13 @@
private static final long serialVersionUID = 1L;
private IScalarEvaluatorFactory listEvalFactory;
private final SourceLocation sourceLoc;
+ private final FunctionIdentifier funID;
- public ScanCollectionUnnestingFunctionFactory(IScalarEvaluatorFactory arg, SourceLocation sourceLoc) {
+ public ScanCollectionUnnestingFunctionFactory(IScalarEvaluatorFactory arg, SourceLocation sourceLoc,
+ FunctionIdentifier funID) {
this.listEvalFactory = arg;
this.sourceLoc = sourceLoc;
+ this.funID = funID;
}
@Override
@@ -88,8 +91,9 @@
}
if (typeTag != ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG
&& typeTag != ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG) {
- throw new TypeMismatchException(sourceLoc, typeTag, ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG,
- ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG);
+ ExceptionUtil.warnTypeMismatch(ctx, sourceLoc, funID, typeTag, 0, ATypeTag.MULTISET);
+ metUnknown = true;
+ return;
}
listAccessor.reset(inputVal.getByteArray(), inputVal.getStartOffset());
itemIndex = 0;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/CcApplicationContext.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/CcApplicationContext.java
deleted file mode 100644
index 66f0e73..0000000
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/CcApplicationContext.java
+++ /dev/null
@@ -1,324 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.runtime.utils;
-
-import java.io.IOException;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-import java.util.function.Supplier;
-
-import org.apache.asterix.common.api.IConfigValidator;
-import org.apache.asterix.common.api.IConfigValidatorFactory;
-import org.apache.asterix.common.api.ICoordinationService;
-import org.apache.asterix.common.api.IMetadataLockManager;
-import org.apache.asterix.common.api.INodeJobTracker;
-import org.apache.asterix.common.api.IReceptionist;
-import org.apache.asterix.common.api.IReceptionistFactory;
-import org.apache.asterix.common.api.IRequestTracker;
-import org.apache.asterix.common.cluster.IClusterStateManager;
-import org.apache.asterix.common.cluster.IGlobalRecoveryManager;
-import org.apache.asterix.common.config.ActiveProperties;
-import org.apache.asterix.common.config.BuildProperties;
-import org.apache.asterix.common.config.CompilerProperties;
-import org.apache.asterix.common.config.ExtensionProperties;
-import org.apache.asterix.common.config.ExternalProperties;
-import org.apache.asterix.common.config.MessagingProperties;
-import org.apache.asterix.common.config.MetadataProperties;
-import org.apache.asterix.common.config.NodeProperties;
-import org.apache.asterix.common.config.PropertiesAccessor;
-import org.apache.asterix.common.config.ReplicationProperties;
-import org.apache.asterix.common.config.StorageProperties;
-import org.apache.asterix.common.config.TransactionProperties;
-import org.apache.asterix.common.context.IStorageComponentProvider;
-import org.apache.asterix.common.dataflow.ICcApplicationContext;
-import org.apache.asterix.common.external.IAdapterFactoryService;
-import org.apache.asterix.common.metadata.IMetadataBootstrap;
-import org.apache.asterix.common.metadata.IMetadataLockUtil;
-import org.apache.asterix.common.replication.INcLifecycleCoordinator;
-import org.apache.asterix.common.storage.ICompressionManager;
-import org.apache.asterix.common.transactions.IResourceIdManager;
-import org.apache.asterix.common.transactions.ITxnIdFactory;
-import org.apache.asterix.runtime.compression.CompressionManager;
-import org.apache.asterix.runtime.job.listener.NodeJobTracker;
-import org.apache.asterix.runtime.transaction.ResourceIdManager;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.api.application.ICCServiceContext;
-import org.apache.hyracks.api.client.IHyracksClientConnection;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.job.IJobLifecycleListener;
-import org.apache.hyracks.ipc.impl.HyracksConnection;
-import org.apache.hyracks.storage.common.IStorageManager;
-
-/*
- * Acts as an holder class for IndexRegistryProvider, AsterixStorageManager
- * instances that are accessed from the NCs. In addition an instance of ICCApplicationContext
- * is stored for access by the CC.
- */
-public class CcApplicationContext implements ICcApplicationContext {
-
- private ICCServiceContext ccServiceCtx;
- private IStorageComponentProvider storageComponentProvider;
- private IGlobalRecoveryManager globalRecoveryManager;
- private IResourceIdManager resourceIdManager;
- private CompilerProperties compilerProperties;
- private ExternalProperties externalProperties;
- private MetadataProperties metadataProperties;
- private StorageProperties storageProperties;
- private TransactionProperties txnProperties;
- private ActiveProperties activeProperties;
- private BuildProperties buildProperties;
- private ReplicationProperties replicationProperties;
- private ExtensionProperties extensionProperties;
- private MessagingProperties messagingProperties;
- private NodeProperties nodeProperties;
- private Supplier<IMetadataBootstrap> metadataBootstrapSupplier;
- private IHyracksClientConnection hcc;
- private Object extensionManager;
- private INcLifecycleCoordinator ftStrategy;
- private IJobLifecycleListener activeLifeCycleListener;
- private IMetadataLockManager mdLockManager;
- private IMetadataLockUtil mdLockUtil;
- private IClusterStateManager clusterStateManager;
- private final INodeJobTracker nodeJobTracker;
- private final ITxnIdFactory txnIdFactory;
- private final ICompressionManager compressionManager;
- private final IReceptionist receptionist;
- private final IRequestTracker requestTracker;
- private final IConfigValidator configValidator;
- private final IAdapterFactoryService adapterFactoryService;
- private final ReentrantReadWriteLock compilationLock = new ReentrantReadWriteLock(true);
-
- public CcApplicationContext(ICCServiceContext ccServiceCtx, IHyracksClientConnection hcc,
- Supplier<IMetadataBootstrap> metadataBootstrapSupplier, IGlobalRecoveryManager globalRecoveryManager,
- INcLifecycleCoordinator ftStrategy, IJobLifecycleListener activeLifeCycleListener,
- IStorageComponentProvider storageComponentProvider, IMetadataLockManager mdLockManager,
- IMetadataLockUtil mdLockUtil, IReceptionistFactory receptionistFactory,
- IConfigValidatorFactory configValidatorFactory, Object extensionManager,
- IAdapterFactoryService adapterFactoryService) throws AlgebricksException, IOException {
- this.ccServiceCtx = ccServiceCtx;
- this.hcc = hcc;
- this.activeLifeCycleListener = activeLifeCycleListener;
- this.extensionManager = extensionManager;
- // Determine whether to use old-style asterix-configuration.xml or new-style configuration.
- // QQQ strip this out eventually
- PropertiesAccessor propertiesAccessor = PropertiesAccessor.getInstance(ccServiceCtx.getAppConfig());
- compilerProperties = new CompilerProperties(propertiesAccessor);
- externalProperties = new ExternalProperties(propertiesAccessor);
- metadataProperties = new MetadataProperties(propertiesAccessor);
- storageProperties = new StorageProperties(propertiesAccessor);
- txnProperties = new TransactionProperties(propertiesAccessor);
- activeProperties = new ActiveProperties(propertiesAccessor);
- extensionProperties = new ExtensionProperties(propertiesAccessor);
- replicationProperties = new ReplicationProperties(propertiesAccessor);
- this.ftStrategy = ftStrategy;
- this.buildProperties = new BuildProperties(propertiesAccessor);
- this.messagingProperties = new MessagingProperties(propertiesAccessor);
- this.nodeProperties = new NodeProperties(propertiesAccessor);
- this.metadataBootstrapSupplier = metadataBootstrapSupplier;
- this.globalRecoveryManager = globalRecoveryManager;
- this.storageComponentProvider = storageComponentProvider;
- this.mdLockManager = mdLockManager;
- this.mdLockUtil = mdLockUtil;
- clusterStateManager = new ClusterStateManager();
- clusterStateManager.setCcAppCtx(this);
- this.resourceIdManager = new ResourceIdManager(clusterStateManager);
- nodeJobTracker = new NodeJobTracker();
- txnIdFactory = new BulkTxnIdFactory();
- compressionManager = new CompressionManager(storageProperties);
- receptionist = receptionistFactory.create();
- requestTracker = new RequestTracker(this);
- configValidator = configValidatorFactory.create();
- this.adapterFactoryService = adapterFactoryService;
- }
-
- @Override
- public ICCServiceContext getServiceContext() {
- return ccServiceCtx;
- }
-
- @Override
- public StorageProperties getStorageProperties() {
- return storageProperties;
- }
-
- @Override
- public TransactionProperties getTransactionProperties() {
- return txnProperties;
- }
-
- @Override
- public CompilerProperties getCompilerProperties() {
- return compilerProperties;
- }
-
- @Override
- public MetadataProperties getMetadataProperties() {
- return metadataProperties;
- }
-
- @Override
- public ExternalProperties getExternalProperties() {
- return externalProperties;
- }
-
- @Override
- public ActiveProperties getActiveProperties() {
- return activeProperties;
- }
-
- @Override
- public BuildProperties getBuildProperties() {
- return buildProperties;
- }
-
- @Override
- public IHyracksClientConnection getHcc() throws HyracksDataException {
- if (!hcc.isConnected()) {
- synchronized (this) {
- if (!hcc.isConnected()) {
- try {
- hcc = new HyracksConnection(hcc.getHost(), hcc.getPort());
- } catch (Exception e) {
- throw HyracksDataException.create(e);
- }
- }
- }
- }
- return hcc;
- }
-
- @Override
- public IStorageManager getStorageManager() {
- return RuntimeComponentsProvider.RUNTIME_PROVIDER;
- }
-
- @Override
- public ReplicationProperties getReplicationProperties() {
- return replicationProperties;
- }
-
- @Override
- public IGlobalRecoveryManager getGlobalRecoveryManager() {
- return globalRecoveryManager;
- }
-
- @Override
- public Object getExtensionManager() {
- return extensionManager;
- }
-
- @Override
- public ExtensionProperties getExtensionProperties() {
- return extensionProperties;
- }
-
- @Override
- public MessagingProperties getMessagingProperties() {
- return messagingProperties;
- }
-
- @Override
- public NodeProperties getNodeProperties() {
- return nodeProperties;
- }
-
- @Override
- public IResourceIdManager getResourceIdManager() {
- return resourceIdManager;
- }
-
- @Override
- public IMetadataBootstrap getMetadataBootstrap() {
- return metadataBootstrapSupplier.get();
- }
-
- @Override
- public INcLifecycleCoordinator getNcLifecycleCoordinator() {
- return ftStrategy;
- }
-
- @Override
- public IJobLifecycleListener getActiveNotificationHandler() {
- return activeLifeCycleListener;
- }
-
- @Override
- public IStorageComponentProvider getStorageComponentProvider() {
- return storageComponentProvider;
- }
-
- @Override
- public IMetadataLockManager getMetadataLockManager() {
- return mdLockManager;
- }
-
- @Override
- public IMetadataLockUtil getMetadataLockUtil() {
- return mdLockUtil;
- }
-
- @Override
- public IClusterStateManager getClusterStateManager() {
- return clusterStateManager;
- }
-
- @Override
- public INodeJobTracker getNodeJobTracker() {
- return nodeJobTracker;
- }
-
- @Override
- public ICoordinationService getCoordinationService() {
- return NoOpCoordinationService.INSTANCE;
- }
-
- @Override
- public ITxnIdFactory getTxnIdFactory() {
- return txnIdFactory;
- }
-
- @Override
- public ICompressionManager getCompressionManager() {
- return compressionManager;
- }
-
- @Override
- public IReceptionist getReceptionist() {
- return receptionist;
- }
-
- @Override
- public IConfigValidator getConfigValidator() {
- return configValidator;
- }
-
- @Override
- public IRequestTracker getRequestTracker() {
- return requestTracker;
- }
-
- @Override
- public IAdapterFactoryService getAdapterFactoryService() {
- return adapterFactoryService;
- }
-
- @Override
- public ReentrantReadWriteLock getCompilationLock() {
- return compilationLock;
- }
-}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java
index d3c87ff..1ff62a9 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java
@@ -289,6 +289,13 @@
return clusterPartitionConstraint;
}
+ @Override
+ public synchronized AlgebricksAbsolutePartitionConstraint getSortedClusterLocations() {
+ String[] clone = getClusterLocations().getLocations().clone();
+ Arrays.sort(clone);
+ return new AlgebricksAbsolutePartitionConstraint(clone);
+ }
+
private synchronized void resetClusterPartitionConstraint() {
ArrayList<String> clusterActiveLocations = new ArrayList<>();
for (ClusterPartition p : clusterPartitions.values()) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/RequestTracker.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/RequestTracker.java
index 333c709..a754eb6 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/RequestTracker.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/RequestTracker.java
@@ -21,26 +21,43 @@
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicLong;
import org.apache.asterix.common.api.IClientRequest;
import org.apache.asterix.common.api.IRequestTracker;
import org.apache.asterix.common.dataflow.ICcApplicationContext;
-import org.apache.commons.collections4.queue.CircularFifoQueue;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.HyracksException;
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.api.job.JobStatus;
+import org.apache.hyracks.api.job.resource.IJobCapacityController;
public class RequestTracker implements IRequestTracker {
private final Map<String, IClientRequest> runningRequests = new ConcurrentHashMap<>();
private final Map<String, IClientRequest> clientIdRequests = new ConcurrentHashMap<>();
- private final CircularFifoQueue<IClientRequest> completedRequests;
+ private final Map<String, IClientRequest> completedRequests;
private final ICcApplicationContext ccAppCtx;
+ private final AtomicLong numRequests;
public RequestTracker(ICcApplicationContext ccAppCtx) {
this.ccAppCtx = ccAppCtx;
- completedRequests = new CircularFifoQueue<>(ccAppCtx.getExternalProperties().getRequestsArchiveSize());
+ int archiveSize = ccAppCtx.getExternalProperties().getRequestsArchiveSize();
+ completedRequests = new LinkedHashMap<>(archiveSize) {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ protected boolean removeEldestEntry(Map.Entry<String, IClientRequest> eldest) {
+ return size() > archiveSize;
+ }
+ };
+ numRequests = new AtomicLong(0);
}
@Override
@@ -57,6 +74,7 @@
@Override
public void track(IClientRequest request) {
runningRequests.put(request.getId(), request);
+ numRequests.incrementAndGet();
if (request.getClientContextId() != null) {
clientIdRequests.put(request.getClientContextId(), request);
}
@@ -90,7 +108,7 @@
@Override
public synchronized Collection<IClientRequest> getCompletedRequests() {
- return Collections.unmodifiableCollection(new ArrayList<>(completedRequests));
+ return Collections.unmodifiableCollection(new ArrayList<>(completedRequests.values()));
}
private void cancel(IClientRequest request) throws HyracksDataException {
@@ -110,6 +128,55 @@
}
private synchronized void archive(IClientRequest request) {
- completedRequests.add(request);
+ completedRequests.put(request.getId(), request);
+ }
+
+ public long getTotalNumberOfRequests() {
+ return numRequests.get();
+ }
+
+ @Override
+ public void notifyJobCreation(JobId jobId, JobSpecification spec, IJobCapacityController.JobSubmissionStatus status)
+ throws HyracksException {
+ String requestId = spec.getRequestId();
+ if (requestId != null) {
+ IClientRequest request = getRequest(requestId);
+ if (request != null) {
+ request.jobCreated(jobId, spec.getRequiredClusterCapacity(), status);
+ }
+ }
+ }
+
+ @Override
+ public void notifyJobStart(JobId jobId, JobSpecification spec) throws HyracksException {
+ String requestId = spec.getRequestId();
+ if (requestId != null) {
+ IClientRequest request = getRequest(requestId);
+ if (request != null) {
+ request.jobStarted(jobId);
+ }
+ }
+ }
+
+ @Override
+ public void notifyJobFinish(JobId jobId, JobSpecification spec, JobStatus jobStatus, List<Exception> exceptions)
+ throws HyracksException {
+ String requestId = spec.getRequestId();
+ if (requestId != null) {
+ IClientRequest request = getRequest(requestId);
+ if (request != null) {
+ request.jobFinished(jobId, jobStatus, exceptions);
+ }
+ }
+ }
+
+ private IClientRequest getRequest(String requestId) {
+ IClientRequest clientRequest = runningRequests.get(requestId);
+ if (clientRequest != null) {
+ return clientRequest;
+ }
+ synchronized (this) {
+ return completedRequests.get(requestId);
+ }
}
}
diff --git a/asterixdb/asterix-runtime/src/test/java/org/apache/asterix/runtime/job/listener/NodeJobTrackerTest.java b/asterixdb/asterix-runtime/src/test/java/org/apache/asterix/runtime/job/listener/NodeJobTrackerTest.java
index be75ecb..82d8cb7 100644
--- a/asterixdb/asterix-runtime/src/test/java/org/apache/asterix/runtime/job/listener/NodeJobTrackerTest.java
+++ b/asterixdb/asterix-runtime/src/test/java/org/apache/asterix/runtime/job/listener/NodeJobTrackerTest.java
@@ -26,6 +26,7 @@
import org.apache.hyracks.api.job.JobId;
import org.apache.hyracks.api.job.JobSpecification;
import org.apache.hyracks.api.job.JobStatus;
+import org.apache.hyracks.api.job.resource.IJobCapacityController;
import org.junit.Assert;
import org.junit.Test;
import org.mockito.Mockito;
@@ -50,18 +51,18 @@
jobSpec.getUserConstraints().add(new Constraint(lValueMock, unknownLocation));
JobId jobId = new JobId(1);
- nodeJobTracker.notifyJobCreation(jobId, jobSpec);
+ nodeJobTracker.notifyJobCreation(jobId, jobSpec, IJobCapacityController.JobSubmissionStatus.EXECUTE);
// make sure nc1 has a pending job
Assert.assertTrue(nodeJobTracker.getPendingJobs(nc1).size() == 1);
Assert.assertTrue(nodeJobTracker.getPendingJobs(unknown).isEmpty());
Assert.assertTrue(nodeJobTracker.getPendingJobs(nc2).isEmpty());
- nodeJobTracker.notifyJobFinish(jobId, JobStatus.TERMINATED, null);
+ nodeJobTracker.notifyJobFinish(jobId, jobSpec, JobStatus.TERMINATED, null);
// make sure nc1 doesn't have pending jobs anymore
Assert.assertTrue(nodeJobTracker.getPendingJobs(nc1).isEmpty());
// make sure node doesn't have pending jobs after failure
jobId = new JobId(2);
- nodeJobTracker.notifyJobCreation(jobId, jobSpec);
+ nodeJobTracker.notifyJobCreation(jobId, jobSpec, IJobCapacityController.JobSubmissionStatus.EXECUTE);
Assert.assertTrue(nodeJobTracker.getPendingJobs(nc1).size() == 1);
nodeJobTracker.notifyNodeFailure(Collections.singleton(nc1));
Assert.assertTrue(nodeJobTracker.getPendingJobs(nc1).isEmpty());
diff --git a/asterixdb/asterix-server/pom.xml b/asterixdb/asterix-server/pom.xml
index 8aea41b..344f915 100644
--- a/asterixdb/asterix-server/pom.xml
+++ b/asterixdb/asterix-server/pom.xml
@@ -24,7 +24,7 @@
<parent>
<groupId>org.apache.asterix</groupId>
<artifactId>apache-asterixdb</artifactId>
- <version>0.9.8-SNAPSHOT</version>
+ <version>0.9.8.2-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/asterixdb/asterix-server/src/test/resources/NCServiceExecutionIT/cc.conf b/asterixdb/asterix-server/src/test/resources/NCServiceExecutionIT/cc.conf
index 1c6d574..2523b6f 100644
--- a/asterixdb/asterix-server/src/test/resources/NCServiceExecutionIT/cc.conf
+++ b/asterixdb/asterix-server/src/test/resources/NCServiceExecutionIT/cc.conf
@@ -46,5 +46,7 @@
heartbeat.max.misses=25
[common]
+compiler.cbo=false
+compiler.cbotest=true
log.dir = ../asterix-server/target/NCServiceExecutionIT
log.level = INFO
\ No newline at end of file
diff --git a/asterixdb/asterix-spidersilk/pom.xml b/asterixdb/asterix-spidersilk/pom.xml
index 656501f..c72f707 100644
--- a/asterixdb/asterix-spidersilk/pom.xml
+++ b/asterixdb/asterix-spidersilk/pom.xml
@@ -24,7 +24,7 @@
<parent>
<groupId>org.apache.asterix</groupId>
<artifactId>apache-asterixdb</artifactId>
- <version>0.9.8-SNAPSHOT</version>
+ <version>0.9.8.2-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/asterixdb/asterix-test-framework/pom.xml b/asterixdb/asterix-test-framework/pom.xml
index c150fae..1657bcd 100644
--- a/asterixdb/asterix-test-framework/pom.xml
+++ b/asterixdb/asterix-test-framework/pom.xml
@@ -21,7 +21,7 @@
<parent>
<artifactId>apache-asterixdb</artifactId>
<groupId>org.apache.asterix</groupId>
- <version>0.9.8-SNAPSHOT</version>
+ <version>0.9.8.2-SNAPSHOT</version>
</parent>
<artifactId>asterix-test-framework</artifactId>
diff --git a/asterixdb/asterix-tools/pom.xml b/asterixdb/asterix-tools/pom.xml
index 2924fc6..e582824 100644
--- a/asterixdb/asterix-tools/pom.xml
+++ b/asterixdb/asterix-tools/pom.xml
@@ -21,7 +21,7 @@
<parent>
<artifactId>apache-asterixdb</artifactId>
<groupId>org.apache.asterix</groupId>
- <version>0.9.8-SNAPSHOT</version>
+ <version>0.9.8.2-SNAPSHOT</version>
</parent>
<artifactId>asterix-tools</artifactId>
diff --git a/asterixdb/asterix-transactions/pom.xml b/asterixdb/asterix-transactions/pom.xml
index 2749096..a038d70 100644
--- a/asterixdb/asterix-transactions/pom.xml
+++ b/asterixdb/asterix-transactions/pom.xml
@@ -21,7 +21,7 @@
<parent>
<artifactId>apache-asterixdb</artifactId>
<groupId>org.apache.asterix</groupId>
- <version>0.9.8-SNAPSHOT</version>
+ <version>0.9.8.2-SNAPSHOT</version>
</parent>
<artifactId>asterix-transactions</artifactId>
<licenses>
@@ -34,13 +34,14 @@
</licenses>
<properties>
<root.dir>${basedir}/..</root.dir>
+ <record-manager-generator-maven-plugin.version>${project.version}</record-manager-generator-maven-plugin.version>
</properties>
<build>
<plugins>
<plugin>
<groupId>org.apache.asterix</groupId>
<artifactId>record-manager-generator-maven-plugin</artifactId>
- <version>${project.version}</version>
+ <version>${record-manager-generator-maven-plugin.version}</version>
<configuration>
<debug>false</debug>
<inputFiles>
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexOperationTrackerFactory.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexOperationTrackerFactory.java
index a104ae3..827b713 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexOperationTrackerFactory.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexOperationTrackerFactory.java
@@ -21,6 +21,7 @@
import org.apache.asterix.common.api.IDatasetLifecycleManager;
import org.apache.asterix.common.api.INcApplicationContext;
import org.apache.asterix.common.context.BaseOperationTracker;
+import org.apache.asterix.common.utils.StoragePathUtil;
import org.apache.hyracks.api.application.INCServiceContext;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.io.IJsonSerializable;
@@ -46,7 +47,8 @@
public ILSMOperationTracker getOperationTracker(INCServiceContext ctx, IResource resource) {
IDatasetLifecycleManager dslcManager =
((INcApplicationContext) ctx.getApplicationContext()).getDatasetLifecycleManager();
- return new BaseOperationTracker(datasetId, dslcManager.getDatasetInfo(datasetId));
+ int partition = StoragePathUtil.getPartitionNumFromRelativePath(resource.getPath());
+ return new BaseOperationTracker(datasetId, dslcManager.getDatasetInfo(datasetId), partition);
}
@Override
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 f9bf175..a80545d 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
@@ -46,8 +46,10 @@
import java.util.stream.Collectors;
import java.util.stream.Stream;
+import org.apache.asterix.common.api.IDatasetLifecycleManager;
import org.apache.asterix.common.dataflow.DatasetLocalResource;
import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.replication.AllDatasetsReplicationStrategy;
import org.apache.asterix.common.replication.IReplicationManager;
import org.apache.asterix.common.replication.IReplicationStrategy;
import org.apache.asterix.common.replication.ReplicationJob;
@@ -110,6 +112,7 @@
private final List<Path> storageRoots;
private final IIndexCheckpointManagerProvider indexCheckpointManagerProvider;
private final IPersistedResourceRegistry persistedResourceRegistry;
+ private IDatasetLifecycleManager datasetLifecycleManager;
public PersistentLocalResourceRepository(IIOManager ioManager,
IIndexCheckpointManagerProvider indexCheckpointManagerProvider,
@@ -346,6 +349,10 @@
}
}
+ public void setDatasetLifecycleManager(IDatasetLifecycleManager datasetLifecycleManager) {
+ this.datasetLifecycleManager = datasetLifecycleManager;
+ }
+
private void createReplicationJob(ReplicationOperation operation, FileReference fileRef)
throws HyracksDataException {
filesToBeReplicated.clear();
@@ -480,6 +487,7 @@
}
public synchronized void cleanup(int partition) throws HyracksDataException {
+ datasetLifecycleManager.waitForIO(AllDatasetsReplicationStrategy.INSTANCE, partition);
final Set<File> partitionIndexes = getPartitionIndexes(partition);
try {
for (File index : partitionIndexes) {
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManagerWithReplication.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManagerWithReplication.java
index 8a1cc65..1c614c9 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManagerWithReplication.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManagerWithReplication.java
@@ -18,6 +18,8 @@
*/
package org.apache.asterix.transaction.management.service.logging;
+import java.util.concurrent.TimeUnit;
+
import org.apache.asterix.common.exceptions.ACIDException;
import org.apache.asterix.common.replication.IReplicationManager;
import org.apache.asterix.common.replication.IReplicationStrategy;
@@ -26,6 +28,7 @@
import org.apache.asterix.common.transactions.LogSource;
import org.apache.asterix.common.transactions.LogType;
import org.apache.hyracks.api.util.InvokeUtil;
+import org.apache.logging.log4j.Logger;
import it.unimi.dsi.fastutil.longs.LongOpenHashSet;
import it.unimi.dsi.fastutil.longs.LongSet;
@@ -33,12 +36,16 @@
public class LogManagerWithReplication extends LogManager {
+ private static final Logger LOGGER = org.apache.logging.log4j.LogManager.getLogger();
private IReplicationManager replicationManager;
private IReplicationStrategy replicationStrategy;
private final LongSet replicatedTxn = LongSets.synchronize(new LongOpenHashSet());
+ private final long replicationTimeoutMillis;
public LogManagerWithReplication(ITransactionSubsystem txnSubsystem) {
super(txnSubsystem);
+ replicationTimeoutMillis = TimeUnit.SECONDS
+ .toMillis(txnSubsystem.getApplicationContext().getReplicationProperties().getReplicationTimeOut());
}
@SuppressWarnings("squid:S2445")
@@ -94,8 +101,18 @@
//wait for job Commit/Abort ACK from replicas
if (logRecord.isReplicate() && (logRecord.getLogType() == LogType.JOB_COMMIT
|| logRecord.getLogType() == LogType.ABORT)) {
+ long replicationTimeOut = replicationTimeoutMillis;
while (!logRecord.isReplicated()) {
- logRecord.wait();
+ if (replicationTimeOut <= 0) {
+ LOGGER.warn(
+ "{} ms passed without receiving acks for log {}; setting log as replicated due to timeout",
+ replicationTimeoutMillis, logRecord.getLogRecordForDisplay());
+ logRecord.setReplicated(true);
+ continue;
+ }
+ final long startTime = System.nanoTime();
+ logRecord.wait(replicationTimeOut);
+ replicationTimeOut -= TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime);
}
}
}
diff --git a/asterixdb/pom.xml b/asterixdb/pom.xml
index 94906f8..dd9c7d8 100644
--- a/asterixdb/pom.xml
+++ b/asterixdb/pom.xml
@@ -21,7 +21,7 @@
<modelVersion>4.0.0</modelVersion>
<groupId>org.apache.asterix</groupId>
<artifactId>apache-asterixdb</artifactId>
- <version>0.9.8-SNAPSHOT</version>
+ <version>0.9.8.2-SNAPSHOT</version>
<packaging>pom</packaging>
<url>${implementation.url}</url>
@@ -82,8 +82,8 @@
<skip-npm-touch.stage>none</skip-npm-touch.stage>
<!-- Versions under dependencymanagement or used in many projects via properties -->
- <algebricks.version>0.3.8-SNAPSHOT</algebricks.version>
- <hyracks.version>0.3.8-SNAPSHOT</hyracks.version>
+ <algebricks.version>0.3.8.2-SNAPSHOT</algebricks.version>
+ <hyracks.version>0.3.8.2-SNAPSHOT</hyracks.version>
<hadoop.version>3.3.6</hadoop.version>
<jacoco.version>0.7.6.201602180812</jacoco.version>
<log4j.version>2.19.0</log4j.version>
@@ -652,7 +652,14 @@
<plugin>
<groupId>net.revelc.code</groupId>
<artifactId>impsort-maven-plugin</artifactId>
- <version>1.2.0</version>
+ <version>1.9.0</version>
+ <dependencies>
+ <dependency>
+ <groupId>com.github.javaparser</groupId>
+ <artifactId>javaparser-core</artifactId>
+ <version>3.25.5</version>
+ </dependency>
+ </dependencies>
</plugin>
<plugin>
<groupId>org.codehaus.mojo</groupId>
@@ -975,16 +982,12 @@
<version>${hadoop.version}</version>
<exclusions>
<exclusion>
- <groupId>log4j</groupId>
- <artifactId>log4j</artifactId>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-reload4j</artifactId>
</exclusion>
<exclusion>
- <groupId>org.slf4j</groupId>
- <artifactId>slf4j-api</artifactId>
- </exclusion>
- <exclusion>
- <groupId>org.slf4j</groupId>
- <artifactId>slf4j-log4j12</artifactId>
+ <groupId>ch.qos.reload4j</groupId>
+ <artifactId>reload4j</artifactId>
</exclusion>
</exclusions>
</dependency>
@@ -994,12 +997,8 @@
<version>${hadoop.version}</version>
<exclusions>
<exclusion>
- <groupId>log4j</groupId>
- <artifactId>log4j</artifactId>
- </exclusion>
- <exclusion>
<groupId>org.slf4j</groupId>
- <artifactId>slf4j-api</artifactId>
+ <artifactId>slf4j-reload4j</artifactId>
</exclusion>
<exclusion>
<groupId>org.slf4j</groupId>
@@ -1049,16 +1048,12 @@
<artifactId>jdk.tools</artifactId>
</exclusion>
<exclusion>
- <groupId>log4j</groupId>
- <artifactId>log4j</artifactId>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-reload4j</artifactId>
</exclusion>
<exclusion>
- <groupId>org.slf4j</groupId>
- <artifactId>slf4j-api</artifactId>
- </exclusion>
- <exclusion>
- <groupId>org.slf4j</groupId>
- <artifactId>slf4j-log4j12</artifactId>
+ <groupId>ch.qos.reload4j</groupId>
+ <artifactId>reload4j</artifactId>
</exclusion>
<exclusion>
<groupId>com.fasterxml</groupId>
@@ -1088,16 +1083,12 @@
<version>${hadoop.version}</version>
<exclusions>
<exclusion>
- <groupId>log4j</groupId>
- <artifactId>log4j</artifactId>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-reload4j</artifactId>
</exclusion>
<exclusion>
- <groupId>org.slf4j</groupId>
- <artifactId>slf4j-api</artifactId>
- </exclusion>
- <exclusion>
- <groupId>org.slf4j</groupId>
- <artifactId>slf4j-log4j12</artifactId>
+ <groupId>ch.qos.reload4j</groupId>
+ <artifactId>reload4j</artifactId>
</exclusion>
</exclusions>
</dependency>
@@ -1116,12 +1107,8 @@
<artifactId>nimbus-jose-jwt</artifactId>
</exclusion>
<exclusion>
- <groupId>log4j</groupId>
- <artifactId>log4j</artifactId>
- </exclusion>
- <exclusion>
<groupId>org.slf4j</groupId>
- <artifactId>slf4j-api</artifactId>
+ <artifactId>slf4j-reload4j</artifactId>
</exclusion>
<exclusion>
<groupId>org.slf4j</groupId>
@@ -1143,16 +1130,12 @@
<artifactId>jdk.tools</artifactId>
</exclusion>
<exclusion>
- <groupId>log4j</groupId>
- <artifactId>log4j</artifactId>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-reload4j</artifactId>
</exclusion>
<exclusion>
- <groupId>org.slf4j</groupId>
- <artifactId>slf4j-api</artifactId>
- </exclusion>
- <exclusion>
- <groupId>org.slf4j</groupId>
- <artifactId>slf4j-log4j12</artifactId>
+ <groupId>ch.qos.reload4j</groupId>
+ <artifactId>reload4j</artifactId>
</exclusion>
<exclusion>
<groupId>org.jline</groupId>
@@ -1166,16 +1149,12 @@
<version>${hadoop.version}</version>
<exclusions>
<exclusion>
- <groupId>log4j</groupId>
- <artifactId>log4j</artifactId>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-reload4j</artifactId>
</exclusion>
<exclusion>
- <groupId>org.slf4j</groupId>
- <artifactId>slf4j-api</artifactId>
- </exclusion>
- <exclusion>
- <groupId>org.slf4j</groupId>
- <artifactId>slf4j-log4j12</artifactId>
+ <groupId>ch.qos.reload4j</groupId>
+ <artifactId>reload4j</artifactId>
</exclusion>
</exclusions>
</dependency>
@@ -1185,12 +1164,8 @@
<version>${hadoop.version}</version>
<exclusions>
<exclusion>
- <groupId>log4j</groupId>
- <artifactId>log4j</artifactId>
- </exclusion>
- <exclusion>
<groupId>org.slf4j</groupId>
- <artifactId>slf4j-api</artifactId>
+ <artifactId>slf4j-reload4j</artifactId>
</exclusion>
<exclusion>
<groupId>org.slf4j</groupId>
@@ -1213,12 +1188,8 @@
<artifactId>commons-logging</artifactId>
</exclusion>
<exclusion>
- <groupId>log4j</groupId>
- <artifactId>log4j</artifactId>
- </exclusion>
- <exclusion>
<groupId>org.slf4j</groupId>
- <artifactId>slf4j-api</artifactId>
+ <artifactId>slf4j-reload4j</artifactId>
</exclusion>
<exclusion>
<groupId>org.slf4j</groupId>
diff --git a/asterixdb/src/main/appended-resources/supplemental-models.xml b/asterixdb/src/main/appended-resources/supplemental-models.xml
index 2a7c46a..35a4076 100644
--- a/asterixdb/src/main/appended-resources/supplemental-models.xml
+++ b/asterixdb/src/main/appended-resources/supplemental-models.xml
@@ -163,9 +163,9 @@
<artifactId>netty-transport</artifactId>
<properties>
<!-- netty is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
- <license.ignoreMissingEmbeddedLicense>4.1.87.Final</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>4.1.87.Final</license.ignoreMissingEmbeddedNotice>
- <license.ignoreNoticeOverride>4.1.87.Final</license.ignoreNoticeOverride>
+ <license.ignoreMissingEmbeddedLicense>4.1.101.Final</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>4.1.101.Final</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreNoticeOverride>4.1.101.Final</license.ignoreNoticeOverride>
</properties>
</project>
</supplement>
@@ -175,9 +175,9 @@
<artifactId>netty-transport-classes-epoll</artifactId>
<properties>
<!-- netty is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
- <license.ignoreMissingEmbeddedLicense>4.1.87.Final</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>4.1.87.Final</license.ignoreMissingEmbeddedNotice>
- <license.ignoreNoticeOverride>4.1.87.Final</license.ignoreNoticeOverride>
+ <license.ignoreMissingEmbeddedLicense>4.1.101.Final</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>4.1.101.Final</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreNoticeOverride>4.1.101.Final</license.ignoreNoticeOverride>
</properties>
</project>
</supplement>
@@ -187,9 +187,9 @@
<artifactId>netty-transport-native-unix-common</artifactId>
<properties>
<!-- netty is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
- <license.ignoreMissingEmbeddedLicense>4.1.87.Final</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>4.1.87.Final</license.ignoreMissingEmbeddedNotice>
- <license.ignoreNoticeOverride>4.1.87.Final</license.ignoreNoticeOverride>
+ <license.ignoreMissingEmbeddedLicense>4.1.101.Final</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>4.1.101.Final</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreNoticeOverride>4.1.101.Final</license.ignoreNoticeOverride>
</properties>
</project>
</supplement>
@@ -199,9 +199,9 @@
<artifactId>netty-codec</artifactId>
<properties>
<!-- netty is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
- <license.ignoreMissingEmbeddedLicense>4.1.87.Final</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>4.1.87.Final</license.ignoreMissingEmbeddedNotice>
- <license.ignoreNoticeOverride>4.1.87.Final</license.ignoreNoticeOverride>
+ <license.ignoreMissingEmbeddedLicense>4.1.101.Final</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>4.1.101.Final</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreNoticeOverride>4.1.101.Final</license.ignoreNoticeOverride>
</properties>
</project>
</supplement>
@@ -211,9 +211,9 @@
<artifactId>netty-codec-dns</artifactId>
<properties>
<!-- netty is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
- <license.ignoreMissingEmbeddedLicense>4.1.87.Final</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>4.1.87.Final</license.ignoreMissingEmbeddedNotice>
- <license.ignoreNoticeOverride>4.1.87.Final</license.ignoreNoticeOverride>
+ <license.ignoreMissingEmbeddedLicense>4.1.101.Final</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>4.1.101.Final</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreNoticeOverride>4.1.101.Final</license.ignoreNoticeOverride>
</properties>
</project>
</supplement>
@@ -223,9 +223,9 @@
<artifactId>netty-codec-http2</artifactId>
<properties>
<!-- netty is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
- <license.ignoreMissingEmbeddedLicense>4.1.87.Final</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>4.1.87.Final</license.ignoreMissingEmbeddedNotice>
- <license.ignoreNoticeOverride>4.1.87.Final</license.ignoreNoticeOverride>
+ <license.ignoreMissingEmbeddedLicense>4.1.101.Final</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>4.1.101.Final</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreNoticeOverride>4.1.101.Final</license.ignoreNoticeOverride>
</properties>
</project>
</supplement>
@@ -235,9 +235,9 @@
<artifactId>netty-codec-socks</artifactId>
<properties>
<!-- netty is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
- <license.ignoreMissingEmbeddedLicense>4.1.87.Final</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>4.1.87.Final</license.ignoreMissingEmbeddedNotice>
- <license.ignoreNoticeOverride>4.1.87.Final</license.ignoreNoticeOverride>
+ <license.ignoreMissingEmbeddedLicense>4.1.101.Final</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>4.1.101.Final</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreNoticeOverride>4.1.101.Final</license.ignoreNoticeOverride>
</properties>
</project>
</supplement>
@@ -247,9 +247,9 @@
<artifactId>netty-handler</artifactId>
<properties>
<!-- netty is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
- <license.ignoreMissingEmbeddedLicense>4.1.87.Final</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>4.1.87.Final</license.ignoreMissingEmbeddedNotice>
- <license.ignoreNoticeOverride>4.1.87.Final</license.ignoreNoticeOverride>
+ <license.ignoreMissingEmbeddedLicense>4.1.101.Final</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>4.1.101.Final</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreNoticeOverride>4.1.101.Final</license.ignoreNoticeOverride>
</properties>
</project>
</supplement>
@@ -259,9 +259,9 @@
<artifactId>netty-handler-proxy</artifactId>
<properties>
<!-- netty is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
- <license.ignoreMissingEmbeddedLicense>4.1.87.Final</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>4.1.87.Final</license.ignoreMissingEmbeddedNotice>
- <license.ignoreNoticeOverride>4.1.87.Final</license.ignoreNoticeOverride>
+ <license.ignoreMissingEmbeddedLicense>4.1.101.Final</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>4.1.101.Final</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreNoticeOverride>4.1.101.Final</license.ignoreNoticeOverride>
</properties>
</project>
</supplement>
@@ -271,9 +271,9 @@
<artifactId>netty-buffer</artifactId>
<properties>
<!-- netty is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
- <license.ignoreMissingEmbeddedLicense>4.1.87.Final</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>4.1.87.Final</license.ignoreMissingEmbeddedNotice>
- <license.ignoreNoticeOverride>4.1.87.Final</license.ignoreNoticeOverride>
+ <license.ignoreMissingEmbeddedLicense>4.1.101.Final</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>4.1.101.Final</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreNoticeOverride>4.1.101.Final</license.ignoreNoticeOverride>
</properties>
</project>
</supplement>
@@ -283,9 +283,9 @@
<artifactId>netty-common</artifactId>
<properties>
<!-- netty is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
- <license.ignoreMissingEmbeddedLicense>4.1.87.Final</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>4.1.87.Final</license.ignoreMissingEmbeddedNotice>
- <license.ignoreNoticeOverride>4.1.87.Final</license.ignoreNoticeOverride>
+ <license.ignoreMissingEmbeddedLicense>4.1.101.Final</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>4.1.101.Final</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreNoticeOverride>4.1.101.Final</license.ignoreNoticeOverride>
</properties>
</project>
</supplement>
@@ -295,9 +295,9 @@
<artifactId>netty-codec-http</artifactId>
<properties>
<!-- netty is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
- <license.ignoreMissingEmbeddedLicense>4.1.87.Final</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>4.1.87.Final</license.ignoreMissingEmbeddedNotice>
- <license.ignoreNoticeOverride>4.1.87.Final</license.ignoreNoticeOverride>
+ <license.ignoreMissingEmbeddedLicense>4.1.101.Final</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>4.1.101.Final</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreNoticeOverride>4.1.101.Final</license.ignoreNoticeOverride>
</properties>
</project>
</supplement>
@@ -307,9 +307,9 @@
<artifactId>netty-resolver</artifactId>
<properties>
<!-- netty is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
- <license.ignoreMissingEmbeddedLicense>4.1.87.Final</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>4.1.87.Final</license.ignoreMissingEmbeddedNotice>
- <license.ignoreNoticeOverride>4.1.87.Final</license.ignoreNoticeOverride>
+ <license.ignoreMissingEmbeddedLicense>4.1.101.Final</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>4.1.101.Final</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreNoticeOverride>4.1.101.Final</license.ignoreNoticeOverride>
</properties>
</project>
</supplement>
@@ -319,9 +319,9 @@
<artifactId>netty-resolver-dns</artifactId>
<properties>
<!-- netty is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
- <license.ignoreMissingEmbeddedLicense>4.1.87.Final</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>4.1.87.Final</license.ignoreMissingEmbeddedNotice>
- <license.ignoreNoticeOverride>4.1.87.Final</license.ignoreNoticeOverride>
+ <license.ignoreMissingEmbeddedLicense>4.1.101.Final</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>4.1.101.Final</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreNoticeOverride>4.1.101.Final</license.ignoreNoticeOverride>
</properties>
</project>
</supplement>
@@ -331,9 +331,9 @@
<artifactId>netty-transport-native-unix-common</artifactId>
<properties>
<!-- netty is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
- <license.ignoreMissingEmbeddedLicense>4.1.87.Final</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>4.1.87.Final</license.ignoreMissingEmbeddedNotice>
- <license.ignoreNoticeOverride>4.1.87.Final</license.ignoreNoticeOverride>
+ <license.ignoreMissingEmbeddedLicense>4.1.101.Final</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>4.1.101.Final</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreNoticeOverride>4.1.101.Final</license.ignoreNoticeOverride>
</properties>
</project>
</supplement>
@@ -660,7 +660,6 @@
<license.ignoreMissingEmbeddedLicense>12.25.1</license.ignoreMissingEmbeddedLicense>
<license.ignoreMissingEmbeddedNotice>12.25.1</license.ignoreMissingEmbeddedNotice>
<license.ignoreLicenseOverride>12.25.1</license.ignoreLicenseOverride>
- <license.ignoreNoticeOverride>12.25.1</license.ignoreNoticeOverride>
</properties>
</project>
</supplement>
@@ -687,7 +686,6 @@
<license.ignoreMissingEmbeddedLicense>12.24.1</license.ignoreMissingEmbeddedLicense>
<license.ignoreMissingEmbeddedNotice>12.24.1</license.ignoreMissingEmbeddedNotice>
<license.ignoreLicenseOverride>12.24.1</license.ignoreLicenseOverride>
- <license.ignoreNoticeOverride>12.24.1</license.ignoreNoticeOverride>
</properties>
</project>
</supplement>
@@ -700,7 +698,6 @@
<license.ignoreMissingEmbeddedLicense>12.10.1</license.ignoreMissingEmbeddedLicense>
<license.ignoreMissingEmbeddedNotice>12.10.1</license.ignoreMissingEmbeddedNotice>
<license.ignoreLicenseOverride>12.10.1</license.ignoreLicenseOverride>
- <license.ignoreNoticeOverride>12.10.1</license.ignoreNoticeOverride>
</properties>
</project>
</supplement>
@@ -713,7 +710,6 @@
<license.ignoreMissingEmbeddedLicense>1.45.1</license.ignoreMissingEmbeddedLicense>
<license.ignoreMissingEmbeddedNotice>1.45.1</license.ignoreMissingEmbeddedNotice>
<license.ignoreLicenseOverride>1.45.1</license.ignoreLicenseOverride>
- <license.ignoreNoticeOverride>1.45.1</license.ignoreNoticeOverride>
</properties>
</project>
</supplement>
@@ -726,7 +722,6 @@
<license.ignoreMissingEmbeddedLicense>1.13.11</license.ignoreMissingEmbeddedLicense>
<license.ignoreMissingEmbeddedNotice>1.13.11</license.ignoreMissingEmbeddedNotice>
<license.ignoreLicenseOverride>1.13.11</license.ignoreLicenseOverride>
- <license.ignoreNoticeOverride>1.13.11</license.ignoreNoticeOverride>
</properties>
</project>
</supplement>
@@ -736,10 +731,9 @@
<groupId>com.azure</groupId>
<artifactId>azure-json</artifactId>
<properties>
- <license.ignoreMissingEmbeddedLicense>1.1.0</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>1.1.0</license.ignoreMissingEmbeddedNotice>
- <license.ignoreLicenseOverride>1.1.0</license.ignoreLicenseOverride>
- <license.ignoreNoticeOverride>1.1.0</license.ignoreNoticeOverride>
+ <license.ignoreMissingEmbeddedLicense>1.3.0</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.3.0</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>1.3.0</license.ignoreLicenseOverride>
</properties>
</project>
</supplement>
@@ -749,9 +743,10 @@
<groupId>com.azure</groupId>
<artifactId>azure-identity</artifactId>
<properties>
- <license.ignoreMissingEmbeddedLicense>1.11.1</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>1.11.1</license.ignoreMissingEmbeddedNotice>
- <license.ignoreLicenseOverride>1.11.1</license.ignoreLicenseOverride>
+ <license.ignoreMissingEmbeddedLicense>1.13.3</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.13.3</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>1.13.3</license.ignoreLicenseOverride>
+ <license.ignoreNoticeOverride>1.13.3</license.ignoreNoticeOverride>
</properties>
</project>
</supplement>
@@ -762,8 +757,9 @@
<groupId>com.microsoft.azure</groupId>
<artifactId>msal4j</artifactId>
<properties>
- <license.ignoreMissingEmbeddedLicense>1.14.0</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>1.14.0</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreMissingEmbeddedLicense>1.17.1</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.17.1</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>1.17.1</license.ignoreLicenseOverride>
</properties>
</project>
</supplement>
@@ -774,9 +770,9 @@
<groupId>com.microsoft.azure</groupId>
<artifactId>msal4j-persistence-extension</artifactId>
<properties>
- <license.ignoreMissingEmbeddedLicense>1.2.0</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>1.2.0</license.ignoreMissingEmbeddedNotice>
- <license.ignoreLicenseOverride>1.2.0</license.ignoreLicenseOverride>
+ <license.ignoreMissingEmbeddedLicense>1.3.0</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.3.0</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>1.3.0</license.ignoreLicenseOverride>
</properties>
</project>
</supplement>
@@ -888,8 +884,8 @@
<groupId>com.nimbusds</groupId>
<artifactId>oauth2-oidc-sdk</artifactId>
<properties>
- <license.ignoreMissingEmbeddedLicense>10.7.1</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>10.7.1</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreMissingEmbeddedLicense>11.18</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>11.18</license.ignoreMissingEmbeddedNotice>
</properties>
</project>
</supplement>
@@ -900,8 +896,8 @@
<groupId>com.nimbusds</groupId>
<artifactId>nimbus-jose-jwt</artifactId>
<properties>
- <license.ignoreMissingEmbeddedLicense>9.30.2</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>9.30.2</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreMissingEmbeddedLicense>9.40</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>9.40</license.ignoreMissingEmbeddedNotice>
</properties>
</project>
</supplement>
@@ -924,8 +920,8 @@
<groupId>com.nimbusds</groupId>
<artifactId>content-type</artifactId>
<properties>
- <license.ignoreMissingEmbeddedLicense>2.2</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>2.2</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreMissingEmbeddedLicense>2.3</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>2.3</license.ignoreMissingEmbeddedNotice>
</properties>
</project>
</supplement>
@@ -1379,7 +1375,7 @@
</properties>
</project>
</supplement>
-
+
<!-- io.grpc uses ALv2 -->
<supplement>
<project>
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_AzureAD_microsoft-authentication-library-for-java_refs_tags_v1.17.1_LICENSE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_AzureAD_microsoft-authentication-library-for-java_refs_tags_v1.17.1_LICENSE.txt
new file mode 100644
index 0000000..d1ca00f
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_AzureAD_microsoft-authentication-library-for-java_refs_tags_v1.17.1_LICENSE.txt
@@ -0,0 +1,21 @@
+ MIT License
+
+ Copyright (c) Microsoft Corporation. All rights reserved.
+
+ Permission is hereby granted, free of charge, to any person obtaining a copy
+ of this software and associated documentation files (the "Software"), to deal
+ in the Software without restriction, including without limitation the rights
+ to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ copies of the Software, and to permit persons to whom the Software is
+ furnished to do so, subject to the following conditions:
+
+ The above copyright notice and this permission notice shall be included in all
+ copies or substantial portions of the Software.
+
+ THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ SOFTWARE
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_Azure_azure-sdk-for-java_azure-identity_1.9.0_LICENSE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_Azure_azure-sdk-for-java_azure-identity_1.9.0_LICENSE.txt
deleted file mode 100644
index 49d2166..0000000
--- a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_Azure_azure-sdk-for-java_azure-identity_1.9.0_LICENSE.txt
+++ /dev/null
@@ -1,21 +0,0 @@
-The MIT License (MIT)
-
-Copyright (c) 2015 Microsoft
-
-Permission is hereby granted, free of charge, to any person obtaining a copy
-of this software and associated documentation files (the "Software"), to deal
-in the Software without restriction, including without limitation the rights
-to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
-copies of the Software, and to permit persons to whom the Software is
-furnished to do so, subject to the following conditions:
-
-The above copyright notice and this permission notice shall be included in all
-copies or substantial portions of the Software.
-
-THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
-IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
-FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
-AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
-LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
-OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
-SOFTWARE.
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_Azure_azure-sdk-for-java_azure-identity_1.9.0_NOTICE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_Azure_azure-sdk-for-java_azure-identity_1.9.0_NOTICE.txt
deleted file mode 100644
index ff05c53..0000000
--- a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_Azure_azure-sdk-for-java_azure-identity_1.9.0_NOTICE.txt
+++ /dev/null
@@ -1,609 +0,0 @@
-NOTICES AND INFORMATION
-Do Not Translate or Localize
-
-This software incorporates material from third parties. Microsoft makes certain
-open source code available at https://3rdpartysource.microsoft.com, or you may
-send a check or money order for US $5.00, including the product name, the open
-source component name, and version number, to:
-
-Source Code Compliance Team
-Microsoft Corporation
-One Microsoft Way
-Redmond, WA 98052
-USA
-
-Notwithstanding any other terms, you may reverse engineer this software to the
-extent required to debug changes to any libraries licensed under the GNU Lesser
-General Public License.
-
-------------------------------------------------------------------------------
-
-Azure SDK for Java uses third-party libraries or other resources that may be
-distributed under licenses different than the Azure SDK for Java software.
-
-In the event that we accidentally failed to list a required notice, please
-bring it to our attention. Post an issue or email us:
-
- azjavasdkhelp@microsoft.com
-
-The attached notices are provided for information only.
-
-License notice for Hamcrest
-------------------------------------------------------------------------------
-
-The 3-Clause BSD License
-
-Copyright (c) 2000-2015 www.hamcrest.org
-All rights reserved.
-
-Redistribution and use in source and binary forms, with or without
-modification, are permitted provided that the following conditions are met:
-
-Redistributions of source code must retain the above copyright notice, this list of
-conditions and the following disclaimer. Redistributions in binary form must reproduce
-the above copyright notice, this list of conditions and the following disclaimer in
-the documentation and/or other materials provided with the distribution.
-
-Neither the name of Hamcrest nor the names of its contributors may be used to endorse
-or promote products derived from this software without specific prior written
-permission.
-
-THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY
-EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
-OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT
-SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT,
-INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED
-TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR
-BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
-CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY
-WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH
-DAMAGE.
-
-License notice for Slf4j API
-------------------------------------------------------------------------------
-
- Copyright (c) 2004-2017 QOS.ch
- All rights reserved.
-
- Permission is hereby granted, free of charge, to any person obtaining
- a copy of this software and associated documentation files (the
- "Software"), to deal in the Software without restriction, including
- without limitation the rights to use, copy, modify, merge, publish,
- distribute, sublicense, and/or sell copies of the Software, and to
- permit persons to whom the Software is furnished to do so, subject to
- the following conditions:
-
- The above copyright notice and this permission notice shall be
- included in all copies or substantial portions of the Software.
-
- THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
- EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
- MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
- NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
- LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION
- OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION
- WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
-
-License notice for Slf4j Simple
-------------------------------------------------------------------------------
-
- Copyright (c) 2004-2017 QOS.ch
- All rights reserved.
-
- Permission is hereby granted, free of charge, to any person obtaining
- a copy of this software and associated documentation files (the
- "Software"), to deal in the Software without restriction, including
- without limitation the rights to use, copy, modify, merge, publish,
- distribute, sublicense, and/or sell copies of the Software, and to
- permit persons to whom the Software is furnished to do so, subject to
- the following conditions:
-
- The above copyright notice and this permission notice shall be
- included in all copies or substantial portions of the Software.
-
- THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
- EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
- MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
- NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
- LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION
- OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION
- WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
-
-License notice for Guava (https://github.com/google/guava)
-------------------------------------------------------------------------------
-
-Copyright (C) 2010 The Guava Authors
-
-Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except
-in compliance with the License. You may obtain a copy of the License at
-
-http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software distributed under the License
-is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
-or implied. See the License for the specific language governing permissions and limitations under
-the License.
-
-License notice for Netty
-------------------------------------------------------------------------------
-
-Copyright 2014 The Netty Project
-
-The Netty Project 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.
-
-License notice for JUG Java Uuid Generator
-------------------------------------------------------------------------------
-
-JUG Java Uuid Generator
-
-Copyright (c) 2002- Tatu Saloranta, tatu.saloranta@iki.fi
-
-Licensed under the License specified in the file LICENSE which is
-included with the source code.
-You may not use this file except in compliance with the License.
-
-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.
-
-
-License notice for Jackson
-------------------------------------------------------------------------------
-
-Copyright (c) 2007 Jackson Project
-
-Jackson-annotations (http://github.com/FasterXML/jackson
-Jackson-core (https://github.com/FasterXML/jackson-core
-jackson-databind (http://github.com/FasterXML/jackson
-Jackson-dataformat-XML (https://github.com/FasterXML/jackson-dataformat-xml)
-Jackson datatype: JSR310 (https://github.com/FasterXML/jackson-modules-java8/jackson-datatype-jsr310)
-Jackson module: Afterburner (https://github.com/FasterXML/jackson-modules-base)
-Jackson module: JAXB Annotations (https://github.com/FasterXML/jackson-modules-base)
-Woodstox (https://github.com/FasterXML/woodstox)
-
-Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except
-in compliance with the License. You may obtain a copy of the License at
-
-http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software distributed under the License
-is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
-or implied. See the License for the specific language governing permissions and limitations under
-the License.
-
-
-License notice for Metrics Core
-------------------------------------------------------------------------------
-
-Copyright (c) 2010-2013 Coda Hale, Yammer.com, 2014-2020 Dropwizard Team
-
-Metrics Core (https://github.com/dropwizard/metrics)
-
-Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except
-in compliance with the License. You may obtain a copy of the License at
-
-http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software distributed under the License
-is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
-or implied. See the License for the specific language governing permissions and limitations under
-the License.
-
-License notice for micrometer-core
-------------------------------------------------------------------------------
-
-Copyright (c) 2017-Present VMware, Inc. All Rights Reserved.
-
-micrometer-core (https://github.com/micrometer-metrics/micrometer)
-
-Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except
-in compliance with the License. You may obtain a copy of the License at
-
-http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software distributed under the License
-is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
-or implied. See the License for the specific language governing permissions and limitations under
-the License.
-
-
-License notice for project Reactor
-------------------------------------------------------------------------------
-
-Copyright (c) 2011-2017 Pivotal Software Inc, All Rights Reserved.
-
-Non-Blocking Reactive Foundation for the JVM (https://github.com/reactor/reactor-core)
-reactor-scala-extensions (https://github.com/reactor/reactor-scala-extensions)
-Reactive Streams Netty driver (https://github.com/reactor/reactor-netty)
-
-Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except
-in compliance with the License. You may obtain a copy of the License at
-
-http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software distributed under the License
-is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
-or implied. See the License for the specific language governing permissions and limitations under
-the License.
-
-
-License notice for JavaBeans Activation Framework API
-------------------------------------------------------------------------------
-
-JavaBeans Activation Framework API jar (https://github.com/eclipse-ee4j/jaf/jakarta.activation-api)
-
-Eclipse Distribution License - v 1.0
-Copyright (c) 2007, Eclipse Foundation, Inc. and its licensors.
-
-All rights reserved.
-
-Redistribution and use in source and binary forms, with or without modification,
-are permitted provided that the following conditions are met:
-
-Redistributions of source code must retain the above copyright notice,
-this list of conditions and the following disclaimer.
-Redistributions in binary form must reproduce the above copyright notice,
-this list of conditions and the following disclaimer in the documentation and/or other materials
-provided with the distribution.
-Neither the name of the Eclipse Foundation, Inc. nor the names of its contributors may be used to
-endorse or promote products derived from this software without specific prior written permission.
-THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY EXPRESS OR
-IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND
-FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS
-BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
-(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
-LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY,
-WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING
-IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
-
-
-License notice for jakarta.xml.bind-api
-------------------------------------------------------------------------------
-
-jakarta.xml.bind-api (https://github.com/eclipse-ee4j/jaxb-api/jakarta.xml.bind-api)
-
-Eclipse Distribution License - v 1.0
-Copyright (c) 2007, Eclipse Foundation, Inc. and its licensors.
-
-All rights reserved.
-
-Redistribution and use in source and binary forms, with or without modification,
-are permitted provided that the following conditions are met:
-
-Redistributions of source code must retain the above copyright notice,
-this list of conditions and the following disclaimer.
-Redistributions in binary form must reproduce the above copyright notice,
-this list of conditions and the following disclaimer in the documentation and/or other materials
-provided with the distribution.
-Neither the name of the Eclipse Foundation, Inc. nor the names of its contributors may be used to
-endorse or promote products derived from this software without specific prior written permission.
-THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY EXPRESS OR
-IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND
-FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS
-BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
-(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
-LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY,
-WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING
-IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
-
-
-License notice for jakarta.xml.bind-api
-------------------------------------------------------------------------------
-
-Stax2 API (http://github.com/FasterXML/stax2-api)
-Copyright (c) 2008 FasterXML LLC info@fasterxml.com
-
-This source code is licensed under standard BSD license, which is compatible with all Free and Open Software (OSS) licenses.
-
-
-License notice for HdrHistogrami
-------------------------------------------------------------------------------
-HdrHistogram (http://hdrhistogram.github.io/HdrHistogram/)
-
-The code in this repository code was Written by Gil Tene, Michael Barker,
-and Matt Warren, and released to the public domain, as explained at
-http://creativecommons.org/publicdomain/zero/1.0/
-
-For users of this code who wish to consume it under the "BSD" license
-rather than under the public domain or CC0 contribution text mentioned
-above, the code found under this directory is *also* provided under the
-following license (commonly referred to as the BSD 2-Clause License). This
-license does not detract from the above stated release of the code into
-the public domain, and simply represents an additional license granted by
-the Author.
-
------------------------------------------------------------------------------
-** Beginning of "BSD 2-Clause License" text. **
-
- Copyright (c) 2012, 2013, 2014, 2015, 2016 Gil Tene
- Copyright (c) 2014 Michael Barker
- Copyright (c) 2014 Matt Warren
- All rights reserved.
-
- Redistribution and use in source and binary forms, with or without
- modification, are permitted provided that the following conditions are met:
-
- 1. Redistributions of source code must retain the above copyright notice,
- this list of conditions and the following disclaimer.
-
- 2. Redistributions in binary form must reproduce the above copyright notice,
- this list of conditions and the following disclaimer in the documentation
- and/or other materials provided with the distribution.
-
- THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
- AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
- IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
- ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE
- LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
- CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
- SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
- INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
- CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
- ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF
- THE POSSIBILITY OF SUCH DAMAGE.
-
-
-License notice for LatencyUtils
-------------------------------------------------------------------------------
-
-LatencyUtils (http://latencyutils.github.io/LatencyUtils/)
-
- * This code was Written by Gil Tene of Azul Systems, and released to the
- * public domain, as explained at http://creativecommons.org/publicdomain/zero/1.0/
-
- For users of this code who wish to consume it under the "BSD" license
- rather than under the public domain or CC0 contribution text mentioned
- above, the code found under this directory is *also* provided under the
- following license (commonly referred to as the BSD 2-Clause License). This
- license does not detract from the above stated release of the code into
- the public domain, and simply represents an additional license granted by
- the Author.
-
- -----------------------------------------------------------------------------
- ** Beginning of "BSD 2-Clause License" text. **
-
- Copyright (c) 2012, 2013, 2014 Gil Tene
- All rights reserved.
-
- Redistribution and use in source and binary forms, with or without
- modification, are permitted provided that the following conditions are met:
-
- 1. Redistributions of source code must retain the above copyright notice,
- this list of conditions and the following disclaimer.
-
- 2. Redistributions in binary form must reproduce the above copyright notice,
- this list of conditions and the following disclaimer in the documentation
- and/or other materials provided with the distribution.
-
- THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
- AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
- IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
- ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE
- LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
- CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
- SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
- INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
- CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
- ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF
- THE POSSIBILITY OF SUCH DAMAGE.
-
-
-License notice for reactive-streams
-------------------------------------------------------------------------------
-
-reactive-streams (http://www.reactive-streams.org/)
-
-Copyright Statement for Contributions to the Reactive Streams Project
-=====================================================================
-
-I hereby represent that all present, past and future contributions I make to
-the Reactive Streams project (which includes all repositories owned by the
-“reactive-streams” github organization) are governed by the Creative Commons
-Zero 1.0 Universal copyright statement, placing my contributions in the public
-domain. This entails that to the extent possible under law I waive all
-copyright and related or neighboring rights to the code or documents I
-contribute. I also represent that I have the authority to perform the above
-waiver with respect to the entirety of my contributions.
-
-The text of the copyright statement is included in the COPYING file at the root
-of the reactive-streams repository at
-https://github.com/reactive-streams/reactive-streams-jvm/blob/master/COPYING.
-
-Underwriting parties:
-
-github name | Real Name, Email Address used for git commits, Company
----------------+----------------------------------------------------------------------------
-rkuhn | Roland Kuhn, rk@rkuhn.info, Typesafe Inc.
-benjchristensen| Ben Christensen, benjchristensen@gmail.com, Netflix Inc.
-viktorklang | Viktor Klang, viktor.klang@gmail.com, Typesafe Inc.
-smaldini | Stephane Maldini, stephane.maldini@gmail.com, Pivotal Software Inc.
-savulchik | Stanislav Savulchik, s.savulchik@gmail.com
-ktoso | Konrad Malawski, konrad.malawski@project13.pl, Typesafe Inc.
-ouertani | Slim Ouertani, ouertani@gmail.com
-2m | Martynas Mickevičius, mmartynas@gmail.com, Typesafe Inc.
-ldaley | Luke Daley, luke.daley@gradleware.com, Gradleware Inc.
-colinrgodsey | Colin Godsey, crgodsey@gmail.com, MediaMath Inc.
-davidmoten | Dave Moten, davidmoten@gmail.com
-briantopping | Brian Topping, brian.topping@gmail.com, Mauswerks LLC
-rstoyanchev | Rossen Stoyanchev, rstoyanchev@pivotal.io, Pivotal
-BjornHamels | Björn Hamels, bjorn@hamels.nl
-JakeWharton | Jake Wharton, jakewharton@gmail.com
-anthonyvdotbe | Anthony Vanelverdinghe, anthonyv.be@outlook.com
-seratch | Kazuhiro Sera, seratch@gmail.com, SmartNews, Inc.
-akarnokd | David Karnok, akarnokd@gmail.com
-egetman | Evgeniy Getman, getman.eugene@gmail.com
-patriknw | Patrik Nordwall, patrik.nordwall@gmail.com, Lightbend Inc
-angelsanz | Ángel Sanz, angelsanz@users.noreply.github.com
-shenghaiyang | 盛海洋, shenghaiyang@aliyun.com
-kiiadi | Kyle Thomson, kylthoms@amazon.com, Amazon.com
-jroper | James Roper, james@jazzy.id.au, Lightbend Inc.
-olegdokuka | Oleh Dokuka, shadowgun@.i.ua, Netifi Inc.
-Scottmitch | Scott Mitchell, scott_mitchell@apple.com, Apple Inc.
-retronym | Jason Zaugg, jzaugg@gmail.com, Lightbend Inc.
-
-Licensed under Public Domain (CC0)
-
-To the extent possible under law, the person who associated CC0 with
-this code has waived all copyright and related or neighboring
-rights to this code.
-
-You should have received a copy of the CC0 legalcode along with this
-work. If not, see <http://creativecommons.org/publicdomain/zero/1.0/>.
-
-License notice for Bouncy Castle
-------------------------------------------------------------------------------
-
-Copyright (c) 2000 - 2021 The Legion of the Bouncy Castle Inc. (https://www.bouncycastle.org)
-
-Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
-documentation files (the "Software"), to deal in the Software without restriction, including without limitation
-the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and
-to permit persons to whom the Software is furnished to do so, subject to the following conditions:
-
-The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software.
-
-THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE
-WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS
-OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
-TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
-
-
-License notice for KeePassJava2
-------------------------------------------------------------------------------
-
-Copyright 2015 Jo Rabin
-
-Licensed under the Apache License, Version 2.0 (the "License");
-you may not use this file except in compliance with the License.
-You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
-
-License notice for openkeepass
-------------------------------------------------------------------------------
-
-Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except
-in compliance with the License. You may obtain a copy of the License at
-
-http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software distributed under the License
-is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
-or implied. See the License for the specific language governing permissions and limitations under
-the License.
-
-Openkeepass License Link: https://github.com/cternes/openkeepass/blob/master/LICENSE.txt
--------------------------------------------------------------------------------------------------
-
-License notice for Apache Commons
-------------------------------------------------------------------------------
-
-Copyright 2002-2019 The Apache Software Foundation
-
-commons-codec (https://github.com/apache/commons-codec)
-commons-text (https://github.com/apache/commons-text)
-commons-lang (https://github.com/apache/commons-lang)
-commons-collections (https://github.com/apache/commons-collections)
-
-Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except
-in compliance with the License. You may obtain a copy of the License at
-
-http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software distributed under the License
-is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and limitations under the License.
-
--------------------------------------------------------------------------------------------------
-
-License notice for GraalVM
-------------------------------------------------------------------------------
-org.graalvm.sdk:graal-sdk - https://github.com/graalvm/native-build-tools/blob/master/common/junit-platform-native/LICENSE
-The Universal Permissive License (UPL), Version 1.0
-
-Subject to the condition set forth below, permission is hereby granted to any person obtaining a copy of this software,
-associated documentation and/or data (collectively the "Software"), free of charge and under any and all copyright rights
-in the Software, and any and all patent rights owned or freely licensable by each licensor hereunder covering either
-(i) the unmodified Software as contributed to or provided by such licensor, or (ii) the Larger Works (as defined below),
-to deal in both
-
-(a) the Software, and
-
-(b) any piece of software and/or hardware listed in the lrgrwrks.txt file if one is included with the Software each a
-"Larger Work" to which the Software is contributed by such licensors), without restriction, including without limitation
-the rights to copy, create derivative works of, display, perform, and distribute the Software and make, use, sell, offer
-for sale, import, export, have made, and have sold the Software and the Larger Work(s), and to sublicense the foregoing
-rights on either these or other terms.
-
-This license is subject to the following condition:
-
-The above copyright notice and either this complete permission notice or at a minimum a reference to the UPL must be
-included in all copies or substantial portions of the Software.
-
-THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE
-WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR
-COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE,
-ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
-
--------------------------------------------------------------------------------------------------
-
-License notice for JUnit platform native plugin
-------------------------------------------------------------------------------
-org.graalvm.buildtools:junit-platform-native - https://github.com/graalvm/native-build-tools/blob/master/common/junit-platform-native/LICENSE
-
-Copyright (c) 2020, 2021, Oracle and/or its affiliates. All rights reserved.
-DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
-
-The Universal Permissive License (UPL), Version 1.0
-
-Subject to the condition set forth below, permission is hereby granted to any
-person obtaining a copy of this software, associated documentation and/or
-data (collectively the "Software"), free of charge and under any and all
-copyright rights in the Software, and any and all patent rights owned or
-freely licensable by each licensor hereunder covering either (i) the
-unmodified Software as contributed to or provided by such licensor, or (ii)
-the Larger Works (as defined below), to deal in both
-
-(a) the Software, and
-
-(b) any piece of software and/or hardware listed in the lrgrwrks.txt file if
-one is included with the Software each a "Larger Work" to which the Software
-is contributed by such licensors),
-
-without restriction, including without limitation the rights to copy, create
-derivative works of, display, perform, and distribute the Software and make,
-use, sell, offer for sale, import, export, have made, and have sold the
-Software and the Larger Work(s), and to sublicense the foregoing rights on
-either these or other terms.
-
-This license is subject to the following condition:
-
-The above copyright notice and either this complete permission notice or at a
-minimum a reference to the UPL must be included in all copies or substantial
-portions of the Software.
-
-THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
-IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
-FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
-AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
-LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
-OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
-SOFTWARE.
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_Azure_azure-sdk-for-java_refs_tags_azure-identity_1.13.3_NOTICE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_Azure_azure-sdk-for-java_refs_tags_azure-identity_1.13.3_NOTICE.txt
new file mode 100644
index 0000000..28e74db
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_Azure_azure-sdk-for-java_refs_tags_azure-identity_1.13.3_NOTICE.txt
@@ -0,0 +1,629 @@
+NOTICES AND INFORMATION
+Do Not Translate or Localize
+
+This software incorporates material from third parties. Microsoft makes certain
+open source code available at https://3rdpartysource.microsoft.com, or you may
+send a check or money order for US $5.00, including the product name, the open
+source component name, and version number, to:
+
+Source Code Compliance Team
+Microsoft Corporation
+One Microsoft Way
+Redmond, WA 98052
+USA
+
+Notwithstanding any other terms, you may reverse engineer this software to the
+extent required to debug changes to any libraries licensed under the GNU Lesser
+General Public License.
+
+------------------------------------------------------------------------------
+
+Azure SDK for Java uses third-party libraries or other resources that may be
+distributed under licenses different than the Azure SDK for Java software.
+
+In the event that we accidentally failed to list a required notice, please
+bring it to our attention. Post an issue or email us:
+
+ azjavasdkhelp@microsoft.com
+
+The attached notices are provided for information only.
+
+License notice for Hamcrest
+------------------------------------------------------------------------------
+
+The 3-Clause BSD License
+
+Copyright (c) 2000-2015 www.hamcrest.org
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are met:
+
+Redistributions of source code must retain the above copyright notice, this list of
+conditions and the following disclaimer. Redistributions in binary form must reproduce
+the above copyright notice, this list of conditions and the following disclaimer in
+the documentation and/or other materials provided with the distribution.
+
+Neither the name of Hamcrest nor the names of its contributors may be used to endorse
+or promote products derived from this software without specific prior written
+permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY
+EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
+OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT
+SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT,
+INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED
+TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR
+BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
+CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY
+WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH
+DAMAGE.
+
+License notice for Slf4j API
+------------------------------------------------------------------------------
+
+ Copyright (c) 2004-2017 QOS.ch
+ All rights reserved.
+
+ Permission is hereby granted, free of charge, to any person obtaining
+ a copy of this software and associated documentation files (the
+ "Software"), to deal in the Software without restriction, including
+ without limitation the rights to use, copy, modify, merge, publish,
+ distribute, sublicense, and/or sell copies of the Software, and to
+ permit persons to whom the Software is furnished to do so, subject to
+ the following conditions:
+
+ The above copyright notice and this permission notice shall be
+ included in all copies or substantial portions of the Software.
+
+ THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
+ EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
+ MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
+ NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
+ LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION
+ OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION
+ WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
+
+License notice for Slf4j Simple
+------------------------------------------------------------------------------
+
+ Copyright (c) 2004-2017 QOS.ch
+ All rights reserved.
+
+ Permission is hereby granted, free of charge, to any person obtaining
+ a copy of this software and associated documentation files (the
+ "Software"), to deal in the Software without restriction, including
+ without limitation the rights to use, copy, modify, merge, publish,
+ distribute, sublicense, and/or sell copies of the Software, and to
+ permit persons to whom the Software is furnished to do so, subject to
+ the following conditions:
+
+ The above copyright notice and this permission notice shall be
+ included in all copies or substantial portions of the Software.
+
+ THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
+ EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
+ MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
+ NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
+ LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION
+ OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION
+ WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
+
+License notice for Guava (https://github.com/google/guava)
+------------------------------------------------------------------------------
+
+Copyright (C) 2010 The Guava Authors
+
+Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except
+in compliance with the License. You may obtain a copy of the License at
+
+http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software distributed under the License
+is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+or implied. See the License for the specific language governing permissions and limitations under
+the License.
+
+License notice for Netty
+------------------------------------------------------------------------------
+
+Copyright 2014 The Netty Project
+
+The Netty Project 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.
+
+License notice for JUG Java Uuid Generator
+------------------------------------------------------------------------------
+
+JUG Java Uuid Generator
+
+Copyright (c) 2002- Tatu Saloranta, tatu.saloranta@iki.fi
+
+Licensed under the License specified in the file LICENSE which is
+included with the source code.
+You may not use this file except in compliance with the License.
+
+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.
+
+
+License notice for Jackson
+------------------------------------------------------------------------------
+
+Copyright (c) 2007 Jackson Project
+
+Jackson-annotations (http://github.com/FasterXML/jackson
+Jackson-core (https://github.com/FasterXML/jackson-core
+jackson-databind (http://github.com/FasterXML/jackson
+Jackson-dataformat-XML (https://github.com/FasterXML/jackson-dataformat-xml)
+Jackson datatype: JSR310 (https://github.com/FasterXML/jackson-modules-java8/jackson-datatype-jsr310)
+Jackson module: Afterburner (https://github.com/FasterXML/jackson-modules-base)
+Jackson module: JAXB Annotations (https://github.com/FasterXML/jackson-modules-base)
+Woodstox (https://github.com/FasterXML/woodstox)
+
+Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except
+in compliance with the License. You may obtain a copy of the License at
+
+http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software distributed under the License
+is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+or implied. See the License for the specific language governing permissions and limitations under
+the License.
+
+
+License notice for Metrics Core
+------------------------------------------------------------------------------
+
+Copyright (c) 2010-2013 Coda Hale, Yammer.com, 2014-2020 Dropwizard Team
+
+Metrics Core (https://github.com/dropwizard/metrics)
+
+Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except
+in compliance with the License. You may obtain a copy of the License at
+
+http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software distributed under the License
+is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+or implied. See the License for the specific language governing permissions and limitations under
+the License.
+
+License notice for micrometer-core
+------------------------------------------------------------------------------
+
+Copyright (c) 2017-Present VMware, Inc. All Rights Reserved.
+
+micrometer-core (https://github.com/micrometer-metrics/micrometer)
+
+Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except
+in compliance with the License. You may obtain a copy of the License at
+
+http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software distributed under the License
+is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+or implied. See the License for the specific language governing permissions and limitations under
+the License.
+
+
+License notice for project Reactor
+------------------------------------------------------------------------------
+
+Copyright (c) 2011-2017 Pivotal Software Inc, All Rights Reserved.
+
+Non-Blocking Reactive Foundation for the JVM (https://github.com/reactor/reactor-core)
+reactor-scala-extensions (https://github.com/reactor/reactor-scala-extensions)
+Reactive Streams Netty driver (https://github.com/reactor/reactor-netty)
+
+Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except
+in compliance with the License. You may obtain a copy of the License at
+
+http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software distributed under the License
+is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+or implied. See the License for the specific language governing permissions and limitations under
+the License.
+
+
+License notice for JavaBeans Activation Framework API
+------------------------------------------------------------------------------
+
+JavaBeans Activation Framework API jar (https://github.com/eclipse-ee4j/jaf/jakarta.activation-api)
+
+Eclipse Distribution License - v 1.0
+Copyright (c) 2007, Eclipse Foundation, Inc. and its licensors.
+
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification,
+are permitted provided that the following conditions are met:
+
+Redistributions of source code must retain the above copyright notice,
+this list of conditions and the following disclaimer.
+Redistributions in binary form must reproduce the above copyright notice,
+this list of conditions and the following disclaimer in the documentation and/or other materials
+provided with the distribution.
+Neither the name of the Eclipse Foundation, Inc. nor the names of its contributors may be used to
+endorse or promote products derived from this software without specific prior written permission.
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY EXPRESS OR
+IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND
+FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS
+BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY,
+WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING
+IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+
+License notice for jakarta.xml.bind-api
+------------------------------------------------------------------------------
+
+jakarta.xml.bind-api (https://github.com/eclipse-ee4j/jaxb-api/jakarta.xml.bind-api)
+
+Eclipse Distribution License - v 1.0
+Copyright (c) 2007, Eclipse Foundation, Inc. and its licensors.
+
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification,
+are permitted provided that the following conditions are met:
+
+Redistributions of source code must retain the above copyright notice,
+this list of conditions and the following disclaimer.
+Redistributions in binary form must reproduce the above copyright notice,
+this list of conditions and the following disclaimer in the documentation and/or other materials
+provided with the distribution.
+Neither the name of the Eclipse Foundation, Inc. nor the names of its contributors may be used to
+endorse or promote products derived from this software without specific prior written permission.
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY EXPRESS OR
+IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND
+FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS
+BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY,
+WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING
+IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+
+License notice for jakarta.xml.bind-api
+------------------------------------------------------------------------------
+
+Stax2 API (http://github.com/FasterXML/stax2-api)
+Copyright (c) 2008 FasterXML LLC info@fasterxml.com
+
+This source code is licensed under standard BSD license, which is compatible with all Free and Open Software (OSS) licenses.
+
+
+License notice for HdrHistogrami
+------------------------------------------------------------------------------
+HdrHistogram (http://hdrhistogram.github.io/HdrHistogram/)
+
+The code in this repository code was Written by Gil Tene, Michael Barker,
+and Matt Warren, and released to the public domain, as explained at
+http://creativecommons.org/publicdomain/zero/1.0/
+
+For users of this code who wish to consume it under the "BSD" license
+rather than under the public domain or CC0 contribution text mentioned
+above, the code found under this directory is *also* provided under the
+following license (commonly referred to as the BSD 2-Clause License). This
+license does not detract from the above stated release of the code into
+the public domain, and simply represents an additional license granted by
+the Author.
+
+-----------------------------------------------------------------------------
+** Beginning of "BSD 2-Clause License" text. **
+
+ Copyright (c) 2012, 2013, 2014, 2015, 2016 Gil Tene
+ Copyright (c) 2014 Michael Barker
+ Copyright (c) 2014 Matt Warren
+ All rights reserved.
+
+ Redistribution and use in source and binary forms, with or without
+ modification, are permitted provided that the following conditions are met:
+
+ 1. Redistributions of source code must retain the above copyright notice,
+ this list of conditions and the following disclaimer.
+
+ 2. Redistributions in binary form must reproduce the above copyright notice,
+ this list of conditions and the following disclaimer in the documentation
+ and/or other materials provided with the distribution.
+
+ THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
+ AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+ IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
+ ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE
+ LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
+ CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
+ SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
+ INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
+ CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
+ ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF
+ THE POSSIBILITY OF SUCH DAMAGE.
+
+
+License notice for LatencyUtils
+------------------------------------------------------------------------------
+
+LatencyUtils (http://latencyutils.github.io/LatencyUtils/)
+
+ * This code was Written by Gil Tene of Azul Systems, and released to the
+ * public domain, as explained at http://creativecommons.org/publicdomain/zero/1.0/
+
+ For users of this code who wish to consume it under the "BSD" license
+ rather than under the public domain or CC0 contribution text mentioned
+ above, the code found under this directory is *also* provided under the
+ following license (commonly referred to as the BSD 2-Clause License). This
+ license does not detract from the above stated release of the code into
+ the public domain, and simply represents an additional license granted by
+ the Author.
+
+ -----------------------------------------------------------------------------
+ ** Beginning of "BSD 2-Clause License" text. **
+
+ Copyright (c) 2012, 2013, 2014 Gil Tene
+ All rights reserved.
+
+ Redistribution and use in source and binary forms, with or without
+ modification, are permitted provided that the following conditions are met:
+
+ 1. Redistributions of source code must retain the above copyright notice,
+ this list of conditions and the following disclaimer.
+
+ 2. Redistributions in binary form must reproduce the above copyright notice,
+ this list of conditions and the following disclaimer in the documentation
+ and/or other materials provided with the distribution.
+
+ THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
+ AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+ IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
+ ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE
+ LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
+ CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
+ SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
+ INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
+ CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
+ ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF
+ THE POSSIBILITY OF SUCH DAMAGE.
+
+
+License notice for reactive-streams
+------------------------------------------------------------------------------
+
+reactive-streams (http://www.reactive-streams.org/)
+
+Copyright Statement for Contributions to the Reactive Streams Project
+=====================================================================
+
+I hereby represent that all present, past and future contributions I make to
+the Reactive Streams project (which includes all repositories owned by the
+“reactive-streams” github organization) are governed by the Creative Commons
+Zero 1.0 Universal copyright statement, placing my contributions in the public
+domain. This entails that to the extent possible under law I waive all
+copyright and related or neighboring rights to the code or documents I
+contribute. I also represent that I have the authority to perform the above
+waiver with respect to the entirety of my contributions.
+
+The text of the copyright statement is included in the COPYING file at the root
+of the reactive-streams repository at
+https://github.com/reactive-streams/reactive-streams-jvm/blob/master/COPYING.
+
+Underwriting parties:
+
+github name | Real Name, Email Address used for git commits, Company
+---------------+----------------------------------------------------------------------------
+rkuhn | Roland Kuhn, rk@rkuhn.info, Typesafe Inc.
+benjchristensen| Ben Christensen, benjchristensen@gmail.com, Netflix Inc.
+viktorklang | Viktor Klang, viktor.klang@gmail.com, Typesafe Inc.
+smaldini | Stephane Maldini, stephane.maldini@gmail.com, Pivotal Software Inc.
+savulchik | Stanislav Savulchik, s.savulchik@gmail.com
+ktoso | Konrad Malawski, konrad.malawski@project13.pl, Typesafe Inc.
+ouertani | Slim Ouertani, ouertani@gmail.com
+2m | Martynas Mickevičius, mmartynas@gmail.com, Typesafe Inc.
+ldaley | Luke Daley, luke.daley@gradleware.com, Gradleware Inc.
+colinrgodsey | Colin Godsey, crgodsey@gmail.com, MediaMath Inc.
+davidmoten | Dave Moten, davidmoten@gmail.com
+briantopping | Brian Topping, brian.topping@gmail.com, Mauswerks LLC
+rstoyanchev | Rossen Stoyanchev, rstoyanchev@pivotal.io, Pivotal
+BjornHamels | Björn Hamels, bjorn@hamels.nl
+JakeWharton | Jake Wharton, jakewharton@gmail.com
+anthonyvdotbe | Anthony Vanelverdinghe, anthonyv.be@outlook.com
+seratch | Kazuhiro Sera, seratch@gmail.com, SmartNews, Inc.
+akarnokd | David Karnok, akarnokd@gmail.com
+egetman | Evgeniy Getman, getman.eugene@gmail.com
+patriknw | Patrik Nordwall, patrik.nordwall@gmail.com, Lightbend Inc
+angelsanz | Ángel Sanz, angelsanz@users.noreply.github.com
+shenghaiyang | 盛海洋, shenghaiyang@aliyun.com
+kiiadi | Kyle Thomson, kylthoms@amazon.com, Amazon.com
+jroper | James Roper, james@jazzy.id.au, Lightbend Inc.
+olegdokuka | Oleh Dokuka, shadowgun@.i.ua, Netifi Inc.
+Scottmitch | Scott Mitchell, scott_mitchell@apple.com, Apple Inc.
+retronym | Jason Zaugg, jzaugg@gmail.com, Lightbend Inc.
+
+Licensed under Public Domain (CC0)
+
+To the extent possible under law, the person who associated CC0 with
+this code has waived all copyright and related or neighboring
+rights to this code.
+
+You should have received a copy of the CC0 legalcode along with this
+work. If not, see <http://creativecommons.org/publicdomain/zero/1.0/>.
+
+License notice for Bouncy Castle
+------------------------------------------------------------------------------
+
+Copyright (c) 2000 - 2021 The Legion of the Bouncy Castle Inc. (https://www.bouncycastle.org)
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+documentation files (the "Software"), to deal in the Software without restriction, including without limitation
+the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and
+to permit persons to whom the Software is furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE
+WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS
+OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
+
+
+License notice for KeePassJava2
+------------------------------------------------------------------------------
+
+Copyright 2015 Jo Rabin
+
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+License notice for openkeepass
+------------------------------------------------------------------------------
+
+Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except
+in compliance with the License. You may obtain a copy of the License at
+
+http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software distributed under the License
+is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+or implied. See the License for the specific language governing permissions and limitations under
+the License.
+
+Openkeepass License Link: https://github.com/cternes/openkeepass/blob/master/LICENSE.txt
+-------------------------------------------------------------------------------------------------
+
+License notice for Apache Commons
+------------------------------------------------------------------------------
+
+Copyright 2002-2019 The Apache Software Foundation
+
+commons-codec (https://github.com/apache/commons-codec)
+commons-text (https://github.com/apache/commons-text)
+commons-lang (https://github.com/apache/commons-lang)
+commons-collections (https://github.com/apache/commons-collections)
+
+Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except
+in compliance with the License. You may obtain a copy of the License at
+
+http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software distributed under the License
+is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and limitations under the License.
+
+-------------------------------------------------------------------------------------------------
+
+License notice for GraalVM
+------------------------------------------------------------------------------
+org.graalvm.sdk:graal-sdk - https://github.com/graalvm/native-build-tools/blob/master/common/junit-platform-native/LICENSE
+The Universal Permissive License (UPL), Version 1.0
+
+Subject to the condition set forth below, permission is hereby granted to any person obtaining a copy of this software,
+associated documentation and/or data (collectively the "Software"), free of charge and under any and all copyright rights
+in the Software, and any and all patent rights owned or freely licensable by each licensor hereunder covering either
+(i) the unmodified Software as contributed to or provided by such licensor, or (ii) the Larger Works (as defined below),
+to deal in both
+
+(a) the Software, and
+
+(b) any piece of software and/or hardware listed in the lrgrwrks.txt file if one is included with the Software each a
+"Larger Work" to which the Software is contributed by such licensors), without restriction, including without limitation
+the rights to copy, create derivative works of, display, perform, and distribute the Software and make, use, sell, offer
+for sale, import, export, have made, and have sold the Software and the Larger Work(s), and to sublicense the foregoing
+rights on either these or other terms.
+
+This license is subject to the following condition:
+
+The above copyright notice and either this complete permission notice or at a minimum a reference to the UPL must be
+included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE
+WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR
+COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE,
+ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
+
+-------------------------------------------------------------------------------------------------
+
+License notice for JUnit platform native plugin
+------------------------------------------------------------------------------
+org.graalvm.buildtools:junit-platform-native - https://github.com/graalvm/native-build-tools/blob/master/common/junit-platform-native/LICENSE
+
+Copyright (c) 2020, 2021, Oracle and/or its affiliates. All rights reserved.
+DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
+
+The Universal Permissive License (UPL), Version 1.0
+
+Subject to the condition set forth below, permission is hereby granted to any
+person obtaining a copy of this software, associated documentation and/or
+data (collectively the "Software"), free of charge and under any and all
+copyright rights in the Software, and any and all patent rights owned or
+freely licensable by each licensor hereunder covering either (i) the
+unmodified Software as contributed to or provided by such licensor, or (ii)
+the Larger Works (as defined below), to deal in both
+
+(a) the Software, and
+
+(b) any piece of software and/or hardware listed in the lrgrwrks.txt file if
+one is included with the Software each a "Larger Work" to which the Software
+is contributed by such licensors),
+
+without restriction, including without limitation the rights to copy, create
+derivative works of, display, perform, and distribute the Software and make,
+use, sell, offer for sale, import, export, have made, and have sold the
+Software and the Larger Work(s), and to sublicense the foregoing rights on
+either these or other terms.
+
+This license is subject to the following condition:
+
+The above copyright notice and either this complete permission notice or at a
+minimum a reference to the UPL must be included in all copies or substantial
+portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
+-------------------------------------------------------------------------------------------------
+
+License notice for OpenTelemetry Java instrumentation
+https://github.com/open-telemetry/opentelemetry-java-instrumentation/
+------------------------------------------------------------------------------
+
+Copyright [2023] [OpenTelemetry]
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_netty_netty_netty-4.1.87.Final_NOTICE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_netty_netty_netty-4.1.101.Final_NOTICE.txt
similarity index 100%
rename from asterixdb/src/main/licenses/content/raw.githubusercontent.com_netty_netty_netty-4.1.87.Final_NOTICE.txt
rename to asterixdb/src/main/licenses/content/raw.githubusercontent.com_netty_netty_netty-4.1.101.Final_NOTICE.txt
diff --git a/hyracks-fullstack/algebricks/algebricks-common/pom.xml b/hyracks-fullstack/algebricks/algebricks-common/pom.xml
index e657319..d27ef18 100644
--- a/hyracks-fullstack/algebricks/algebricks-common/pom.xml
+++ b/hyracks-fullstack/algebricks/algebricks-common/pom.xml
@@ -25,7 +25,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>algebricks</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/hyracks-fullstack/algebricks/algebricks-compiler/pom.xml b/hyracks-fullstack/algebricks/algebricks-compiler/pom.xml
index fbe9dbc..d82a670 100644
--- a/hyracks-fullstack/algebricks/algebricks-compiler/pom.xml
+++ b/hyracks-fullstack/algebricks/algebricks-compiler/pom.xml
@@ -25,7 +25,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>algebricks</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<licenses>
@@ -59,6 +59,11 @@
</dependency>
<dependency>
<groupId>org.apache.hyracks</groupId>
+ <artifactId>algebricks-runtime</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hyracks</groupId>
<artifactId>hyracks-api</artifactId>
<version>${project.version}</version>
</dependency>
diff --git a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/AbstractCompilerFactoryBuilder.java b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/AbstractCompilerFactoryBuilder.java
index c22d54d..3a2fe3b 100644
--- a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/AbstractCompilerFactoryBuilder.java
+++ b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/AbstractCompilerFactoryBuilder.java
@@ -19,6 +19,8 @@
package org.apache.hyracks.algebricks.compiler.api;
import java.util.List;
+import java.util.function.Function;
+import java.util.function.Supplier;
import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
@@ -32,7 +34,9 @@
import org.apache.hyracks.algebricks.core.algebra.expressions.IPartialAggregationTypeComputer;
import org.apache.hyracks.algebricks.core.rewriter.base.AbstractRuleController;
import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.algebricks.core.rewriter.base.IRuleSetKind;
import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
+import org.apache.hyracks.algebricks.data.IAWriterFactory;
import org.apache.hyracks.algebricks.data.IBinaryBooleanInspectorFactory;
import org.apache.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
import org.apache.hyracks.algebricks.data.IBinaryHashFunctionFactoryProvider;
@@ -40,6 +44,7 @@
import org.apache.hyracks.algebricks.data.IBinaryIntegerInspectorFactory;
import org.apache.hyracks.algebricks.data.INormalizedKeyComputerFactoryProvider;
import org.apache.hyracks.algebricks.data.IPrinterFactoryProvider;
+import org.apache.hyracks.algebricks.data.IResultSerializerFactoryProvider;
import org.apache.hyracks.algebricks.data.ISerializerDeserializerProvider;
import org.apache.hyracks.algebricks.data.ITypeTraitProvider;
import org.apache.hyracks.algebricks.data.IUnnestingPositionWriterFactory;
@@ -49,8 +54,9 @@
public abstract class AbstractCompilerFactoryBuilder {
- protected List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> logicalRewrites;
- protected List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> physicalRewrites;
+ protected Supplier<List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>>> logicalRewrites;
+ protected Function<IRuleSetKind, List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>>> logicalRewritesByKind;
+ protected Supplier<List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>>> physicalRewrites;
protected ITypeTraitProvider typeTraitProvider;
protected ISerializerDeserializerProvider serializerDeserializerProvider;
protected IBinaryHashFunctionFactoryProvider hashFunctionFactoryProvider;
@@ -59,6 +65,8 @@
protected IBinaryBooleanInspectorFactory binaryBooleanInspectorFactory;
protected IBinaryIntegerInspectorFactory binaryIntegerInspectorFactory;
protected IPrinterFactoryProvider printerProvider;
+ protected IAWriterFactory writerFactory;
+ protected IResultSerializerFactoryProvider resultSerializerFactoryProvider;
protected IPredicateEvaluatorFactoryProvider predEvaluatorFactoryProvider;
protected IExpressionRuntimeProvider expressionRuntimeProvider;
protected IExpressionTypeComputer expressionTypeComputer;
@@ -78,11 +86,18 @@
public abstract ICompilerFactory create();
- public void setLogicalRewrites(List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> logicalRewrites) {
+ public void setLogicalRewrites(
+ Supplier<List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>>> logicalRewrites) {
this.logicalRewrites = logicalRewrites;
}
- public void setPhysicalRewrites(List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> physicalRewrites) {
+ public void setLogicalRewritesByKind(
+ Function<IRuleSetKind, List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>>> logicalRewritesByKind) {
+ this.logicalRewritesByKind = logicalRewritesByKind;
+ }
+
+ public void setPhysicalRewrites(
+ Supplier<List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>>> physicalRewrites) {
this.physicalRewrites = physicalRewrites;
}
@@ -158,6 +173,22 @@
return printerProvider;
}
+ public void setWriterFactory(IAWriterFactory writerFactory) {
+ this.writerFactory = writerFactory;
+ }
+
+ public IAWriterFactory getWriterFactory() {
+ return writerFactory;
+ }
+
+ public void setResultSerializerFactoryProvider(IResultSerializerFactoryProvider resultSerializerFactoryProvider) {
+ this.resultSerializerFactoryProvider = resultSerializerFactoryProvider;
+ }
+
+ public IResultSerializerFactoryProvider getResultSerializerFactoryProvider() {
+ return resultSerializerFactoryProvider;
+ }
+
public void setExpressionRuntimeProvider(IExpressionRuntimeProvider expressionRuntimeProvider) {
this.expressionRuntimeProvider = expressionRuntimeProvider;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java
index 891980f..85910aa 100644
--- a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java
+++ b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java
@@ -18,8 +18,11 @@
*/
package org.apache.hyracks.algebricks.compiler.api;
+import java.util.List;
+
import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalPlan;
import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
import org.apache.hyracks.algebricks.core.algebra.expressions.IConflictingTypeResolver;
@@ -33,10 +36,15 @@
import org.apache.hyracks.algebricks.core.config.AlgebricksConfig;
import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
import org.apache.hyracks.algebricks.core.jobgen.impl.PlanCompiler;
+import org.apache.hyracks.algebricks.core.rewriter.base.AbstractRuleController;
import org.apache.hyracks.algebricks.core.rewriter.base.AlgebricksOptimizationContext;
import org.apache.hyracks.algebricks.core.rewriter.base.HeuristicOptimizer;
+import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
import org.apache.hyracks.algebricks.core.rewriter.base.IOptimizationContextFactory;
+import org.apache.hyracks.algebricks.core.rewriter.base.IRuleSetKind;
import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
+import org.apache.hyracks.algebricks.data.IAWriterFactory;
+import org.apache.hyracks.algebricks.runtime.writers.SerializedDataWriterFactory;
import org.apache.hyracks.api.exceptions.IWarningCollector;
import org.apache.hyracks.api.job.IJobletEventListenerFactory;
import org.apache.hyracks.api.job.JobSpecification;
@@ -58,14 +66,19 @@
IConflictingTypeResolver conflictingTypeResolver, PhysicalOptimizationConfig physicalOptimizationConfig,
AlgebricksPartitionConstraint clusterLocations, IWarningCollector warningCollector) {
IPlanPrettyPrinter prettyPrinter = PlanPrettyPrinter.createStringPlanPrettyPrinter();
- return new AlgebricksOptimizationContext(varCounter, expressionEvalSizeComputer,
+ return new AlgebricksOptimizationContext(this, varCounter, expressionEvalSizeComputer,
mergeAggregationExpressionFactory, expressionTypeComputer, missableTypeComputer,
conflictingTypeResolver, physicalOptimizationConfig, clusterLocations, prettyPrinter,
warningCollector);
}
+
+ @Override
+ public IOptimizationContext cloneOptimizationContext(IOptimizationContext oc) {
+ return new AlgebricksOptimizationContext((AlgebricksOptimizationContext) oc);
+ }
}
- private IOptimizationContextFactory optCtxFactory;
+ private final IOptimizationContextFactory optCtxFactory;
public HeuristicCompilerFactoryBuilder() {
this.optCtxFactory = DefaultOptimizationContextFactory.INSTANCE;
@@ -77,42 +90,90 @@
@Override
public ICompilerFactory create() {
- return new ICompilerFactory() {
- @Override
- public ICompiler createCompiler(final ILogicalPlan plan, final IMetadataProvider<?, ?> metadata,
- int varCounter) {
- final IOptimizationContext oc = optCtxFactory.createOptimizationContext(varCounter,
- expressionEvalSizeComputer, mergeAggregationExpressionFactory, expressionTypeComputer,
- missableTypeComputer, conflictingTypeResolver, physicalOptimizationConfig, clusterLocations,
- warningCollector);
- oc.setMetadataDeclarations(metadata);
- final HeuristicOptimizer opt = new HeuristicOptimizer(plan, logicalRewrites, physicalRewrites, oc);
- return new ICompiler() {
-
- @Override
- public void optimize() throws AlgebricksException {
- opt.optimize();
- }
-
- @Override
- public JobSpecification createJob(Object appContext,
- IJobletEventListenerFactory jobEventListenerFactory) throws AlgebricksException {
- AlgebricksConfig.ALGEBRICKS_LOGGER.trace("Starting Job Generation.\n");
- JobGenContext context = new JobGenContext(null, metadata, appContext,
- serializerDeserializerProvider, hashFunctionFactoryProvider, hashFunctionFamilyProvider,
- comparatorFactoryProvider, typeTraitProvider, binaryBooleanInspectorFactory,
- binaryIntegerInspectorFactory, printerProvider, missingWriterFactory, nullWriterFactory,
- unnestingPositionWriterFactory, normalizedKeyComputerFactoryProvider,
- expressionRuntimeProvider, expressionTypeComputer, oc, expressionEvalSizeComputer,
- partialAggregationTypeComputer, predEvaluatorFactoryProvider,
- physicalOptimizationConfig.getFrameSize(), clusterLocations, warningCollector,
- maxWarnings, physicalOptimizationConfig);
- PlanCompiler pc = new PlanCompiler(context);
- return pc.compilePlan(plan, jobEventListenerFactory);
- }
- };
- }
- };
+ return new CompilerFactoryImpl();
}
+ private class CompilerFactoryImpl implements ICompilerFactory {
+ @Override
+ public ICompiler createCompiler(ILogicalPlan plan, IMetadataProvider<?, ?> metadata, int varCounter) {
+ IOptimizationContext optContext =
+ optCtxFactory.createOptimizationContext(varCounter, expressionEvalSizeComputer,
+ mergeAggregationExpressionFactory, expressionTypeComputer, missableTypeComputer,
+ conflictingTypeResolver, physicalOptimizationConfig, clusterLocations, warningCollector);
+ optContext.setMetadataDeclarations(metadata);
+ optContext.setCompilerFactory(this);
+ return new CompilerImpl(this, plan, optContext, logicalRewrites.get(), physicalRewrites.get(),
+ writerFactory);
+ }
+
+ @Override
+ public ICompiler createCompiler(ILogicalPlan plan, IOptimizationContext newOptContext,
+ IRuleSetKind ruleSetKind) {
+ if (newOptContext.getCompilerFactory() != this) {
+ throw new IllegalStateException();
+ }
+ return new CompilerImpl(this, plan, newOptContext, logicalRewritesByKind.apply(ruleSetKind),
+ physicalRewrites.get(), SerializedDataWriterFactory.WITHOUT_RECORD_DESCRIPTOR);
+ }
+
+ private PlanCompiler createPlanCompiler(IOptimizationContext oc, Object appContext,
+ IAWriterFactory writerFactory) {
+ JobGenContext context = new JobGenContext(null, oc.getMetadataProvider(), appContext,
+ serializerDeserializerProvider, hashFunctionFactoryProvider, hashFunctionFamilyProvider,
+ comparatorFactoryProvider, typeTraitProvider, binaryBooleanInspectorFactory,
+ binaryIntegerInspectorFactory, printerProvider, writerFactory, resultSerializerFactoryProvider,
+ missingWriterFactory, nullWriterFactory, unnestingPositionWriterFactory,
+ normalizedKeyComputerFactoryProvider, expressionRuntimeProvider, expressionTypeComputer, oc,
+ expressionEvalSizeComputer, partialAggregationTypeComputer, predEvaluatorFactoryProvider,
+ physicalOptimizationConfig.getFrameSize(), clusterLocations, warningCollector, maxWarnings,
+ physicalOptimizationConfig);
+ return new PlanCompiler(context);
+ }
+ }
+
+ private static class CompilerImpl implements ICompiler {
+
+ private final CompilerFactoryImpl factory;
+
+ private final ILogicalPlan plan;
+
+ private final IOptimizationContext oc;
+
+ private final List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> logicalRewrites;
+
+ private final List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> physicalRewrites;
+
+ private final IAWriterFactory writerFactory;
+
+ private CompilerImpl(CompilerFactoryImpl factory, ILogicalPlan plan, IOptimizationContext oc,
+ List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> logicalRewrites,
+ List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> physicalRewrites,
+ IAWriterFactory writerFactory) {
+ this.factory = factory;
+ this.plan = plan;
+ this.oc = oc;
+ this.logicalRewrites = logicalRewrites;
+ this.physicalRewrites = physicalRewrites;
+ this.writerFactory = writerFactory;
+ }
+
+ @Override
+ public void optimize() throws AlgebricksException {
+ HeuristicOptimizer opt = new HeuristicOptimizer(plan, logicalRewrites, physicalRewrites, oc);
+ opt.optimize();
+ }
+
+ @Override
+ public JobSpecification createJob(Object appContext, IJobletEventListenerFactory jobEventListenerFactory)
+ throws AlgebricksException {
+ AlgebricksConfig.ALGEBRICKS_LOGGER.trace("Starting Job Generation.\n");
+ PlanCompiler pc = factory.createPlanCompiler(oc, appContext, writerFactory);
+ return pc.compilePlan(plan, jobEventListenerFactory);
+ }
+
+ @Override
+ public boolean skipJobCapacityAssignment() {
+ return oc.skipJobCapacityAssignment();
+ }
+ }
}
diff --git a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/ICompiler.java b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/ICompiler.java
index 7dac3eb..0f5798e 100644
--- a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/ICompiler.java
+++ b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/ICompiler.java
@@ -27,4 +27,6 @@
public JobSpecification createJob(Object appContext, IJobletEventListenerFactory jobEventListenerFactory)
throws AlgebricksException;
+
+ boolean skipJobCapacityAssignment();
}
diff --git a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/ICompilerFactory.java b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/ICompilerFactory.java
index 7c138ea..07a8034 100644
--- a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/ICompilerFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/ICompilerFactory.java
@@ -19,8 +19,12 @@
package org.apache.hyracks.algebricks.compiler.api;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+import org.apache.hyracks.algebricks.core.rewriter.base.IRuleSetKind;
public interface ICompilerFactory {
ICompiler createCompiler(ILogicalPlan plan, IMetadataProvider<?, ?> metadata, int varCounter);
+
+ ICompiler createCompiler(ILogicalPlan plan, IOptimizationContext newOptContext, IRuleSetKind ruleSetKind);
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/pom.xml b/hyracks-fullstack/algebricks/algebricks-core/pom.xml
index 9c81540..d1294e2 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/pom.xml
+++ b/hyracks-fullstack/algebricks/algebricks-core/pom.xml
@@ -25,7 +25,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>algebricks</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/Counter.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/Counter.java
index 7f2d3c8..3982171 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/Counter.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/Counter.java
@@ -32,10 +32,18 @@
return counter;
}
+ public int getAndInc() {
+ return counter++;
+ }
+
public void inc() {
++counter;
}
+ public int incAndGet() {
+ return ++counter;
+ }
+
public void set(int newStart) {
counter = newStart;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/ILogicalExpression.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/ILogicalExpression.java
index aac660a..1e4a388 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/ILogicalExpression.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/ILogicalExpression.java
@@ -41,32 +41,27 @@
// constraints (e.g., FDs, equivalences)
/**
- * @param fds
- * Output argument: functional dependencies that can be inferred
- * from this expression.
- * @param equivClasses
- * Output argument: Equivalence classes that can be inferred from
- * this expression.
+ * @param fds Output argument: functional dependencies that can be inferred
+ * from this expression.
+ * @param equivClasses Output argument: Equivalence classes that can be inferred from
+ * this expression.
*/
public void getConstraintsAndEquivClasses(Collection<FunctionalDependency> fds,
Map<LogicalVariable, EquivalenceClass> equivClasses);
/**
- * @param fds
- * Output argument: functional dependencies that can be inferred
- * from this expression.
- * @param outerVars
- * Input argument: variables coming from outer branch(es), e.g.,
- * the left branch of a left outer join.
+ * @param fds Output argument: functional dependencies that can be inferred
+ * from this expression.
+ * @param outerVars Input argument: variables coming from outer branch(es), e.g.,
+ * the left branch of a left outer join.
*/
public void getConstraintsForOuterJoin(Collection<FunctionalDependency> fds, Collection<LogicalVariable> outerVars);
/**
- * @param conjs
- * Output argument: a list of expression whose conjunction, in
- * any order, can replace the current expression.
+ * @param conjs Output argument: a list of expression whose conjunction, in
+ * any order, can replace the current expression.
* @return true if the expression can be broken in at least two conjuncts,
- * false otherwise.
+ * false otherwise.
*/
public boolean splitIntoConjuncts(List<Mutable<ILogicalExpression>> conjs);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/IOptimizationContext.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/IOptimizationContext.java
index 166ab9a..10b0451 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/IOptimizationContext.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/IOptimizationContext.java
@@ -33,6 +33,7 @@
import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
import org.apache.hyracks.algebricks.core.algebra.typing.ITypingContext;
import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.algebricks.core.rewriter.base.IOptimizationContextFactory;
import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
import org.apache.hyracks.api.exceptions.IWarningCollector;
@@ -93,4 +94,14 @@
public PlanStructureVerifier getPlanStructureVerifier();
public PlanStabilityVerifier getPlanStabilityVerifier();
+
+ void setCompilerFactory(Object factory);
+
+ Object getCompilerFactory();
+
+ IOptimizationContextFactory getOptimizationContextFactory();
+
+ default boolean skipJobCapacityAssignment() {
+ return false;
+ }
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/OperatorAnnotations.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/OperatorAnnotations.java
index cbe0882..3b44a97 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/OperatorAnnotations.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/OperatorAnnotations.java
@@ -29,4 +29,10 @@
// Integer
public static final String MAX_NUMBER_FRAMES = "MAX_NUMBER_FRAMES"; // -->
// Integer
+ public static final String OP_INPUT_CARDINALITY = "INPUT_CARDINALITY";
+ public static final String OP_OUTPUT_CARDINALITY = "OUTPUT_CARDINALITY";
+ public static final String OP_COST_TOTAL = "TOTAL_COST";
+ public static final String OP_COST_LOCAL = "OP_COST";
+ public static final String OP_LEFT_EXCHANGE_COST = "LEFT_EXCHANGE_COST";
+ public static final String OP_RIGHT_EXCHANGE_COST = "RIGHT_EXCHANGE_COST";
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/BroadcastExpressionAnnotation.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/BroadcastExpressionAnnotation.java
index 79b9e2c..f1b5093 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/BroadcastExpressionAnnotation.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/BroadcastExpressionAnnotation.java
@@ -38,13 +38,37 @@
}
}
- private final BroadcastSide side;
+ private String name;
+ private BroadcastSide side;
+
+ public BroadcastExpressionAnnotation(String name) {
+ this.name = validateName(name);
+ this.side = null;
+ }
public BroadcastExpressionAnnotation(BroadcastSide side) {
+ this.name = null;
this.side = Objects.requireNonNull(side);
}
+ public String getName() {
+ return name;
+ }
+
public BroadcastSide getBroadcastSide() {
return side;
}
+
+ public void setBroadcastSide(BroadcastSide side) {
+ this.name = null;
+ this.side = Objects.requireNonNull(side);
+ }
+
+ private String validateName(String name) {
+ String n = Objects.requireNonNull(name);
+ if (n.isBlank()) {
+ throw new IllegalArgumentException("BroadcastExpressionAnnotation:" + name + "cannot be blank");
+ }
+ return n;
+ }
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/ConstantExpression.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/ConstantExpression.java
index e2a68da..cecbaf7 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/ConstantExpression.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/ConstantExpression.java
@@ -16,6 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
+
package org.apache.hyracks.algebricks.core.algebra.expressions;
import java.util.Collection;
@@ -55,7 +56,7 @@
@Override
public String toString() {
- return "TRUE";
+ return "true";
}
@Override
@@ -92,7 +93,7 @@
@Override
public String toString() {
- return "FALSE";
+ return "false";
}
@Override
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/HashJoinExpressionAnnotation.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/HashJoinExpressionAnnotation.java
new file mode 100644
index 0000000..88d9dd5
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/HashJoinExpressionAnnotation.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.algebricks.core.algebra.expressions;
+
+import java.util.Objects;
+
+import org.apache.hyracks.algebricks.common.utils.Pair;
+
+public class HashJoinExpressionAnnotation implements IExpressionAnnotation {
+
+ public enum BuildOrProbe {
+ BUILD,
+ PROBE
+ }
+
+ public enum BuildSide {
+ LEFT,
+ RIGHT
+ }
+
+ private BuildOrProbe buildOrProbe;
+ private String name;
+ private BuildSide side;
+
+ public HashJoinExpressionAnnotation(Pair<BuildOrProbe, String> pair) {
+ this.buildOrProbe = Objects.requireNonNull(pair.getFirst());
+ this.name = validateName(pair.getSecond());
+ this.side = null;
+ }
+
+ public HashJoinExpressionAnnotation(BuildSide side) {
+ this.buildOrProbe = null;
+ this.name = null;
+ this.side = Objects.requireNonNull(side);
+ }
+
+ public BuildOrProbe getBuildOrProbe() {
+ return buildOrProbe;
+ }
+
+ public String getName() {
+ return name;
+ }
+
+ public BuildSide getBuildSide() {
+ return side;
+ }
+
+ public void setBuildSide(BuildSide side) {
+ this.buildOrProbe = null;
+ this.name = null;
+ this.side = Objects.requireNonNull(side);
+ }
+
+ private String validateName(String name) {
+ String n = Objects.requireNonNull(name);
+ if (n.isBlank()) {
+ throw new IllegalArgumentException("HashJoinExpressionAnnotation:" + name + "cannot be blank");
+ }
+ return n;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/JoinProductivityAnnotation.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/JoinProductivityAnnotation.java
new file mode 100644
index 0000000..0191b85
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/JoinProductivityAnnotation.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.algebricks.core.algebra.expressions;
+
+public class JoinProductivityAnnotation implements IExpressionAnnotation {
+
+ private final double productivity;
+ private final String leftSideDataSet;
+
+ public JoinProductivityAnnotation(double productivity, String leftSideDataSet) {
+ this.productivity = productivity;
+ this.leftSideDataSet = leftSideDataSet;
+ }
+
+ public double getJoinProductivity() {
+ return productivity;
+ }
+
+ public String getLeftSideDataSet() {
+ return leftSideDataSet;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/PredicateCardinalityAnnotation.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/PredicateCardinalityAnnotation.java
new file mode 100644
index 0000000..e172449
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/PredicateCardinalityAnnotation.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.algebricks.core.algebra.expressions;
+
+public class PredicateCardinalityAnnotation implements IExpressionAnnotation {
+
+ private final double selectivity;
+
+ public PredicateCardinalityAnnotation(double selectivity) {
+ this.selectivity = selectivity;
+ }
+
+ public double getSelectivity() {
+ return selectivity;
+ }
+}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IDataSource.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IDataSource.java
index e15b699..f7cd6da 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IDataSource.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IDataSource.java
@@ -18,7 +18,9 @@
*/
package org.apache.hyracks.algebricks.core.algebra.metadata;
+import java.io.Serializable;
import java.util.List;
+import java.util.Map;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
import org.apache.hyracks.algebricks.core.algebra.properties.FunctionalDependency;
@@ -34,7 +36,17 @@
public void computeFDs(List<LogicalVariable> scanVariables, List<FunctionalDependency> fdList);
// https://issues.apache.org/jira/browse/ASTERIXDB-1619
+
+ /**
+ * Return true if this data source is the start of the job pipeline making its scan op the start of the job pipeline
+ * instead of an ETS op, for example. This flag is used to disable the Hyracks op generation of the input
+ * operators to the data scan (i.e. ETS op that is an input to the data scan will not be generated).
+ */
public boolean isScanAccessPathALeaf();
public INodeDomain getDomain();
+
+ public Map<String, Serializable> getProperties();
+
+ boolean sameAs(IDataSource<?> other);
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
index 77fdd74..d350789 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
@@ -31,7 +31,9 @@
import org.apache.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import org.apache.hyracks.algebricks.data.IAWriterFactory;
import org.apache.hyracks.algebricks.data.IPrinterFactory;
+import org.apache.hyracks.algebricks.data.IResultSerializerFactoryProvider;
import org.apache.hyracks.algebricks.runtime.base.AlgebricksPipeline;
import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
@@ -56,11 +58,12 @@
IProjectionInfo<?> projectionInfo) throws AlgebricksException;
public Pair<IPushRuntimeFactory, AlgebricksPartitionConstraint> getWriteFileRuntime(IDataSink sink,
- int[] printColumns, IPrinterFactory[] printerFactories, RecordDescriptor inputDesc)
- throws AlgebricksException;
+ int[] printColumns, IPrinterFactory[] printerFactories, IAWriterFactory writerFactory,
+ RecordDescriptor inputDesc) throws AlgebricksException;
public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getResultHandleRuntime(IDataSink sink,
- int[] printColumns, IPrinterFactory[] printerFactories, RecordDescriptor inputDesc,
+ int[] printColumns, IPrinterFactory[] printerFactories, IAWriterFactory writerFactory,
+ IResultSerializerFactoryProvider resultSerializerFactoryProvider, RecordDescriptor inputDesc,
IResultMetadata metadata, JobSpecification spec) throws AlgebricksException;
public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getWriteResultRuntime(IDataSource<S> dataSource,
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractDataSourceOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractDataSourceOperator.java
index 7d262a8..12a0572 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractDataSourceOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractDataSourceOperator.java
@@ -35,4 +35,8 @@
public IDataSource<?> getDataSource() {
return dataSource;
}
+
+ public void setDataSource(IDataSource<?> datasource) {
+ this.dataSource = datasource;
+ }
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/BroadcastSideSwitchingVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/BroadcastSideSwitchingVisitor.java
index aa0f642..d8a8348 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/BroadcastSideSwitchingVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/BroadcastSideSwitchingVisitor.java
@@ -70,7 +70,7 @@
}
}
BroadcastExpressionAnnotation bcastAnn = expr.removeAnnotation(BroadcastExpressionAnnotation.class);
- if (bcastAnn != null) {
+ if (bcastAnn != null && bcastAnn.getBroadcastSide() != null) {
BroadcastExpressionAnnotation.BroadcastSide oppositeSide =
BroadcastExpressionAnnotation.BroadcastSide.getOppositeSide(bcastAnn.getBroadcastSide());
expr.putAnnotation(new BroadcastExpressionAnnotation(oppositeSide));
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
index e4df397..d81550c 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
@@ -34,6 +34,7 @@
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalPlan;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSource;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
@@ -249,8 +250,8 @@
return Boolean.FALSE;
}
OrderOperator orderOpArg = (OrderOperator) copyAndSubstituteVar(op, arg);
- boolean isomorphic = compareIOrderAndExpressions(op.getOrderExpressions(), orderOpArg.getOrderExpressions());
- return isomorphic;
+ return op.getTopK() == orderOpArg.getTopK()
+ && compareIOrderAndExpressions(op.getOrderExpressions(), orderOpArg.getOrderExpressions());
}
@Override
@@ -468,14 +469,21 @@
if (aop.getOperatorTag() != LogicalOperatorTag.DATASOURCESCAN) {
return Boolean.FALSE;
}
- DataSourceScanOperator argScan = (DataSourceScanOperator) arg;
- boolean isomorphic = op.getDataSource().getId().equals(argScan.getDataSource().getId())
- && op.getOutputLimit() == argScan.getOutputLimit()
- && Objects.equals(op.getProjectionInfo(), argScan.getProjectionInfo());
+ DataSourceScanOperator argScan = (DataSourceScanOperator) arg;
+ IDataSource<?> dataSource = op.getDataSource();
+ IDataSource<?> argDataSource = argScan.getDataSource();
+ boolean isomorphic = dataSource.sameAs(argDataSource);
if (!isomorphic) {
return Boolean.FALSE;
}
+
+ isomorphic = op.getOutputLimit() == argScan.getOutputLimit()
+ && Objects.equals(op.getProjectionInfo(), argScan.getProjectionInfo());
+ if (!isomorphic) {
+ return Boolean.FALSE;
+ }
+
DataSourceScanOperator scanOpArg = (DataSourceScanOperator) copyAndSubstituteVar(op, arg);
ILogicalExpression opCondition = op.getSelectCondition() != null ? op.getSelectCondition().getValue() : null;
ILogicalExpression argCondition =
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/VariableUtilities.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/VariableUtilities.java
index 69db58d..623e15a 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/VariableUtilities.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/VariableUtilities.java
@@ -33,6 +33,7 @@
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
import org.apache.hyracks.algebricks.core.algebra.typing.ITypingContext;
@@ -102,6 +103,17 @@
}
}
+ public static void getLiveVariablesInDescendantDataScans(ILogicalOperator op, Collection<LogicalVariable> vars)
+ throws AlgebricksException {
+ // DFS traversal
+ if (op.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
+ VariableUtilities.getLiveVariables(op, vars);
+ }
+ for (Mutable<ILogicalOperator> c : op.getInputs()) {
+ getLiveVariablesInDescendantDataScans(c.getValue(), vars);
+ }
+ }
+
public static void getProducedVariablesInDescendantsAndSelf(ILogicalOperator op, Collection<LogicalVariable> vars)
throws AlgebricksException {
// DFS traversal
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractHashJoinPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractHashJoinPOperator.java
index 38ccee5..c9f59c6 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractHashJoinPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractHashJoinPOperator.java
@@ -168,7 +168,9 @@
EquivalenceClass ecFst = eqmap.get(v2);
for (LogicalVariable vset1 : set1) {
if (vset1 == v2 || ecFst != null && eqmap.get(vset1) == ecFst) {
- covered.add(vset1);
+ if (!covered.add(vset1)) {
+ continue;
+ }
modifuppreq.add(r);
break;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/DistributeResultPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/DistributeResultPOperator.java
index 138cff8..1f7c16c 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/DistributeResultPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/DistributeResultPOperator.java
@@ -102,8 +102,9 @@
IPrinterFactory[] pf =
JobGenHelper.mkPrinterFactories(inputSchemas[0], context.getTypeEnvironment(op), context, columns);
- Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> runtimeAndConstraints = mp.getResultHandleRuntime(
- resultOp.getDataSink(), columns, pf, inputDesc, resultOp.getResultMetadata(), spec);
+ Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> runtimeAndConstraints =
+ mp.getResultHandleRuntime(resultOp.getDataSink(), columns, pf, context.getWriterFactory(),
+ context.getResultSerializerFactoryProvider(), inputDesc, resultOp.getResultMetadata(), spec);
IOperatorDescriptor opDesc = runtimeAndConstraints.first;
opDesc.setSourceLocation(resultOp.getSourceLocation());
builder.contributeHyracksOperator(resultOp, opDesc);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/NestedLoopJoinPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/NestedLoopJoinPOperator.java
index 6c5c8d0..cae5137 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/NestedLoopJoinPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/NestedLoopJoinPOperator.java
@@ -75,16 +75,18 @@
}
IPartitioningProperty pp;
-
AbstractLogicalOperator op = (AbstractLogicalOperator) iop;
+ // the partitioning property of the nested loop join is the same as the left branch.
+ // it cannot be the same as the right branch (BROADCAST) because the final joined data is not replicated at
+ // all partitions, and hence the final joined data is not BROADCAST.
if (op.getExecutionMode() == AbstractLogicalOperator.ExecutionMode.PARTITIONED) {
- AbstractLogicalOperator op2 = (AbstractLogicalOperator) op.getInputs().get(1).getValue();
- IPhysicalPropertiesVector pv1 = op2.getPhysicalOperator().getDeliveredProperties();
- if (pv1 == null) {
+ AbstractLogicalOperator leftOp = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
+ IPhysicalPropertiesVector leftOpProperties = leftOp.getPhysicalOperator().getDeliveredProperties();
+ if (leftOpProperties == null) {
pp = null;
} else {
- pp = pv1.getPartitioningProperty();
+ pp = leftOpProperties.getPartitioningProperty();
}
} else {
pp = IPartitioningProperty.UNPARTITIONED;
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SinkWritePOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SinkWritePOperator.java
index 3521a27..07c798f 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SinkWritePOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SinkWritePOperator.java
@@ -101,7 +101,7 @@
IMetadataProvider<?, ?> mp = context.getMetadataProvider();
Pair<IPushRuntimeFactory, AlgebricksPartitionConstraint> runtimeAndConstraints =
- mp.getWriteFileRuntime(write.getDataSink(), columns, pf, inputDesc);
+ mp.getWriteFileRuntime(write.getDataSink(), columns, pf, context.getWriterFactory(), inputDesc);
IPushRuntimeFactory runtime = runtimeAndConstraints.first;
runtime.setSourceLocation(write.getSourceLocation());
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/plan/ALogicalPlanImpl.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/plan/ALogicalPlanImpl.java
index b5b01a0..1bf2579 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/plan/ALogicalPlanImpl.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/plan/ALogicalPlanImpl.java
@@ -53,7 +53,7 @@
}
public static String prettyPrintPlan(ILogicalPlan plan) throws AlgebricksException {
- return PlanPrettyPrinter.createStringPlanPrettyPrinter().printPlan(plan).toString();
+ return PlanPrettyPrinter.createStringPlanPrettyPrinter().printPlan(plan, true).toString();
}
@Override
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/plan/PlanStabilityVerifier.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/plan/PlanStabilityVerifier.java
index 65aa64a..d6ffe9e 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/plan/PlanStabilityVerifier.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/plan/PlanStabilityVerifier.java
@@ -263,7 +263,7 @@
static String printOperator(ILogicalOperator op, IPlanPrettyPrinter printer) {
try {
- return printer.reset().printOperator((AbstractLogicalOperator) op, false).toString();
+ return printer.reset().printOperator((AbstractLogicalOperator) op, false, false).toString();
} catch (AlgebricksException e) {
// shouldn't happen
return op.toString();
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/AbstractLogicalOperatorPrettyPrintVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/AbstractLogicalOperatorPrettyPrintVisitor.java
index 2ddf257..36ee59f 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/AbstractLogicalOperatorPrettyPrintVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/AbstractLogicalOperatorPrettyPrintVisitor.java
@@ -18,11 +18,15 @@
*/
package org.apache.hyracks.algebricks.core.algebra.prettyprint;
+import java.util.Map;
+
import org.apache.commons.lang3.mutable.Mutable;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalPlan;
import org.apache.hyracks.algebricks.core.algebra.base.IPhysicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.base.OperatorAnnotations;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionVisitor;
@@ -30,6 +34,9 @@
public abstract class AbstractLogicalOperatorPrettyPrintVisitor<T> implements ILogicalOperatorVisitor<Void, T> {
+ protected static final String CARDINALITY = "cardinality";
+ protected static final String OP_COST_LOCAL = "op-cost";
+ protected static final String OP_COST_TOTAL = "total-cost";
protected final ILogicalExpressionVisitor<String, T> exprVisitor;
protected final AlgebricksStringBuilderWriter buffer;
@@ -49,6 +56,52 @@
buffer.getBuilder().setLength(0);
}
+ protected double getPlanCardinality(ILogicalOperator op) {
+ Double planCard = null;
+ if (op.getOperatorTag() == LogicalOperatorTag.DISTRIBUTE_RESULT) {
+ planCard = (Double) getAnnotationValue(op, OperatorAnnotations.OP_OUTPUT_CARDINALITY);
+ }
+ return (planCard != null) ? planCard : 0.0;
+ }
+
+ protected double getPlanCost(ILogicalOperator op) {
+ Double planCost = null;
+ if (op.getOperatorTag() == LogicalOperatorTag.DISTRIBUTE_RESULT) {
+ planCost = (Double) getAnnotationValue(op, OperatorAnnotations.OP_COST_TOTAL);
+ }
+ return (planCost != null) ? planCost : 0.0;
+ }
+
+ protected double getOpCardinality(ILogicalOperator op) {
+ Double opCard;
+
+ if (op.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
+ opCard = (Double) getAnnotationValue(op, OperatorAnnotations.OP_INPUT_CARDINALITY);
+ } else {
+ opCard = (Double) getAnnotationValue(op, OperatorAnnotations.OP_OUTPUT_CARDINALITY);
+ }
+
+ return (opCard != null) ? opCard : 0.0;
+ }
+
+ protected double getOpLocalCost(ILogicalOperator op) {
+ Double opLocalCost = (Double) getAnnotationValue(op, OperatorAnnotations.OP_COST_LOCAL);
+ return (opLocalCost != null) ? opLocalCost : 0.0;
+ }
+
+ protected double getOpTotalCost(ILogicalOperator op) {
+ Double opTotalCost = (Double) getAnnotationValue(op, OperatorAnnotations.OP_COST_TOTAL);
+ return (opTotalCost != null) ? opTotalCost : 0.0;
+ }
+
+ protected Object getAnnotationValue(ILogicalOperator op, String key) {
+ Map<String, Object> annotations = op.getAnnotations();
+ if (annotations != null && annotations.containsKey(key)) {
+ return annotations.get(key);
+ }
+ return null;
+ }
+
@Override
public String toString() {
return buffer.toString();
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/IPlanPrettyPrinter.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/IPlanPrettyPrinter.java
index faf4976..66d48d3 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/IPlanPrettyPrinter.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/IPlanPrettyPrinter.java
@@ -30,17 +30,18 @@
/** Prints the plan rooted at the operator argument. */
default IPlanPrettyPrinter printOperator(AbstractLogicalOperator operator) throws AlgebricksException {
- return printOperator(operator, true);
+ return printOperator(operator, true, false);
}
/** Prints given operator and optionally it's inputs */
- IPlanPrettyPrinter printOperator(AbstractLogicalOperator operator, boolean printInputs) throws AlgebricksException;
+ IPlanPrettyPrinter printOperator(AbstractLogicalOperator operator, boolean printInputs,
+ boolean printOptimizerEstimates) throws AlgebricksException;
/** Prints given expression */
IPlanPrettyPrinter printExpression(ILogicalExpression expression) throws AlgebricksException;
/** Prints the whole logical plan. */
- IPlanPrettyPrinter printPlan(ILogicalPlan plan) throws AlgebricksException;
+ IPlanPrettyPrinter printPlan(ILogicalPlan plan, boolean printOptimizerEstimates) throws AlgebricksException;
/** Resets the state of the pretty printer. */
IPlanPrettyPrinter reset() throws AlgebricksException;
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
index 64abaee..a60c969 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
@@ -90,27 +90,49 @@
}
@Override
- public final IPlanPrettyPrinter printPlan(ILogicalPlan plan) throws AlgebricksException {
- printPlanImpl(plan, 0);
+ public final IPlanPrettyPrinter printPlan(ILogicalPlan plan, boolean printOptimizerEstimates)
+ throws AlgebricksException {
+ printPlanImpl(plan, 0, printOptimizerEstimates);
return this;
}
@Override
- public final IPlanPrettyPrinter printOperator(AbstractLogicalOperator op, boolean printInputs)
- throws AlgebricksException {
- printOperatorImpl(op, 0, printInputs);
+ public final IPlanPrettyPrinter printOperator(AbstractLogicalOperator op, boolean printInputs,
+ boolean printOptimizerEstimates) throws AlgebricksException {
+ printOperatorImpl(op, 0, printInputs, printOptimizerEstimates);
return this;
}
- private void printPlanImpl(ILogicalPlan plan, int indent) throws AlgebricksException {
+ private void printPlanImpl(ILogicalPlan plan, int indent, boolean printOptimizerEstimates)
+ throws AlgebricksException {
for (Mutable<ILogicalOperator> root : plan.getRoots()) {
- printOperatorImpl((AbstractLogicalOperator) root.getValue(), indent, true);
+ printOperatorImpl((AbstractLogicalOperator) root.getValue(), indent, true, printOptimizerEstimates);
}
}
- private void printOperatorImpl(AbstractLogicalOperator op, int indent, boolean printInputs)
- throws AlgebricksException {
+ private void printOperatorImpl(AbstractLogicalOperator op, int indent, boolean printInputs,
+ boolean printOptimizerEstimates) throws AlgebricksException {
+ double opCard, opLocalCost, opTotalCost;
+
op.accept(this, indent);
+ if (printOptimizerEstimates) {
+ opCard = getOpCardinality(op);
+ opLocalCost = getOpLocalCost(op);
+ opTotalCost = getOpTotalCost(op);
+ buffer.append(" [");
+ buffer.append(CARDINALITY);
+ buffer.append(": ");
+ buffer.append(Double.toString(opCard));
+ buffer.append(", ");
+ buffer.append(OP_COST_LOCAL);
+ buffer.append(": ");
+ buffer.append(Double.toString(opLocalCost));
+ buffer.append(", ");
+ buffer.append(OP_COST_TOTAL);
+ buffer.append(": ");
+ buffer.append(Double.toString(opTotalCost));
+ buffer.append("]");
+ }
IPhysicalOperator pOp = op.getPhysicalOperator();
if (pOp != null) {
@@ -123,7 +145,8 @@
if (printInputs) {
for (Mutable<ILogicalOperator> i : op.getInputs()) {
- printOperatorImpl((AbstractLogicalOperator) i.getValue(), indent + INIT_INDENT, printInputs);
+ printOperatorImpl((AbstractLogicalOperator) i.getValue(), indent + INIT_INDENT, printInputs,
+ printOptimizerEstimates);
}
}
}
@@ -608,7 +631,7 @@
} else {
addIndent(indent).append(" {\n");
}
- printPlanImpl(p, indent + SUBPLAN_INDENT);
+ printPlanImpl(p, indent + SUBPLAN_INDENT, true);
addIndent(indent).append(" }");
}
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java
index 2728e46..464d15e 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java
@@ -34,8 +34,10 @@
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalPlan;
import org.apache.hyracks.algebricks.core.algebra.base.IPhysicalOperator;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
import org.apache.hyracks.algebricks.core.algebra.metadata.IProjectionInfo;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractUnnestMapOperator;
@@ -96,7 +98,7 @@
private static final String EXPRESSION_FIELD = "expression";
private static final String CONDITION_FIELD = "condition";
private static final String MISSING_VALUE_FIELD = "missing-value";
-
+ private static final String OPTIMIZER_ESTIMATES = "optimizer-estimates";
private final Map<AbstractLogicalOperator, String> operatorIdentity = new HashMap<>();
private final IdCounter idCounter = new IdCounter();
private final JsonGenerator jsonGenerator;
@@ -138,8 +140,7 @@
stringPrefix = stringPrefix.isEmpty() ? val.toString() : stringPrefix + "." + val.toString();
}
if (!operatorIdentity.containsKey(op)) {
- String opId = stringPrefix.isEmpty() ? "" + Integer.toString(++id)
- : stringPrefix + "." + Integer.toString(++id);
+ String opId = stringPrefix.isEmpty() ? "" + (++id) : stringPrefix + "." + (++id);
operatorIdentity.put(op, opId);
}
return operatorIdentity.get(op);
@@ -155,28 +156,30 @@
}
@Override
- public final IPlanPrettyPrinter printPlan(ILogicalPlan plan) throws AlgebricksException {
- printPlanImpl(plan);
+ public final IPlanPrettyPrinter printPlan(ILogicalPlan plan, boolean printOptimizerEstimates)
+ throws AlgebricksException {
+ printPlanImpl(plan, printOptimizerEstimates);
flushContentToWriter();
return this;
}
@Override
- public final IPlanPrettyPrinter printOperator(AbstractLogicalOperator op, boolean printInputs)
- throws AlgebricksException {
- printOperatorImpl(op, printInputs);
+
+ public final IPlanPrettyPrinter printOperator(AbstractLogicalOperator op, boolean printInputs,
+ boolean printOptimizerEstimates) throws AlgebricksException {
+ printOperatorImpl(op, printInputs, printOptimizerEstimates);
flushContentToWriter();
return this;
}
- private void printPlanImpl(ILogicalPlan plan) throws AlgebricksException {
+ private void printPlanImpl(ILogicalPlan plan, boolean printOptimizerEstimates) throws AlgebricksException {
try {
boolean writeArrayOfRoots = plan.getRoots().size() > 1;
if (writeArrayOfRoots) {
jsonGenerator.writeStartArray();
}
for (Mutable<ILogicalOperator> root : plan.getRoots()) {
- printOperatorImpl((AbstractLogicalOperator) root.getValue(), true);
+ printOperatorImpl((AbstractLogicalOperator) root.getValue(), true, printOptimizerEstimates);
}
if (writeArrayOfRoots) {
jsonGenerator.writeEndArray();
@@ -186,7 +189,8 @@
}
}
- private void printOperatorImpl(AbstractLogicalOperator op, boolean printInputs) throws AlgebricksException {
+ private void printOperatorImpl(AbstractLogicalOperator op, boolean printInputs, boolean printOptimizerEstimates)
+ throws AlgebricksException {
try {
jsonGenerator.writeStartObject();
op.accept(this, null);
@@ -196,12 +200,12 @@
jsonGenerator.writeStringField("physical-operator", pOp.toString(false));
}
jsonGenerator.writeStringField("execution-mode", op.getExecutionMode().toString());
- if (printInputs && !op.getInputs().isEmpty()) {
- jsonGenerator.writeArrayFieldStart("inputs");
- for (Mutable<ILogicalOperator> k : op.getInputs()) {
- printOperatorImpl((AbstractLogicalOperator) k.getValue(), printInputs);
- }
- jsonGenerator.writeEndArray();
+
+ generateCardCostFields(op, printOptimizerEstimates);
+
+ List<Mutable<ILogicalOperator>> inputs = op.getInputs();
+ if (printInputs && !inputs.isEmpty()) {
+ printInputs(op, inputs, printOptimizerEstimates);
}
jsonGenerator.writeEndObject();
} catch (IOException e) {
@@ -209,6 +213,41 @@
}
}
+ private void printInputs(AbstractLogicalOperator op, List<Mutable<ILogicalOperator>> inputs,
+ boolean printOptimizerEstimates) throws IOException, AlgebricksException {
+ jsonGenerator.writeArrayFieldStart("inputs");
+ if (printInputsInReverse(op)) {
+ for (int i = inputs.size() - 1; i >= 0; i--) {
+ Mutable<ILogicalOperator> inOp = inputs.get(i);
+ printOperatorImpl((AbstractLogicalOperator) inOp.getValue(), true, printOptimizerEstimates);
+ }
+ } else {
+ for (Mutable<ILogicalOperator> inOp : inputs) {
+ printOperatorImpl((AbstractLogicalOperator) inOp.getValue(), true, printOptimizerEstimates);
+ }
+ }
+ jsonGenerator.writeEndArray();
+ }
+
+ private void generateCardCostFields(AbstractLogicalOperator op, boolean printOptimizerEstimates)
+ throws AlgebricksException {
+ double opCard, opLocalCost, opTotalCost;
+ if (printOptimizerEstimates) {
+ opCard = getOpCardinality(op);
+ opLocalCost = getOpLocalCost(op);
+ opTotalCost = getOpTotalCost(op);
+ try {
+ jsonGenerator.writeObjectFieldStart(OPTIMIZER_ESTIMATES);
+ jsonGenerator.writeNumberField(CARDINALITY, opCard);
+ jsonGenerator.writeNumberField(OP_COST_LOCAL, opLocalCost);
+ jsonGenerator.writeNumberField(OP_COST_TOTAL, opTotalCost);
+ jsonGenerator.writeEndObject();
+ } catch (IOException e) {
+ throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
+ }
+ }
+ }
+
@Override
public IPlanPrettyPrinter printExpression(ILogicalExpression expression) throws AlgebricksException {
try {
@@ -292,6 +331,7 @@
try {
jsonGenerator.writeStringField(OPERATOR_FIELD, "join");
writeStringFieldExpression(CONDITION_FIELD, op.getCondition(), indent);
+ writeBuildSide(op);
return null;
} catch (IOException e) {
throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
@@ -306,6 +346,7 @@
if (op.getMissingValue().isNull()) {
writeNullField(MISSING_VALUE_FIELD);
}
+ writeBuildSide(op);
return null;
} catch (IOException e) {
throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
@@ -767,7 +808,7 @@
jsonGenerator.writeArrayFieldStart("subplan");
List<ILogicalPlan> nestedPlans = op.getNestedPlans();
for (int i = 0, size = nestedPlans.size(); i < size; i++) {
- printPlanImpl(nestedPlans.get(i));
+ printPlanImpl(nestedPlans.get(i), false);
}
jsonGenerator.writeEndArray();
idCounter.previousPrefix();
@@ -857,6 +898,22 @@
}
}
+ private void writeBuildSide(AbstractBinaryJoinOperator op) throws IOException {
+ if (isHashJoin(op)) {
+ jsonGenerator.writeNumberField("build-side", 0);
+ }
+ }
+
+ private static boolean printInputsInReverse(AbstractLogicalOperator op) {
+ return isHashJoin(op);
+ }
+
+ private static boolean isHashJoin(AbstractLogicalOperator op) {
+ IPhysicalOperator pOp = op.getPhysicalOperator();
+ return pOp != null && (pOp.getOperatorTag() == PhysicalOperatorTag.IN_MEMORY_HASH_JOIN
+ || pOp.getOperatorTag() == PhysicalOperatorTag.HYBRID_HASH_JOIN);
+ }
+
private String getIndexOpString(Kind opKind) {
switch (opKind) {
case DELETE:
@@ -884,13 +941,17 @@
/////////////// string fields ///////////////
- /** Writes "fieldName": "expr" */
+ /**
+ * Writes "fieldName": "expr"
+ */
private void writeStringFieldExpression(String fieldName, Mutable<ILogicalExpression> expressionRef, Void indent)
throws AlgebricksException, IOException {
writeStringFieldExpression(fieldName, expressionRef.getValue(), indent);
}
- /** Writes "fieldName": "expr" */
+ /**
+ * Writes "fieldName": "expr"
+ */
private void writeStringFieldExpression(String fieldName, ILogicalExpression expression, Void indent)
throws AlgebricksException, IOException {
jsonGenerator.writeStringField(fieldName, expression.accept(exprVisitor, indent));
@@ -898,7 +959,9 @@
/////////////// array fields ///////////////
- /** Writes "fieldName": [ "var1", "var2", ... ] */
+ /**
+ * Writes "fieldName": [ "var1", "var2", ... ]
+ */
private void writeArrayFieldOfVariables(String fieldName, List<LogicalVariable> variables) throws IOException {
jsonGenerator.writeArrayFieldStart(fieldName);
for (int i = 0, size = variables.size(); i < size; i++) {
@@ -907,7 +970,9 @@
jsonGenerator.writeEndArray();
}
- /** Writes "fieldName": [ ["var1", "var2", ...], ["var1", "var2", ...] ] */
+ /**
+ * Writes "fieldName": [ ["var1", "var2", ...], ["var1", "var2", ...] ]
+ */
private void writeArrayFieldOfNestedVariablesList(String fieldName, List<List<LogicalVariable>> nestedVarList)
throws IOException {
jsonGenerator.writeArrayFieldStart(fieldName);
@@ -920,7 +985,9 @@
jsonGenerator.writeEndArray();
}
- /** Writes "fieldName" : [ "expr" ] */
+ /**
+ * Writes "fieldName" : [ "expr" ]
+ */
private void writeArrayFieldOfExpression(String fieldName, Mutable<ILogicalExpression> expr, Void indent)
throws IOException, AlgebricksException {
jsonGenerator.writeArrayFieldStart(fieldName);
@@ -928,7 +995,9 @@
jsonGenerator.writeEndArray();
}
- /** Writes "fieldName" : [ "expr1", "expr2", ...] */
+ /**
+ * Writes "fieldName" : [ "expr1", "expr2", ...]
+ */
private void writeArrayFieldOfExpressions(String fieldName, List<Mutable<ILogicalExpression>> exprs, Void indent)
throws IOException, AlgebricksException {
jsonGenerator.writeArrayFieldStart(fieldName);
@@ -938,7 +1007,9 @@
jsonGenerator.writeEndArray();
}
- /** Writes "fieldName" : [ { "variable": "var1", "expression": "expr1" }, ... ] */
+ /**
+ * Writes "fieldName" : [ { "variable": "var1", "expression": "expr1" }, ... ]
+ */
private void writeArrayFieldOfVariableExpressionPairs(String fieldName,
List<Pair<LogicalVariable, Mutable<ILogicalExpression>>> varExprPairs, Void indent)
throws AlgebricksException, IOException {
@@ -954,7 +1025,9 @@
jsonGenerator.writeEndArray();
}
- /** Writes "fieldName" : [ { "order": "", "expression": "" }, ... ] */
+ /**
+ * Writes "fieldName" : [ { "order": "", "expression": "" }, ... ]
+ */
private void writeArrayFieldOfOrderExprList(String fieldName,
List<Pair<OrderOperator.IOrder, Mutable<ILogicalExpression>>> orderList, Void indent)
throws AlgebricksException, IOException {
@@ -970,7 +1043,9 @@
/////////////// object fields ///////////////
- /** Writes "fieldName" : { "expressions": [ "expr1", "expr2", ...] } */
+ /**
+ * Writes "fieldName" : { "expressions": [ "expr1", "expr2", ...] }
+ */
private void writeObjectFieldWithExpressions(String fieldName, List<Mutable<ILogicalExpression>> exprs, Void indent)
throws IOException, AlgebricksException {
jsonGenerator.writeObjectFieldStart(fieldName);
@@ -980,7 +1055,9 @@
/////////////// other fields ///////////////
- /** Writes "fieldName": null */
+ /**
+ * Writes "fieldName": null
+ */
private void writeNullField(String fieldName) throws IOException {
jsonGenerator.writeNullField(fieldName);
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/IPartitioningRequirementsCoordinator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/IPartitioningRequirementsCoordinator.java
index d515fcf..a49c4b3 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/IPartitioningRequirementsCoordinator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/IPartitioningRequirementsCoordinator.java
@@ -29,6 +29,7 @@
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.api.exceptions.ErrorCode;
/**
* Implements constraints in between requirements for the children of the same
@@ -70,20 +71,29 @@
for (LogicalVariable v : set1) {
EquivalenceClass ecFirst = eqmap.get(v);
for (LogicalVariable r : uppreq.getColumnSet()) {
- EquivalenceClass ec = eqmap.get(r);
- if (ecFirst == ec) {
- covered.add(v);
- modifuppreq.add(r);
- break;
+ if (!modifuppreq.contains(r)) {
+ EquivalenceClass ec = eqmap.get(r);
+ if (ecFirst == ec) {
+ covered.add(v);
+ modifuppreq.add(r);
+ break;
+ }
}
}
}
if (!covered.equals(set1)) {
- throw new AlgebricksException("Could not modify " + rqdpp
- + " to agree with partitioning property " + firstDeliveredPartitioning
- + " delivered by previous input operator.");
+ throw new AlgebricksException(ErrorCode.ILLEGAL_STATE,
+ "Could not modify " + rqdpp + " to agree with partitioning property "
+ + firstDeliveredPartitioning
+ + " delivered by previous input operator.");
}
+
+ if (modifuppreq.size() != set1.size()) {
+ throw new AlgebricksException(ErrorCode.ILLEGAL_STATE,
+ "The number of variables are not equal in both partitioning sides");
+ }
+
UnorderedPartitionedProperty upp2 =
new UnorderedPartitionedProperty(modifuppreq, rqdpp.getNodeDomain());
return new Pair<Boolean, IPartitioningProperty>(false, upp2);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/UnorderedPartitionedProperty.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/UnorderedPartitionedProperty.java
index fa8650c..3d52d01 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/UnorderedPartitionedProperty.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/UnorderedPartitionedProperty.java
@@ -72,11 +72,13 @@
@Override
public IPartitioningProperty substituteColumnVars(Map<LogicalVariable, LogicalVariable> varMap) {
boolean applied = false;
- Set<LogicalVariable> newColumnSet = new ListSet<>(columnSet);
- for (Map.Entry<LogicalVariable, LogicalVariable> me : varMap.entrySet()) {
- if (newColumnSet.remove(me.getKey())) {
- newColumnSet.add(me.getValue());
+ Set<LogicalVariable> newColumnSet = new ListSet<>();
+ for (LogicalVariable variable : columnSet) {
+ if (varMap.containsKey(variable)) {
+ newColumnSet.add(varMap.get(variable));
applied = true;
+ } else {
+ newColumnSet.add(variable);
}
}
return applied ? new UnorderedPartitionedProperty(newColumnSet, domain) : this;
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorManipulationUtil.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorManipulationUtil.java
index 26be09c..f8fb6c2 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorManipulationUtil.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorManipulationUtil.java
@@ -37,8 +37,11 @@
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalPlan;
import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.base.OperatorAnnotations;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
@@ -178,17 +181,24 @@
}
public static void substituteVarRec(AbstractLogicalOperator op, LogicalVariable v1, LogicalVariable v2,
- boolean goThroughNts, ITypingContext ctx) throws AlgebricksException {
+ boolean goThroughNts, ITypingContext ctx, Set<ILogicalOperator> visited) throws AlgebricksException {
VariableUtilities.substituteVariables(op, v1, v2, goThroughNts, ctx);
for (Mutable<ILogicalOperator> opRef2 : op.getInputs()) {
- substituteVarRec((AbstractLogicalOperator) opRef2.getValue(), v1, v2, goThroughNts, ctx);
+ if (visited.contains(opRef2.getValue())) {
+ continue;
+ }
+ visited.add(opRef2.getValue());
+ substituteVarRec((AbstractLogicalOperator) opRef2.getValue(), v1, v2, goThroughNts, ctx, visited);
}
if (op.getOperatorTag() == LogicalOperatorTag.NESTEDTUPLESOURCE && goThroughNts) {
NestedTupleSourceOperator nts = (NestedTupleSourceOperator) op;
if (nts.getDataSourceReference() != null) {
AbstractLogicalOperator op2 =
(AbstractLogicalOperator) nts.getDataSourceReference().getValue().getInputs().get(0).getValue();
- substituteVarRec(op2, v1, v2, goThroughNts, ctx);
+ if (!visited.contains(op2)) {
+ visited.add(op2);
+ substituteVarRec(op2, v1, v2, goThroughNts, ctx, visited);
+ }
}
}
if (op.hasNestedPlans()) {
@@ -196,7 +206,11 @@
for (ILogicalPlan p : aonp.getNestedPlans()) {
for (Mutable<ILogicalOperator> ref : p.getRoots()) {
AbstractLogicalOperator aop = (AbstractLogicalOperator) ref.getValue();
- substituteVarRec(aop, v1, v2, goThroughNts, ctx);
+ if (visited.contains(aop)) {
+ continue;
+ }
+ visited.add(aop);
+ substituteVarRec(aop, v1, v2, goThroughNts, ctx, visited);
}
}
}
@@ -305,10 +319,8 @@
/**
* Compute type environment of a newly generated operator {@code op} and its input.
*
- * @param op,
- * the logical operator.
- * @param context,the
- * optimization context.
+ * @param op, the logical operator.
+ * @param context,the optimization context.
* @throws AlgebricksException
*/
public static void computeTypeEnvironmentBottomUp(ILogicalOperator op, ITypingContext context)
@@ -330,10 +342,8 @@
/**
* Computes the type environment for a logical query plan.
*
- * @param plan,
- * the logical plan to consider.
- * @param context
- * the typing context.
+ * @param plan, the logical plan to consider.
+ * @param context the typing context.
* @throws AlgebricksException
*/
public static void computeTypeEnvironment(ILogicalPlan plan, ITypingContext context) throws AlgebricksException {
@@ -362,6 +372,15 @@
return false;
}
+ public static boolean ancestorOfOperatorsExcludeCurrent(ILogicalOperator op, Set<LogicalOperatorTag> tags) {
+ for (Mutable<ILogicalOperator> children : op.getInputs()) {
+ if (ancestorOfOperators(children.getValue(), tags)) {
+ return true;
+ }
+ }
+ return false;
+ }
+
/**
* Returns all descendants of an operator that are leaf operators
*
@@ -402,8 +421,8 @@
/**
* Find an item a given list
*
- * @param list list to search in
- * @param predicate predicate to test
+ * @param list list to search in
+ * @param predicate predicate to test
* @param predicateParam parameter to pass to the predicate
* @return item position in the given list or {@code -1} if not found
*/
@@ -445,9 +464,10 @@
/**
* Finds a variable assigned to a given expression and returns a new {@link VariableReferenceExpression}
* referring to this variable.
- * @param assignVarList list of variables
+ *
+ * @param assignVarList list of variables
* @param assignExprList list of expressions assigned to those variables
- * @param searchExpr expression to search for
+ * @param searchExpr expression to search for
* @return said value, {@code null} if a variable is not found
*/
public static VariableReferenceExpression findAssignedVariable(List<LogicalVariable> assignVarList,
@@ -509,4 +529,39 @@
outVarRefList.add(new MutableObject<>(varRef));
}
}
+
+ public static void replaceVarWithExpr(AbstractFunctionCallExpression inExpr, LogicalVariable var,
+ ILogicalExpression replacementExpr) {
+ for (Mutable<ILogicalExpression> arg : inExpr.getArguments()) {
+ if (arg.getValue().getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+ replaceVarWithExpr((AbstractFunctionCallExpression) (arg.getValue()), var, replacementExpr);
+ } else if (arg.getValue().getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+ LogicalVariable v = ((VariableReferenceExpression) arg.getValue()).getVariableReference();
+ if (v.equals(var)) {
+ arg.setValue(replacementExpr.cloneExpression());
+ }
+ }
+ }
+ }
+
+ public static void copyCardCostAnnotations(ILogicalOperator sourceOp, ILogicalOperator destOp) {
+ for (Map.Entry<String, Object> anno : sourceOp.getAnnotations().entrySet()) {
+ Object annotationVal = anno.getValue();
+ if (annotationVal != null) {
+ String annotation = anno.getKey();
+ switch (annotation) {
+ case OperatorAnnotations.OP_COST_LOCAL:
+ case OperatorAnnotations.OP_COST_TOTAL:
+ case OperatorAnnotations.OP_INPUT_CARDINALITY:
+ case OperatorAnnotations.OP_OUTPUT_CARDINALITY:
+ case OperatorAnnotations.OP_LEFT_EXCHANGE_COST:
+ case OperatorAnnotations.OP_RIGHT_EXCHANGE_COST:
+ destOp.getAnnotations().put(annotation, annotationVal);
+ break;
+ default:
+ break;
+ }
+ }
+ }
+ }
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java
index 86c6066..e53322c 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java
@@ -35,6 +35,13 @@
public static final boolean SUBPLAN_NESTEDPUSHDOWN_DEFAULT = true;
public static final boolean MIN_MEMORY_ALLOCATION_DEFAULT = true;
public static final boolean ARRAY_INDEX_DEFAULT = true;
+ public static final boolean CBO_DEFAULT = true;
+ public static final boolean CBO_TEST_DEFAULT = false;
+ public static final boolean FORCE_JOIN_ORDER_DEFAULT = false;
+ public static final String QUERY_PLAN_SHAPE_ZIGZAG = "zigzag";
+ public static final String QUERY_PLAN_SHAPE_LEFTDEEP = "leftdeep";
+ public static final String QUERY_PLAN_SHAPE_RIGHTDEEP = "rightdeep";
+ public static final String QUERY_PLAN_SHAPE_DEFAULT = QUERY_PLAN_SHAPE_ZIGZAG;
public static final int EXTERNAL_SCAN_BUFFER_SIZE =
StorageUtil.getIntSizeInBytes(8, StorageUtil.StorageUnit.KILOBYTE);
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobGenContext.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobGenContext.java
index 7c7d5a8..471380c 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobGenContext.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobGenContext.java
@@ -35,6 +35,7 @@
import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
import org.apache.hyracks.algebricks.core.algebra.typing.ITypingContext;
import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
+import org.apache.hyracks.algebricks.data.IAWriterFactory;
import org.apache.hyracks.algebricks.data.IBinaryBooleanInspectorFactory;
import org.apache.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
import org.apache.hyracks.algebricks.data.IBinaryHashFunctionFactoryProvider;
@@ -42,6 +43,7 @@
import org.apache.hyracks.algebricks.data.IBinaryIntegerInspectorFactory;
import org.apache.hyracks.algebricks.data.INormalizedKeyComputerFactoryProvider;
import org.apache.hyracks.algebricks.data.IPrinterFactoryProvider;
+import org.apache.hyracks.algebricks.data.IResultSerializerFactoryProvider;
import org.apache.hyracks.algebricks.data.ISerializerDeserializerProvider;
import org.apache.hyracks.algebricks.data.ITypeTraitProvider;
import org.apache.hyracks.algebricks.data.IUnnestingPositionWriterFactory;
@@ -57,6 +59,8 @@
private final IBinaryHashFunctionFamilyProvider hashFunctionFamilyProvider;
private final IBinaryComparatorFactoryProvider comparatorFactoryProvider;
private final IPrinterFactoryProvider printerFactoryProvider;
+ private final IAWriterFactory writerFactory;
+ private final IResultSerializerFactoryProvider resultSerializerFactoryProvider;
private final ITypeTraitProvider typeTraitProvider;
private final IMetadataProvider<?, ?> metadataProvider;
private final IMissingWriterFactory missingWriterFactory;
@@ -86,6 +90,7 @@
IBinaryComparatorFactoryProvider comparatorFactoryProvider, ITypeTraitProvider typeTraitProvider,
IBinaryBooleanInspectorFactory booleanInspectorFactory,
IBinaryIntegerInspectorFactory integerInspectorFactory, IPrinterFactoryProvider printerFactoryProvider,
+ IAWriterFactory writerFactory, IResultSerializerFactoryProvider resultSerializerFactoryProvider,
IMissingWriterFactory missingWriterFactory, IMissingWriterFactory nullWriterFactory,
IUnnestingPositionWriterFactory unnestingPositionWriterFactory,
INormalizedKeyComputerFactoryProvider normalizedKeyComputerFactoryProvider,
@@ -106,6 +111,8 @@
this.booleanInspectorFactory = booleanInspectorFactory;
this.integerInspectorFactory = integerInspectorFactory;
this.printerFactoryProvider = printerFactoryProvider;
+ this.writerFactory = writerFactory;
+ this.resultSerializerFactoryProvider = resultSerializerFactoryProvider;
this.clusterLocations = clusterLocations;
this.normalizedKeyComputerFactoryProvider = normalizedKeyComputerFactoryProvider;
this.missingWriterFactory = missingWriterFactory;
@@ -172,6 +179,14 @@
return printerFactoryProvider;
}
+ public IAWriterFactory getWriterFactory() {
+ return writerFactory;
+ }
+
+ public IResultSerializerFactoryProvider getResultSerializerFactoryProvider() {
+ return resultSerializerFactoryProvider;
+ }
+
public IPredicateEvaluatorFactoryProvider getPredicateEvaluatorFactoryProvider() {
return predEvaluatorFactoryProvider;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/AlgebricksOptimizationContext.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/AlgebricksOptimizationContext.java
index d1fd247..bee842c 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/AlgebricksOptimizationContext.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/AlgebricksOptimizationContext.java
@@ -46,45 +46,29 @@
import org.apache.hyracks.algebricks.core.algebra.properties.ILogicalPropertiesVector;
import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.NoOpWarningCollector;
/**
* The Algebricks default implementation for IOptimizationContext.
*/
-@SuppressWarnings({ "unchecked", "rawtypes" })
+@SuppressWarnings({ "rawtypes" })
public class AlgebricksOptimizationContext implements IOptimizationContext {
- private int varCounter;
+ private final IOptimizationContextFactory optContextFactory;
private final IExpressionEvalSizeComputer expressionEvalSizeComputer;
private final IMergeAggregationExpressionFactory mergeAggregationExpressionFactory;
private final PhysicalOptimizationConfig physicalOptimizationConfig;
- private final IVariableEvalSizeEnvironment varEvalSizeEnv = new IVariableEvalSizeEnvironment() {
- Map<LogicalVariable, Integer> varSizeMap = new HashMap<>();
-
- @Override
- public void setVariableEvalSize(LogicalVariable var, int size) {
- varSizeMap.put(var, size);
- }
-
- @Override
- public int getVariableEvalSize(LogicalVariable var) {
- return varSizeMap.get(var);
- }
- };
-
- private Map<ILogicalOperator, IVariableTypeEnvironment> typeEnvMap = new HashMap<>();
-
- private Map<ILogicalOperator, HashSet<ILogicalOperator>> alreadyCompared = new HashMap<>();
- private Map<IAlgebraicRewriteRule, HashSet<ILogicalOperator>> dontApply = new HashMap<>();
- private Map<LogicalVariable, FunctionalDependency> varToPrimaryKey = new HashMap<>();
-
- private IMetadataProvider metadataProvider;
- private HashSet<LogicalVariable> notToBeInlinedVars = new HashSet<>();
-
+ private final VariableEvalSizeEnvironmentImpl varEvalSizeEnv = new VariableEvalSizeEnvironmentImpl();
+ private final Map<ILogicalOperator, IVariableTypeEnvironment> typeEnvMap = new HashMap<>();
+ private final Map<ILogicalOperator, HashSet<ILogicalOperator>> alreadyCompared = new HashMap<>();
+ private final Map<IAlgebraicRewriteRule, HashSet<ILogicalOperator>> dontApply = new HashMap<>();
+ private final Map<LogicalVariable, FunctionalDependency> varToPrimaryKey = new HashMap<>();
+ private final HashSet<LogicalVariable> notToBeInlinedVars = new HashSet<>();
protected final Map<ILogicalOperator, List<FunctionalDependency>> fdGlobalMap = new HashMap<>();
protected final Map<ILogicalOperator, Map<LogicalVariable, EquivalenceClass>> eqClassGlobalMap = new HashMap<>();
-
protected final Map<ILogicalOperator, ILogicalPropertiesVector> logicalProps = new HashMap<>();
+
private final IExpressionTypeComputer expressionTypeComputer;
private final IMissableTypeComputer nullableTypeComputer;
private final INodeDomain defaultNodeDomain;
@@ -94,12 +78,18 @@
private final PlanStructureVerifier planStructureVerifier;
private final PlanStabilityVerifier planStabilityVerifier;
- public AlgebricksOptimizationContext(int varCounter, IExpressionEvalSizeComputer expressionEvalSizeComputer,
+ private int varCounter;
+ private IMetadataProvider metadataProvider;
+ private Object compilerFactory;
+
+ public AlgebricksOptimizationContext(IOptimizationContextFactory optContextFactory, int varCounter,
+ IExpressionEvalSizeComputer expressionEvalSizeComputer,
IMergeAggregationExpressionFactory mergeAggregationExpressionFactory,
IExpressionTypeComputer expressionTypeComputer, IMissableTypeComputer nullableTypeComputer,
IConflictingTypeResolver conflictingTypeResovler, PhysicalOptimizationConfig physicalOptimizationConfig,
AlgebricksPartitionConstraint clusterLocations, IPlanPrettyPrinter prettyPrinter,
IWarningCollector warningCollector) {
+ this.optContextFactory = optContextFactory;
this.varCounter = varCounter;
this.expressionEvalSizeComputer = expressionEvalSizeComputer;
this.mergeAggregationExpressionFactory = mergeAggregationExpressionFactory;
@@ -115,6 +105,35 @@
this.planStabilityVerifier = isSanityCheckEnabled ? new PlanStabilityVerifier(prettyPrinter) : null;
}
+ public AlgebricksOptimizationContext(AlgebricksOptimizationContext from) {
+ optContextFactory = from.optContextFactory;
+ varCounter = from.varCounter;
+ expressionEvalSizeComputer = from.expressionEvalSizeComputer;
+ mergeAggregationExpressionFactory = from.mergeAggregationExpressionFactory;
+ expressionTypeComputer = from.expressionTypeComputer;
+ nullableTypeComputer = from.nullableTypeComputer;
+ physicalOptimizationConfig = from.physicalOptimizationConfig;
+ defaultNodeDomain = from.defaultNodeDomain;
+ prettyPrinter = from.prettyPrinter;
+ conflictingTypeResovler = from.conflictingTypeResovler;
+ warningCollector = NoOpWarningCollector.INSTANCE;
+ boolean isSanityCheckEnabled = physicalOptimizationConfig.isSanityCheckEnabled();
+ planStructureVerifier = isSanityCheckEnabled ? new PlanStructureVerifier(from.prettyPrinter, this) : null;
+ planStabilityVerifier = isSanityCheckEnabled ? new PlanStabilityVerifier(from.prettyPrinter) : null;
+ metadataProvider = from.metadataProvider;
+ compilerFactory = from.compilerFactory;
+
+ varEvalSizeEnv.varSizeMap.putAll(from.varEvalSizeEnv.varSizeMap);
+ typeEnvMap.putAll(from.typeEnvMap);
+ alreadyCompared.putAll(from.alreadyCompared);
+ dontApply.putAll(from.dontApply);
+ varToPrimaryKey.putAll(from.varToPrimaryKey);
+ notToBeInlinedVars.addAll(from.notToBeInlinedVars);
+ fdGlobalMap.putAll(from.fdGlobalMap);
+ eqClassGlobalMap.putAll(from.eqClassGlobalMap);
+ logicalProps.putAll(from.logicalProps);
+ }
+
@Override
public int getVarCounter() {
return varCounter;
@@ -354,4 +373,34 @@
public PlanStabilityVerifier getPlanStabilityVerifier() {
return planStabilityVerifier;
}
+
+ @Override
+ public IOptimizationContextFactory getOptimizationContextFactory() {
+ return optContextFactory;
+ }
+
+ @Override
+ public void setCompilerFactory(Object compilerFactory) {
+ this.compilerFactory = compilerFactory;
+ }
+
+ @Override
+ public Object getCompilerFactory() {
+ return compilerFactory;
+ }
+
+ protected static class VariableEvalSizeEnvironmentImpl implements IVariableEvalSizeEnvironment {
+
+ protected final Map<LogicalVariable, Integer> varSizeMap = new HashMap<>();
+
+ @Override
+ public void setVariableEvalSize(LogicalVariable var, int size) {
+ varSizeMap.put(var, size);
+ }
+
+ @Override
+ public int getVariableEvalSize(LogicalVariable var) {
+ return varSizeMap.get(var);
+ }
+ }
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/HeuristicOptimizer.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/HeuristicOptimizer.java
index f93d1f9..676b9c7 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/HeuristicOptimizer.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/HeuristicOptimizer.java
@@ -61,6 +61,7 @@
logPlanAt("Plan Before Optimization", Level.TRACE);
sanityCheckBeforeOptimization(plan);
runLogicalOptimizationSets(plan, logicalRewrites);
+ logPlanAt("LogicalPlan ", Level.TRACE);
computeSchemaBottomUpForPlan(plan);
runPhysicalOptimizationSets(plan, physicalRewrites);
logPlanAt("Plan After Optimization", Level.TRACE);
@@ -68,7 +69,7 @@
private void logPlanAt(String name, Level lvl) throws AlgebricksException {
if (AlgebricksConfig.ALGEBRICKS_LOGGER.isEnabled(lvl)) {
- String planStr = context.getPrettyPrinter().reset().printPlan(plan).toString();
+ String planStr = context.getPrettyPrinter().reset().printPlan(plan, true).toString();
AlgebricksConfig.ALGEBRICKS_LOGGER.log(lvl, name + ":\n" + LogRedactionUtil.userData(planStr));
}
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/IOptimizationContextFactory.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/IOptimizationContextFactory.java
index 1c41e9a..c81025d 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/IOptimizationContextFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/IOptimizationContextFactory.java
@@ -28,10 +28,12 @@
import org.apache.hyracks.api.exceptions.IWarningCollector;
public interface IOptimizationContextFactory {
- public IOptimizationContext createOptimizationContext(int varCounter,
+ IOptimizationContext createOptimizationContext(int varCounter,
IExpressionEvalSizeComputer expressionEvalSizeComputer,
IMergeAggregationExpressionFactory mergeAggregationExpressionFactory,
IExpressionTypeComputer expressionTypeComputer, IMissableTypeComputer missableTypeComputer,
IConflictingTypeResolver conflictintTypeResolver, PhysicalOptimizationConfig physicalOptimizationConfig,
AlgebricksPartitionConstraint clusterLocations, IWarningCollector warningCollector);
+
+ IOptimizationContext cloneOptimizationContext(IOptimizationContext oc);
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/IRuleSetKind.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/IRuleSetKind.java
new file mode 100644
index 0000000..5ce61ec
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/IRuleSetKind.java
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.algebricks.core.rewriter.base;
+
+public interface IRuleSetKind {
+ String name();
+}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java
index 9c9b21d..fb77ba0 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java
@@ -47,6 +47,10 @@
private static final String MIN_MEMORY_ALLOCATION = "MIN_MEMORY_ALLOCATION";
private static final String ARRAY_INDEX = "ARRAY_INDEX";
private static final String EXTERNAL_SCAN_BUFFER_SIZE = "EXTERNAL_SCAN_BUFFER_SIZE";
+ private static final String CBO = "CBO";
+ private static final String CBO_TEST = "CBO_TEST";
+ private static final String FORCE_JOIN_ORDER = "FORCE_JOIN_ORDER";
+ private static final String QUERY_PLAN_SHAPE = "QUERY_PLAN_SHAPE";
private final Properties properties = new Properties();
@@ -239,6 +243,44 @@
return getInt(EXTERNAL_SCAN_BUFFER_SIZE, AlgebricksConfig.EXTERNAL_SCAN_BUFFER_SIZE);
}
+ public boolean getCBOMode() {
+ return getBoolean(CBO, AlgebricksConfig.CBO_DEFAULT);
+ }
+
+ public boolean getCBOTestMode() {
+ return getBoolean(CBO_TEST, AlgebricksConfig.CBO_TEST_DEFAULT);
+ }
+
+ public boolean getForceJoinOrderMode() {
+ return getBoolean(FORCE_JOIN_ORDER, AlgebricksConfig.FORCE_JOIN_ORDER_DEFAULT);
+ }
+
+ public String getQueryPlanShapeMode() {
+ String queryPlanShapeMode = getString(QUERY_PLAN_SHAPE, AlgebricksConfig.QUERY_PLAN_SHAPE_DEFAULT);
+ if (!(queryPlanShapeMode.equals(AlgebricksConfig.QUERY_PLAN_SHAPE_ZIGZAG)
+ || queryPlanShapeMode.equals(AlgebricksConfig.QUERY_PLAN_SHAPE_LEFTDEEP)
+ || queryPlanShapeMode.equals(AlgebricksConfig.QUERY_PLAN_SHAPE_RIGHTDEEP))) {
+ return AlgebricksConfig.QUERY_PLAN_SHAPE_DEFAULT;
+ }
+ return queryPlanShapeMode;
+ }
+
+ public void setCBOMode(boolean cbo) {
+ setBoolean(CBO, cbo);
+ }
+
+ public void setCBOTestMode(boolean cboTest) {
+ setBoolean(CBO_TEST, cboTest);
+ }
+
+ public void setForceJoinOrderMode(boolean forceJoinOrder) {
+ setBoolean(FORCE_JOIN_ORDER, forceJoinOrder);
+ }
+
+ public void setQueryPlanShapeMode(String queryPlanShape) {
+ setString(QUERY_PLAN_SHAPE, queryPlanShape);
+ }
+
public void setExternalScanBufferSize(int bufferSize) {
setInt(EXTERNAL_SCAN_BUFFER_SIZE, bufferSize);
}
@@ -269,4 +311,13 @@
String value = properties.getProperty(property);
return value == null ? defaultValue : Boolean.parseBoolean(value);
}
+
+ private void setString(String property, String value) {
+ properties.setProperty(property, value);
+ }
+
+ private String getString(String property, String defaultValue) {
+ String value = properties.getProperty(property);
+ return value == null ? defaultValue : value;
+ }
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/test/java/org/apache/hyracks/algebricks/core/algebra/plan/PlanVerifierTestBase.java b/hyracks-fullstack/algebricks/algebricks-core/src/test/java/org/apache/hyracks/algebricks/core/algebra/plan/PlanVerifierTestBase.java
index 65d6460..2f7bc9b 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/test/java/org/apache/hyracks/algebricks/core/algebra/plan/PlanVerifierTestBase.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/test/java/org/apache/hyracks/algebricks/core/algebra/plan/PlanVerifierTestBase.java
@@ -71,7 +71,7 @@
String printOp(ILogicalOperator op) {
try {
- return planPrinter.reset().printOperator((AbstractLogicalOperator) op, false).toString();
+ return planPrinter.reset().printOperator((AbstractLogicalOperator) op, false, false).toString();
} catch (AlgebricksException e) {
throw new RuntimeException(e);
}
diff --git a/hyracks-fullstack/algebricks/algebricks-data/pom.xml b/hyracks-fullstack/algebricks/algebricks-data/pom.xml
index f48c59ce..8ca0765 100644
--- a/hyracks-fullstack/algebricks/algebricks-data/pom.xml
+++ b/hyracks-fullstack/algebricks/algebricks-data/pom.xml
@@ -25,7 +25,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>algebricks</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<licenses>
@@ -54,11 +54,6 @@
</dependency>
<dependency>
<groupId>org.apache.hyracks</groupId>
- <artifactId>hyracks-util</artifactId>
- <version>${project.version}</version>
- </dependency>
- <dependency>
- <groupId>org.apache.hyracks</groupId>
<artifactId>hyracks-api</artifactId>
<version>${project.version}</version>
</dependency>
diff --git a/hyracks-fullstack/algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/IResultSerializerFactoryProvider.java b/hyracks-fullstack/algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/IResultSerializerFactoryProvider.java
index 7525180..75fd791 100644
--- a/hyracks-fullstack/algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/IResultSerializerFactoryProvider.java
+++ b/hyracks-fullstack/algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/IResultSerializerFactoryProvider.java
@@ -32,10 +32,8 @@
* - A printer factory array to print the tuple containing different fields.
* @param writerFactory
* - A writer factory to write the serialized data to the print stream.
- * @param inputRecordDesc
- * - The record descriptor describing the input frame to be serialized.
* @return A new instance of result serialized appender.
*/
- public IResultSerializerFactory getAqlResultSerializerFactoryProvider(int[] fields,
- IPrinterFactory[] printerFactories, IAWriterFactory writerFactory);
+ public IResultSerializerFactory getResultSerializerFactoryProvider(int[] fields, IPrinterFactory[] printerFactories,
+ IAWriterFactory writerFactory);
}
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/pom.xml b/hyracks-fullstack/algebricks/algebricks-rewriter/pom.xml
index b0ea2fc..5297e17 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/pom.xml
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/pom.xml
@@ -23,7 +23,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>algebricks</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<licenses>
<license>
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/AbstractDecorrelationRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/AbstractDecorrelationRule.java
index 55c8400..dbdf6c4 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/AbstractDecorrelationRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/AbstractDecorrelationRule.java
@@ -92,6 +92,7 @@
protected void buildVarExprList(Collection<LogicalVariable> vars, IOptimizationContext context, GroupByOperator g,
List<Pair<LogicalVariable, Mutable<ILogicalExpression>>> outVeList) throws AlgebricksException {
SourceLocation sourceLoc = g.getSourceLocation();
+ Set<ILogicalOperator> visited = new HashSet<>();
for (LogicalVariable ov : vars) {
LogicalVariable newVar = context.newVar();
VariableReferenceExpression varExpr = new VariableReferenceExpression(newVar);
@@ -101,7 +102,8 @@
for (ILogicalPlan p : g.getNestedPlans()) {
for (Mutable<ILogicalOperator> r : p.getRoots()) {
OperatorManipulationUtil.substituteVarRec((AbstractLogicalOperator) r.getValue(), ov, newVar, true,
- context);
+ context, visited);
+ visited.clear();
}
}
}
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractCommonOperatorsRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractCommonOperatorsRule.java
index bd24644..fba095a 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractCommonOperatorsRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractCommonOperatorsRule.java
@@ -99,7 +99,7 @@
changed = false;
// applying the rewriting until fixpoint
topDownMaterialization(roots);
- genCandidates(context);
+ genCandidates();
removeTrivialShare();
if (!equivalenceClasses.isEmpty()) {
changed = rewrite(context);
@@ -110,9 +110,7 @@
equivalenceClasses.clear();
childrenToParents.clear();
opToCandidateInputs.clear();
- clusterMap.clear();
- clusterWaitForMap.clear();
- lastUsedClusterId = 0; // Resets lastUsedClusterId to 0.
+ resetPlanClusterState();
} while (changed);
roots.clear();
}
@@ -151,7 +149,7 @@
private boolean rewriteForOneEquivalentClass(List<Mutable<ILogicalOperator>> members, IOptimizationContext context)
throws AlgebricksException {
- List<Mutable<ILogicalOperator>> group = new ArrayList<Mutable<ILogicalOperator>>();
+ List<Mutable<ILogicalOperator>> group = new ArrayList<>();
boolean rewritten = false;
while (members.size() > 0) {
group.clear();
@@ -176,14 +174,14 @@
AbstractLogicalOperator aopCandidate = (AbstractLogicalOperator) candidate.getValue();
List<Mutable<ILogicalOperator>> originalCandidateParents = childrenToParents.get(candidate);
- rop.setExecutionMode(((AbstractLogicalOperator) candidate.getValue()).getExecutionMode());
+ rop.setExecutionMode(candidate.getValue().getExecutionMode());
if (aopCandidate.getOperatorTag() == LogicalOperatorTag.EXCHANGE) {
rop.getInputs().add(candidate);
} else {
AbstractLogicalOperator beforeExchange = new ExchangeOperator();
beforeExchange.setPhysicalOperator(new OneToOneExchangePOperator());
beforeExchange.setExecutionMode(rop.getExecutionMode());
- Mutable<ILogicalOperator> beforeExchangeRef = new MutableObject<ILogicalOperator>(beforeExchange);
+ Mutable<ILogicalOperator> beforeExchangeRef = new MutableObject<>(beforeExchange);
beforeExchange.getInputs().add(candidate);
context.computeAndSetTypeEnvironmentForOperator(beforeExchange);
beforeExchange.recomputeSchema();
@@ -202,7 +200,7 @@
AbstractLogicalOperator exchange = new ExchangeOperator();
exchange.setPhysicalOperator(new OneToOneExchangePOperator());
exchange.setExecutionMode(rop.getExecutionMode());
- MutableObject<ILogicalOperator> exchangeRef = new MutableObject<ILogicalOperator>(exchange);
+ MutableObject<ILogicalOperator> exchangeRef = new MutableObject<>(exchange);
exchange.getInputs().add(new MutableObject<>(rop));
rop.getOutputs().add(exchangeRef);
context.computeAndSetTypeEnvironmentForOperator(exchange);
@@ -246,10 +244,10 @@
exchOp.setPhysicalOperator(new OneToOneExchangePOperator());
exchOp.setExecutionMode(rop.getExecutionMode());
exchOp.getInputs().add(new MutableObject<>(rop));
- MutableObject<ILogicalOperator> exchOpRef = new MutableObject<ILogicalOperator>(exchOp);
+ MutableObject<ILogicalOperator> exchOpRef = new MutableObject<>(exchOp);
rop.getOutputs().add(exchOpRef);
assignOperator.getInputs().add(exchOpRef);
- projectOperator.getInputs().add(new MutableObject<ILogicalOperator>(assignOperator));
+ projectOperator.getInputs().add(new MutableObject<>(assignOperator));
// set the types
context.computeAndSetTypeEnvironmentForOperator(exchOp);
@@ -267,15 +265,15 @@
parentOp.getOperatorTag() == LogicalOperatorTag.PROJECT ? assignOperator : projectOperator;
if (parentOp.getPhysicalOperator().isMicroOperator()
|| parentOp.getOperatorTag() == LogicalOperatorTag.EXCHANGE) {
- parentOp.getInputs().set(index, new MutableObject<ILogicalOperator>(childOp));
+ parentOp.getInputs().set(index, new MutableObject<>(childOp));
} else {
// If the parent operator is a hyracks operator,
// an extra one-to-one exchange is needed.
AbstractLogicalOperator exchg = new ExchangeOperator();
exchg.setPhysicalOperator(new OneToOneExchangePOperator());
exchg.setExecutionMode(childOp.getExecutionMode());
- exchg.getInputs().add(new MutableObject<ILogicalOperator>(childOp));
- parentOp.getInputs().set(index, new MutableObject<ILogicalOperator>(exchg));
+ exchg.getInputs().add(new MutableObject<>(childOp));
+ parentOp.getInputs().set(index, new MutableObject<>(exchg));
context.computeAndSetTypeEnvironmentForOperator(exchg);
exchg.recomputeSchema();
}
@@ -285,10 +283,17 @@
}
cleanupPlan();
rewritten = true;
+ resetPlanClusterState();
}
return rewritten;
}
+ private void resetPlanClusterState() {
+ clusterMap.clear();
+ clusterWaitForMap.clear();
+ lastUsedClusterId = 0;
+ }
+
/**
* Cleans up the plan after combining similar branches into one branch making sure parents & children point to
* each other correctly.
@@ -348,17 +353,15 @@
}
}
- private void genCandidates(IOptimizationContext context) throws AlgebricksException {
- List<List<Mutable<ILogicalOperator>>> previousEquivalenceClasses =
- new ArrayList<List<Mutable<ILogicalOperator>>>();
+ private void genCandidates() throws AlgebricksException {
+ List<List<Mutable<ILogicalOperator>>> previousEquivalenceClasses = new ArrayList<>();
while (equivalenceClasses.size() > 0) {
previousEquivalenceClasses.clear();
for (List<Mutable<ILogicalOperator>> candidates : equivalenceClasses) {
- List<Mutable<ILogicalOperator>> candidatesCopy = new ArrayList<Mutable<ILogicalOperator>>();
- candidatesCopy.addAll(candidates);
+ List<Mutable<ILogicalOperator>> candidatesCopy = new ArrayList<>(candidates);
previousEquivalenceClasses.add(candidatesCopy);
}
- List<Mutable<ILogicalOperator>> currentLevelOpRefs = new ArrayList<Mutable<ILogicalOperator>>();
+ List<Mutable<ILogicalOperator>> currentLevelOpRefs = new ArrayList<>();
for (List<Mutable<ILogicalOperator>> candidates : equivalenceClasses) {
if (candidates.size() > 0) {
for (Mutable<ILogicalOperator> opRef : candidates) {
@@ -376,11 +379,11 @@
if (currentLevelOpRefs.size() == 0) {
break;
}
- prune(context);
+ prune();
}
if (equivalenceClasses.size() < 1 && previousEquivalenceClasses.size() > 0) {
equivalenceClasses.addAll(previousEquivalenceClasses);
- prune(context);
+ prune();
}
}
@@ -412,8 +415,7 @@
}
private void candidatesGrow(List<Mutable<ILogicalOperator>> opList, List<Mutable<ILogicalOperator>> candidates) {
- List<Mutable<ILogicalOperator>> previousCandidates = new ArrayList<Mutable<ILogicalOperator>>();
- previousCandidates.addAll(candidates);
+ List<Mutable<ILogicalOperator>> previousCandidates = new ArrayList<>(candidates);
candidates.clear();
boolean validCandidate = false;
for (Mutable<ILogicalOperator> op : opList) {
@@ -450,23 +452,18 @@
}
}
- private void prune(IOptimizationContext context) throws AlgebricksException {
- List<List<Mutable<ILogicalOperator>>> previousEquivalenceClasses =
- new ArrayList<List<Mutable<ILogicalOperator>>>();
+ private void prune() throws AlgebricksException {
+ List<List<Mutable<ILogicalOperator>>> previousEquivalenceClasses = new ArrayList<>();
for (List<Mutable<ILogicalOperator>> candidates : equivalenceClasses) {
- List<Mutable<ILogicalOperator>> candidatesCopy = new ArrayList<Mutable<ILogicalOperator>>();
- candidatesCopy.addAll(candidates);
+ List<Mutable<ILogicalOperator>> candidatesCopy = new ArrayList<>(candidates);
previousEquivalenceClasses.add(candidatesCopy);
}
equivalenceClasses.clear();
for (List<Mutable<ILogicalOperator>> candidates : previousEquivalenceClasses) {
boolean[] reserved = new boolean[candidates.size()];
- for (int i = 0; i < reserved.length; i++) {
- reserved[i] = false;
- }
for (int i = candidates.size() - 1; i >= 0; i--) {
- if (reserved[i] == false) {
- List<Mutable<ILogicalOperator>> equivalentClass = new ArrayList<Mutable<ILogicalOperator>>();
+ if (!reserved[i]) {
+ List<Mutable<ILogicalOperator>> equivalentClass = new ArrayList<>();
ILogicalOperator candidate = candidates.get(i).getValue();
equivalentClass.add(candidates.get(i));
for (int j = i - 1; j >= 0; j--) {
@@ -502,7 +499,7 @@
boolean worthMaterialization = worthMaterialization(group.get(0));
boolean requiresMaterialization;
// get clusterIds for each candidate in the group
- List<Integer> groupClusterIds = new ArrayList<Integer>(group.size());
+ List<Integer> groupClusterIds = new ArrayList<>(group.size());
for (int i = 0; i < group.size(); i++) {
groupClusterIds.add(clusterMap.get(group.get(i)).getValue());
}
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/InlineSingleReferenceVariablesRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/InlineSingleReferenceVariablesRule.java
index f072312..be169f4 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/InlineSingleReferenceVariablesRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/InlineSingleReferenceVariablesRule.java
@@ -27,7 +27,6 @@
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
/**
@@ -83,7 +82,7 @@
}
@Override
- protected boolean performBottomUpAction(AbstractLogicalOperator op) throws AlgebricksException {
+ protected boolean performBottomUpAction(ILogicalOperator op) throws AlgebricksException {
usedVars.clear();
VariableUtilities.getUsedVariables(op, usedVars);
for (LogicalVariable var : usedVars) {
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/InlineVariablesRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/InlineVariablesRule.java
index cec830e..6ad50e6 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/InlineVariablesRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/InlineVariablesRule.java
@@ -26,6 +26,7 @@
import java.util.Set;
import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableInt;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
@@ -37,7 +38,6 @@
import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.WindowOperator;
@@ -80,6 +80,10 @@
// set to prevent re-visiting a subtree from the other sides. Operators with multiple outputs are the ones that
// could be re-visited twice or more (e.g. replicate and split operators)
private final Map<ILogicalOperator, Boolean> subTreesDone = new HashMap<>();
+ // temporary set to get used variables
+ private final List<LogicalVariable> usedVars = new ArrayList<>();
+ // map of variables and the counts of how many times they were used
+ private final Map<LogicalVariable, MutableInt> usedVariableCounter = new HashMap<>();
@Override
public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
@@ -108,9 +112,10 @@
varAssignRhs.clear();
inlineVisitor.setContext(context);
subTreesDone.clear();
+ usedVariableCounter.clear();
}
- protected boolean performBottomUpAction(AbstractLogicalOperator op) throws AlgebricksException {
+ protected boolean performBottomUpAction(ILogicalOperator op) throws AlgebricksException {
// Only inline variables in operators that can deal with arbitrary expressions.
if (!op.requiresVariableReferenceExpressions()) {
inlineVisitor.setOperator(op);
@@ -125,27 +130,33 @@
private boolean inlineVariables(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
throws AlgebricksException {
- AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+ ILogicalOperator op = opRef.getValue();
// check if you have already visited the subtree rooted at this operator
if (subTreesDone.containsKey(op)) {
return subTreesDone.get(op);
}
+
+ // compute how many times a variable was used
+ computeUsedVariableCount(op);
+
// Update mapping from variables to expressions during top-down traversal.
if (op.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
AssignOperator assignOp = (AssignOperator) op;
List<LogicalVariable> vars = assignOp.getVariables();
List<Mutable<ILogicalExpression>> exprs = assignOp.getExpressions();
for (int i = 0; i < vars.size(); i++) {
+ LogicalVariable variable = vars.get(i);
ILogicalExpression expr = exprs.get(i).getValue();
// Ignore functions that are either in the doNotInline set or are non-functional
if (expr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
- if (doNotInlineFuncs.contains(funcExpr.getFunctionIdentifier()) || !funcExpr.isFunctional()) {
+ if (doNotInlineFuncs.contains(funcExpr.getFunctionIdentifier())
+ || skipNonFunctional(variable, funcExpr)) {
continue;
}
}
- varAssignRhs.put(vars.get(i), exprs.get(i).getValue());
+ varAssignRhs.put(variable, expr);
}
}
@@ -200,6 +211,54 @@
return modified;
}
+ /**
+ * Skip inlining non-pure functions if they are referenced more than once.
+ *
+ * @param variable assigned to the function
+ * @param expr of the assigned variable (potentially a non-pure one)
+ * @return true if inlining should be skipped, false if the non-pure functions can be inlined
+ */
+ private boolean skipNonFunctional(LogicalVariable variable, AbstractFunctionCallExpression expr) {
+ return !expr.isFunctional() && usedVariableCounter.containsKey(variable)
+ && usedVariableCounter.get(variable).getValue() > 1;
+ }
+
+ /**
+ * Computes how many times the variables in the plan were used. The variable counts
+ * help to determine if whether we can inline non-pure functions (e.g., current_date()) or not.
+ * Inlining non-pure functions can help unlocking other optimizations. For instance, we can pushdown
+ * limits and selects into data-scans or avoid create_query_uuid().
+ * <p>
+ * Non-pure functions can only be inlined if they were referenced once. For example,
+ * in the following plan, the function current_date() (or variable $$x) can be inlined as it was used
+ * once by select:
+ * <p>
+ * Before:
+ * select (get_year($$x) > 2000)
+ * * assign [$$x] <- [current_date()]
+ * After:
+ * select (get_year(current_date()) > 2000)
+ * <p>
+ * However, the following current_date() (or variable $$x) cannot be inlined as it referenced twice:
+ * select (get_year($$x) > 2000 && get_month($$x) == 11)
+ * * assign [$$x] <- [current_date()]
+ *
+ * @param op the logical operator that potentially uses one or more variables
+ */
+ private void computeUsedVariableCount(ILogicalOperator op) throws AlgebricksException {
+ if (op.getOperatorTag() == LogicalOperatorTag.SUBPLAN) {
+ // to avoid count a variable twice as this routine traverses the subplan
+ return;
+ }
+
+ usedVars.clear();
+ VariableUtilities.getUsedVariables(op, usedVars);
+ for (LogicalVariable variable : usedVars) {
+ MutableInt counter = usedVariableCounter.computeIfAbsent(variable, k -> new MutableInt(0));
+ counter.increment();
+ }
+ }
+
public static class InlineVariablesVisitor extends LogicalExpressionReferenceTransformVisitor
implements ILogicalExpressionReferenceTransform {
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PopulateResultMetadataRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PopulateResultMetadataRule.java
index e1b2e5d..1eda133 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PopulateResultMetadataRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PopulateResultMetadataRule.java
@@ -47,7 +47,7 @@
}
DistributeResultOperator dop = (DistributeResultOperator) op;
IResultMetadata resultMetadata = dop.getResultMetadata();
- if (resultMetadata.getOutputTypes() != null) {
+ if (resultMetadata == null || resultMetadata.getOutputTypes() != null) {
return false;
}
List<Mutable<ILogicalExpression>> exprList = dop.getExpressions();
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PushMapOperatorDownThroughProductRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PushMapOperatorDownThroughProductRule.java
index 2a009f0..5d466d5 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PushMapOperatorDownThroughProductRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PushMapOperatorDownThroughProductRule.java
@@ -75,6 +75,8 @@
op2Ref.setValue(b0);
b0Ref.setValue(op1);
opRef.setValue(op2);
+ context.computeAndSetTypeEnvironmentForOperator(op1);
+ context.computeAndSetTypeEnvironmentForOperator(op2);
return true;
} else {
Mutable<ILogicalOperator> b1Ref = op2.getInputs().get(1);
@@ -86,6 +88,8 @@
op2Ref.setValue(b1);
b1Ref.setValue(op1);
opRef.setValue(op2);
+ context.computeAndSetTypeEnvironmentForOperator(op1);
+ context.computeAndSetTypeEnvironmentForOperator(op2);
return true;
} else {
return false;
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/RemoveRedundantVariablesRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/RemoveRedundantVariablesRule.java
index 3760be5..1ed7170 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/RemoveRedundantVariablesRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/RemoveRedundantVariablesRule.java
@@ -138,9 +138,12 @@
if (expr.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
continue;
}
- VariableReferenceExpression rhsVarRefExpr = (VariableReferenceExpression) expr;
- // Update equivalence class map.
LogicalVariable lhs = assignOp.getVariables().get(i);
+ if (context.shouldNotBeInlined(lhs)) {
+ continue;
+ }
+ // Update equivalence class map.
+ VariableReferenceExpression rhsVarRefExpr = (VariableReferenceExpression) expr;
LogicalVariable rhs = rhsVarRefExpr.getVariableReference();
updateEquivalenceClassMap(lhs, rhs);
}
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/RemoveUnusedAssignAndAggregateRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/RemoveUnusedAssignAndAggregateRule.java
index 956f0b7..84346fd 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/RemoveUnusedAssignAndAggregateRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/RemoveUnusedAssignAndAggregateRule.java
@@ -42,6 +42,7 @@
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.RunningAggregateOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
@@ -163,9 +164,15 @@
if (!canRemoveOperator(op, opInSubplan, parentOp)) {
break;
}
- op = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
- opRef.setValue(op);
- assignVarsSetForThisOp = removeAssignVarFromConsideration(opRef);
+ if (op.getOperatorTag() == LogicalOperatorTag.AGGREGATE) {
+ op = new EmptyTupleSourceOperator();
+ context.computeAndSetTypeEnvironmentForOperator(op);
+ opRef.setValue(op);
+ } else {
+ op = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
+ opRef.setValue(op);
+ assignVarsSetForThisOp = removeAssignVarFromConsideration(opRef);
+ }
isTransformed = true;
}
@@ -445,7 +452,7 @@
private static boolean canRemoveOperator(ILogicalOperator op, boolean opInsideSubplan, ILogicalOperator parentOp) {
LogicalOperatorTag opTag = op.getOperatorTag();
- if (opTag == LogicalOperatorTag.AGGREGATE || opTag == LogicalOperatorTag.UNIONALL) {
+ if ((opTag == LogicalOperatorTag.AGGREGATE && opInsideSubplan) || opTag == LogicalOperatorTag.UNIONALL) {
return false;
}
if (!opInsideSubplan) {
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/IntroduceGroupByForSubplanRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/IntroduceGroupByForSubplanRule.java
index 391d03a..36cad77 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/IntroduceGroupByForSubplanRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/IntroduceGroupByForSubplanRule.java
@@ -331,6 +331,7 @@
List<Pair<LogicalVariable, Mutable<ILogicalExpression>>> outVeList) throws AlgebricksException {
SourceLocation sourceLoc = g.getSourceLocation();
Map<LogicalVariable, LogicalVariable> m = new HashMap<LogicalVariable, LogicalVariable>();
+ Set<ILogicalOperator> visited = new HashSet<>();
for (LogicalVariable ov : vars) {
LogicalVariable newVar = context.newVar();
ILogicalExpression varExpr = new VariableReferenceExpression(newVar);
@@ -340,11 +341,13 @@
for (ILogicalPlan p : g.getNestedPlans()) {
for (Mutable<ILogicalOperator> r : p.getRoots()) {
OperatorManipulationUtil.substituteVarRec((AbstractLogicalOperator) r.getValue(), ov, newVar, true,
- context);
+ context, visited);
+ visited.clear();
}
}
AbstractLogicalOperator opUnder = (AbstractLogicalOperator) g.getInputs().get(0).getValue();
- OperatorManipulationUtil.substituteVarRec(opUnder, ov, newVar, true, context);
+ OperatorManipulationUtil.substituteVarRec(opUnder, ov, newVar, true, context, visited);
+ visited.clear();
m.put(ov, newVar);
}
return m;
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/util/JoinUtils.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/util/JoinUtils.java
index 1ac7614..49ff483 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/util/JoinUtils.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/util/JoinUtils.java
@@ -32,12 +32,15 @@
import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.BroadcastExpressionAnnotation;
import org.apache.hyracks.algebricks.core.algebra.expressions.BroadcastExpressionAnnotation.BroadcastSide;
+import org.apache.hyracks.algebricks.core.algebra.expressions.HashJoinExpressionAnnotation;
+import org.apache.hyracks.algebricks.core.algebra.expressions.HashJoinExpressionAnnotation.BuildSide;
import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions.ComparisonKind;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.LogicalPropertiesVisitor;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
import org.apache.hyracks.algebricks.core.algebra.operators.physical.AbstractJoinPOperator.JoinPartitioningType;
import org.apache.hyracks.algebricks.core.algebra.operators.physical.HybridHashJoinPOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.physical.InMemoryHashJoinPOperator;
@@ -55,7 +58,7 @@
}
public static void setJoinAlgorithmAndExchangeAlgo(AbstractBinaryJoinOperator op, boolean topLevelOp,
- IOptimizationContext context) {
+ IOptimizationContext context) throws AlgebricksException {
if (!topLevelOp) {
throw new IllegalStateException("Micro operator not implemented for: " + op.getOperatorTag());
}
@@ -65,11 +68,39 @@
List<LogicalVariable> varsRight = op.getInputs().get(1).getValue().getSchema();
ILogicalExpression conditionExpr = op.getCondition().getValue();
if (isHashJoinCondition(conditionExpr, varsLeft, varsRight, sideLeft, sideRight)) {
- BroadcastSide side = getBroadcastJoinSide(conditionExpr);
- if (side == null) {
- setHashJoinOp(op, JoinPartitioningType.PAIRWISE, sideLeft, sideRight, context);
+ List<LogicalVariable> scanVarsLeft = new LinkedList<>();
+ List<LogicalVariable> scanVarsRight = new LinkedList<>();
+ VariableUtilities.getLiveVariablesInDescendantDataScans(op.getInputs().get(0).getValue(), scanVarsLeft);
+ VariableUtilities.getLiveVariablesInDescendantDataScans(op.getInputs().get(1).getValue(), scanVarsRight);
+ BroadcastSide broadcastSide = getBroadcastJoinSide(conditionExpr, scanVarsLeft, scanVarsRight, context);
+ if (broadcastSide == null) {
+ BuildSide buildSide = getHashJoinBuildSide(conditionExpr, scanVarsLeft, scanVarsRight, context);
+ if (buildSide == null) {
+ setHashJoinOp(op, JoinPartitioningType.PAIRWISE, sideLeft, sideRight, context);
+ } else {
+ switch (buildSide) {
+ case RIGHT:
+ setHashJoinOp(op, JoinPartitioningType.PAIRWISE, sideLeft, sideRight, context);
+ break;
+ case LEFT:
+ if (op.getJoinKind() == AbstractBinaryJoinOperator.JoinKind.INNER) {
+ Mutable<ILogicalOperator> opRef0 = op.getInputs().get(0);
+ Mutable<ILogicalOperator> opRef1 = op.getInputs().get(1);
+ ILogicalOperator tmp = opRef0.getValue();
+ opRef0.setValue(opRef1.getValue());
+ opRef1.setValue(tmp);
+ setHashJoinOp(op, JoinPartitioningType.PAIRWISE, sideRight, sideLeft, context);
+ } else {
+ setHashJoinOp(op, JoinPartitioningType.PAIRWISE, sideLeft, sideRight, context);
+ }
+ break;
+ default:
+ // This should never happen
+ throw new IllegalStateException(buildSide.toString());
+ }
+ }
} else {
- switch (side) {
+ switch (broadcastSide) {
case RIGHT:
setHashJoinOp(op, JoinPartitioningType.BROADCAST, sideLeft, sideRight, context);
break;
@@ -87,7 +118,7 @@
break;
default:
// This should never happen
- throw new IllegalStateException(side.toString());
+ throw new IllegalStateException(broadcastSide.toString());
}
}
} else {
@@ -188,7 +219,8 @@
}
}
- private static BroadcastSide getBroadcastJoinSide(ILogicalExpression e) {
+ private static BroadcastSide getBroadcastJoinSide(ILogicalExpression e, List<LogicalVariable> varsLeft,
+ List<LogicalVariable> varsRight, IOptimizationContext context) {
BroadcastSide side = null;
if (e.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
return null;
@@ -197,7 +229,7 @@
FunctionIdentifier fi = fexp.getFunctionIdentifier();
if (fi.equals(AlgebricksBuiltinFunctions.AND)) {
for (Mutable<ILogicalExpression> a : fexp.getArguments()) {
- BroadcastSide newSide = getBroadcastJoinSide(a.getValue());
+ BroadcastSide newSide = getBroadcastJoinSide(a.getValue(), varsLeft, varsRight, context);
if (side == null) {
side = newSide;
} else if (newSide != null && !newSide.equals(side)) {
@@ -206,9 +238,83 @@
}
return side;
} else {
- BroadcastExpressionAnnotation bcastAnnnotation = fexp.getAnnotation(BroadcastExpressionAnnotation.class);
- if (bcastAnnnotation != null) {
- return bcastAnnnotation.getBroadcastSide();
+ BroadcastExpressionAnnotation bcastAnnotation = fexp.getAnnotation(BroadcastExpressionAnnotation.class);
+ if (bcastAnnotation != null) {
+ BroadcastExpressionAnnotation.BroadcastSide bcastSide = bcastAnnotation.getBroadcastSide();
+ if (bcastSide != null) {
+ return bcastSide;
+ }
+ String broadcastObject = "$$" + bcastAnnotation.getName();
+ if (varsRight.stream().map(LogicalVariable::toString).anyMatch(v -> v.equals(broadcastObject))) {
+ bcastAnnotation.setBroadcastSide(BroadcastSide.RIGHT);
+ return bcastAnnotation.getBroadcastSide();
+ } else if (varsLeft.stream().map(LogicalVariable::toString).anyMatch(v -> v.equals(broadcastObject))) {
+ bcastAnnotation.setBroadcastSide(BroadcastSide.LEFT);
+ return bcastAnnotation.getBroadcastSide();
+ } else {
+ IWarningCollector warningCollector = context.getWarningCollector();
+ if (warningCollector.shouldWarn()) {
+ warningCollector.warn(Warning.of(e.getSourceLocation(), ErrorCode.INAPPLICABLE_HINT,
+ "broadcast hash join", "broadcast " + bcastAnnotation.getName()));
+ }
+ return null;
+ }
+ }
+ }
+ return null;
+ }
+
+ private static BuildSide getHashJoinBuildSide(ILogicalExpression e, List<LogicalVariable> varsLeft,
+ List<LogicalVariable> varsRight, IOptimizationContext context) {
+ BuildSide side = null;
+ if (e.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+ return null;
+ }
+ AbstractFunctionCallExpression fexp = (AbstractFunctionCallExpression) e;
+ FunctionIdentifier fi = fexp.getFunctionIdentifier();
+ if (fi.equals(AlgebricksBuiltinFunctions.AND)) {
+ for (Mutable<ILogicalExpression> a : fexp.getArguments()) {
+ BuildSide newSide = getHashJoinBuildSide(a.getValue(), varsLeft, varsRight, context);
+ if (side == null) {
+ side = newSide;
+ } else if (newSide != null && !newSide.equals(side)) {
+ return null;
+ }
+ }
+ return side;
+ } else {
+ HashJoinExpressionAnnotation hashJoinAnnotation = fexp.getAnnotation(HashJoinExpressionAnnotation.class);
+ if (hashJoinAnnotation != null) {
+ BuildSide buildSide = hashJoinAnnotation.getBuildSide();
+ if (buildSide != null) {
+ return buildSide;
+ }
+ boolean build =
+ (hashJoinAnnotation.getBuildOrProbe() == HashJoinExpressionAnnotation.BuildOrProbe.BUILD);
+ boolean probe =
+ (hashJoinAnnotation.getBuildOrProbe() == HashJoinExpressionAnnotation.BuildOrProbe.PROBE);
+
+ String buildOrProbeObject = "$$" + hashJoinAnnotation.getName();
+ if ((build && varsRight.stream().map(LogicalVariable::toString)
+ .anyMatch(v -> v.equals(buildOrProbeObject)))
+ || (probe && varsLeft.stream().map(LogicalVariable::toString)
+ .anyMatch(v -> v.equals(buildOrProbeObject)))) {
+ hashJoinAnnotation.setBuildSide(HashJoinExpressionAnnotation.BuildSide.RIGHT);
+ return hashJoinAnnotation.getBuildSide();
+ } else if ((build
+ && varsLeft.stream().map(LogicalVariable::toString).anyMatch(v -> v.equals(buildOrProbeObject)))
+ || (probe && varsRight.stream().map(LogicalVariable::toString)
+ .anyMatch(v -> v.equals(buildOrProbeObject)))) {
+ hashJoinAnnotation.setBuildSide(HashJoinExpressionAnnotation.BuildSide.LEFT);
+ return hashJoinAnnotation.getBuildSide();
+ } else {
+ IWarningCollector warningCollector = context.getWarningCollector();
+ if (warningCollector.shouldWarn()) {
+ warningCollector.warn(Warning.of(e.getSourceLocation(), ErrorCode.INAPPLICABLE_HINT,
+ "hash join", (build ? "build " : "probe ") + "with " + hashJoinAnnotation.getName()));
+ }
+ return null;
+ }
}
}
return null;
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/pom.xml b/hyracks-fullstack/algebricks/algebricks-runtime/pom.xml
index fb8afca..1b8eab5 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/pom.xml
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/pom.xml
@@ -25,7 +25,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>algebricks</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputOneFramePushRuntime.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputOneFramePushRuntime.java
index 9f4541f5..0c74260 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputOneFramePushRuntime.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputOneFramePushRuntime.java
@@ -74,7 +74,7 @@
}
protected void flushIfNotFailed() throws HyracksDataException {
- if (!failed && appender.getTupleCount() > 0) {
+ if (!failed && appender != null && appender.getTupleCount() > 0) {
flushAndReset();
}
}
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/serializer/ResultSerializerFactoryProvider.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/serializer/ResultSerializerFactoryProvider.java
index 763e6ff..90fa824 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/serializer/ResultSerializerFactoryProvider.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/serializer/ResultSerializerFactoryProvider.java
@@ -40,7 +40,7 @@
}
@Override
- public IResultSerializerFactory getAqlResultSerializerFactoryProvider(final int[] fields,
+ public IResultSerializerFactory getResultSerializerFactoryProvider(final int[] fields,
final IPrinterFactory[] printerFactories, final IAWriterFactory writerFactory) {
return new IResultSerializerFactory() {
private static final long serialVersionUID = 1L;
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/writers/SerializedDataWriterFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/writers/SerializedDataWriterFactory.java
index bc7634d..8ce4f2b 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/writers/SerializedDataWriterFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/writers/SerializedDataWriterFactory.java
@@ -32,7 +32,15 @@
public class SerializedDataWriterFactory implements IAWriterFactory {
- private static final long serialVersionUID = 1L;
+ private static final long serialVersionUID = 2L;
+
+ public static final SerializedDataWriterFactory WITHOUT_RECORD_DESCRIPTOR = new SerializedDataWriterFactory(false);
+
+ private final boolean writeRecordDescriptor;
+
+ public SerializedDataWriterFactory(boolean writeRecordDescriptor) {
+ this.writeRecordDescriptor = writeRecordDescriptor;
+ }
@Override
public IAWriter createWriter(final int[] fields, final PrintStream ps, IPrinterFactory[] printerFactories,
@@ -41,15 +49,17 @@
@Override
public void init() throws HyracksDataException {
- // dump the SerializerDeserializers to disk
- try {
- ByteArrayOutputStream baos = new ByteArrayOutputStream();
- ObjectOutputStream oos = new ObjectOutputStream(baos);
- oos.writeObject(inputRecordDescriptor);
- baos.writeTo(ps);
- oos.close();
- } catch (IOException e) {
- throw HyracksDataException.create(e);
+ if (writeRecordDescriptor) {
+ // dump the SerializerDeserializers to disk
+ try {
+ ByteArrayOutputStream baos = new ByteArrayOutputStream();
+ ObjectOutputStream oos = new ObjectOutputStream(baos);
+ oos.writeObject(inputRecordDescriptor);
+ baos.writeTo(ps);
+ oos.close();
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
}
}
diff --git a/hyracks-fullstack/algebricks/algebricks-tests/pom.xml b/hyracks-fullstack/algebricks/algebricks-tests/pom.xml
index d11df8a..34894cb 100644
--- a/hyracks-fullstack/algebricks/algebricks-tests/pom.xml
+++ b/hyracks-fullstack/algebricks/algebricks-tests/pom.xml
@@ -25,7 +25,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>algebricks</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/hyracks-fullstack/algebricks/pom.xml b/hyracks-fullstack/algebricks/pom.xml
index 498581f..1c1615f 100644
--- a/hyracks-fullstack/algebricks/pom.xml
+++ b/hyracks-fullstack/algebricks/pom.xml
@@ -26,7 +26,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>apache-hyracks</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/hyracks-fullstack/hyracks-fullstack-license/pom.xml b/hyracks-fullstack/hyracks-fullstack-license/pom.xml
index 7dd97f9..be37b55 100644
--- a/hyracks-fullstack/hyracks-fullstack-license/pom.xml
+++ b/hyracks-fullstack/hyracks-fullstack-license/pom.xml
@@ -23,7 +23,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>apache-hyracks</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/hyracks-fullstack/hyracks/hyracks-api/pom.xml b/hyracks-fullstack/hyracks/hyracks-api/pom.xml
index fe8fb24..131731d 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-api/pom.xml
@@ -24,7 +24,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<licenses>
@@ -65,6 +65,13 @@
<version>${project.version}</version>
</dependency>
<dependency>
+ <groupId>org.apache.hyracks</groupId>
+ <artifactId>hyracks-util</artifactId>
+ <version>${project.version}</version>
+ <type>test-jar</type>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
<groupId>org.mockito</groupId>
<artifactId>mockito-core</artifactId>
<scope>test</scope>
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksClientInterfaceFunctions.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksClientInterfaceFunctions.java
index 0ee3658..7a320b1 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksClientInterfaceFunctions.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksClientInterfaceFunctions.java
@@ -346,12 +346,19 @@
}
public static class WaitForCompletionFunction extends Function {
- private static final long serialVersionUID = 1L;
+ private static final long serialVersionUID = 2L;
private final JobId jobId;
+ private final List<String> statOperatorNames;
+
public WaitForCompletionFunction(JobId jobId) {
+ this(jobId, null);
+ }
+
+ public WaitForCompletionFunction(JobId jobId, List<String> statOperatorNames) {
this.jobId = jobId;
+ this.statOperatorNames = statOperatorNames;
}
@Override
@@ -362,6 +369,10 @@
public JobId getJobId() {
return jobId;
}
+
+ public List<String> getStatOperatorNames() {
+ return statOperatorNames;
+ }
}
public static class GetNodeControllersInfoFunction extends Function {
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientConnection.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientConnection.java
index 1118a68..be965eb 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientConnection.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientConnection.java
@@ -31,6 +31,7 @@
import org.apache.hyracks.api.job.JobInfo;
import org.apache.hyracks.api.job.JobSpecification;
import org.apache.hyracks.api.job.JobStatus;
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
/**
* Interface used by clients to communicate with the Hyracks Cluster Controller.
@@ -157,6 +158,19 @@
void waitForCompletion(JobId jobId) throws Exception;
/**
+ * Waits until the specified job has completed, either successfully or has
+ * encountered a permanent failure.
+ *
+ * @param jobId
+ * JobId of the Job
+ * @param statsOperatorNames
+ * names of the operators which stats should be returned
+ * @return stats for the specified operators
+ * @throws Exception
+ */
+ List<IOperatorStats> waitForCompletion(JobId jobId, List<String> statsOperatorNames) throws Exception;
+
+ /**
* Deploy files to the cluster
*
* @param files
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientInterface.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientInterface.java
index 4cc47d2..be47014 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientInterface.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientInterface.java
@@ -30,6 +30,7 @@
import org.apache.hyracks.api.job.JobId;
import org.apache.hyracks.api.job.JobInfo;
import org.apache.hyracks.api.job.JobStatus;
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
import org.apache.hyracks.api.topology.ClusterTopology;
public interface IHyracksClientInterface {
@@ -53,6 +54,8 @@
public void waitForCompletion(JobId jobId) throws Exception;
+ public List<IOperatorStats> waitForCompletion(JobId jobId, List<String> statOperatorNames) throws Exception;
+
public Map<String, NodeControllerInfo> getNodeControllersInfo() throws Exception;
public ClusterTopology getClusterTopology() throws Exception;
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/comm/IFrameAppender.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/comm/IFrameAppender.java
index 0fc24c7..98215bf 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/comm/IFrameAppender.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/comm/IFrameAppender.java
@@ -63,7 +63,7 @@
* @param writer the FrameWriter to write to and flush
* @throws HyracksDataException
*/
- public default void flush(IFrameWriter writer) throws HyracksDataException {
+ default void flush(IFrameWriter writer) throws HyracksDataException {
write(writer, true);
writer.flush();
}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/comm/IFrameReader.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/comm/IFrameReader.java
index 62eacc8..3e7e2ab 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/comm/IFrameReader.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/comm/IFrameReader.java
@@ -20,7 +20,7 @@
import org.apache.hyracks.api.exceptions.HyracksDataException;
-public interface IFrameReader {
+public interface IFrameReader extends AutoCloseable {
void open() throws HyracksDataException;
boolean nextFrame(IFrame frame) throws HyracksDataException;
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/comm/IFrameWriter.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/comm/IFrameWriter.java
index 19d7afb..a49f226 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/comm/IFrameWriter.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/comm/IFrameWriter.java
@@ -56,7 +56,7 @@
* Note: If the call to {@link IFrameWriter#open()} failed, the {@link IFrameWriter#close()} must still be called by the
* producer.
*/
-public interface IFrameWriter {
+public interface IFrameWriter extends AutoCloseable {
/**
* First call to allocate any resources.
*/
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/comm/IPartitionCollector.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/comm/IPartitionCollector.java
index 4a977ec..7b52875 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/comm/IPartitionCollector.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/comm/IPartitionCollector.java
@@ -25,7 +25,7 @@
import org.apache.hyracks.api.job.JobId;
import org.apache.hyracks.api.partitions.PartitionId;
-public interface IPartitionCollector {
+public interface IPartitionCollector extends AutoCloseable {
public JobId getJobId();
public ConnectorDescriptorId getConnectorId();
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/context/IHyracksJobletContext.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/context/IHyracksJobletContext.java
index 4d324f4..422782e 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/context/IHyracksJobletContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/context/IHyracksJobletContext.java
@@ -44,4 +44,6 @@
Class<?> loadClass(String className) throws HyracksException;
ClassLoader getClassLoader() throws HyracksException;
+
+ long nextUniqueId();
}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/context/IHyracksTaskContext.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/context/IHyracksTaskContext.java
index 4561af1..ad5624c 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/context/IHyracksTaskContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/context/IHyracksTaskContext.java
@@ -42,6 +42,8 @@
TaskAttemptId getTaskAttemptId();
+ int getPartitionCount();
+
ICounterContext getCounterContext();
ExecutorService getExecutorService();
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java
index 7291473..8170f07 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java
@@ -154,6 +154,7 @@
PARSING_ERROR(124),
INVALID_INVERTED_LIST_TYPE_TRAITS(125),
ILLEGAL_STATE(126),
+ INVALID_STRING_UNICODE(127),
// Compilation error codes.
RULECOLLECTION_NOT_INSTANCE_OF_LIST(10000),
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksException.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksException.java
index 977e5d2..12f1095 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksException.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksException.java
@@ -158,6 +158,10 @@
return message;
}
+ public String getMessageNoCode() {
+ return ErrorMessageUtil.getMessageNoCode(component, getMessage());
+ }
+
@Override
public String toString() {
return getLocalizedMessage();
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/NoOpWarningCollector.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/NoOpWarningCollector.java
new file mode 100644
index 0000000..caf2464
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/NoOpWarningCollector.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.api.exceptions;
+
+public final class NoOpWarningCollector implements IWarningCollector {
+
+ public static final IWarningCollector INSTANCE = new NoOpWarningCollector();
+
+ private NoOpWarningCollector() {
+ }
+
+ @Override
+ public void warn(Warning warning) {
+ // no-op
+ }
+
+ @Override
+ public boolean shouldWarn() {
+ return false;
+ }
+
+ @Override
+ public long getTotalWarningsCount() {
+ return 0;
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/IJobLifecycleListener.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/IJobLifecycleListener.java
index 338c331..6773dde 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/IJobLifecycleListener.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/IJobLifecycleListener.java
@@ -21,6 +21,7 @@
import java.util.List;
import org.apache.hyracks.api.exceptions.HyracksException;
+import org.apache.hyracks.api.job.resource.IJobCapacityController;
/**
* A listener for job related events
@@ -29,27 +30,33 @@
/**
* Notify the listener that a job has been created
*
- * @param jobId
- * @param spec
- * @throws HyracksException
+ * @param jobId the job id
+ * @param spec the job specification
+ * @param status the status of the job; whether it will be executed or queued
+ * @throws HyracksException HyracksException
*/
- void notifyJobCreation(JobId jobId, JobSpecification spec) throws HyracksException;
+ void notifyJobCreation(JobId jobId, JobSpecification spec, IJobCapacityController.JobSubmissionStatus status)
+ throws HyracksException;
/**
* Notify the listener that the job has started on the cluster controller
*
- * @param jobId
- * @throws HyracksException
+ * @param jobId the job id
+ * @param spec the job specification
+ *
+ * @throws HyracksException HyracksException
*/
- void notifyJobStart(JobId jobId) throws HyracksException;
+ void notifyJobStart(JobId jobId, JobSpecification spec) throws HyracksException;
/**
* Notify the listener that the job has been terminated, passing exceptions in case of failure
*
- * @param jobId
- * @param jobStatus
- * @param exceptions
- * @throws HyracksException
+ * @param jobId the job id
+ * @param spec the job specification
+ * @param jobStatus the job status
+ * @param exceptions the job exceptions
+ * @throws HyracksException HyracksException
*/
- void notifyJobFinish(JobId jobId, JobStatus jobStatus, List<Exception> exceptions) throws HyracksException;
+ void notifyJobFinish(JobId jobId, JobSpecification spec, JobStatus jobStatus, List<Exception> exceptions)
+ throws HyracksException;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/JobSpecification.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/JobSpecification.java
index 58336a0..6d0ce6b 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/JobSpecification.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/JobSpecification.java
@@ -79,6 +79,8 @@
private long maxWarnings;
+ private String requestId;
+
private IJobletEventListenerFactory jobletEventListenerFactory;
private IGlobalJobDataFactory globalJobDataFactory;
@@ -254,6 +256,14 @@
this.connectorPolicyAssignmentPolicy = connectorPolicyAssignmentPolicy;
}
+ public void setRequestId(String requestId) {
+ this.requestId = requestId;
+ }
+
+ public String getRequestId() {
+ return requestId;
+ }
+
public void setFrameSize(int frameSize) {
this.frameSize = frameSize;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/profiling/IOperatorStats.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/profiling/IOperatorStats.java
index 181249a..0d38fac 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/profiling/IOperatorStats.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/profiling/IOperatorStats.java
@@ -19,6 +19,7 @@
package org.apache.hyracks.api.job.profiling;
import java.io.Serializable;
+import java.util.Map;
import org.apache.hyracks.api.io.IWritable;
import org.apache.hyracks.api.job.profiling.counters.ICounter;
@@ -46,4 +47,10 @@
* @return A counter used to track the number of pages pinned by an opeartor
*/
ICounter getDiskIoCounter();
+
+ void updateIndexesStats(Map<String, IndexStats> indexesStats);
+
+ Map<String, IndexStats> getIndexesStats();
+
+ void updateFrom(IOperatorStats stats);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/profiling/IStatsCollector.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/profiling/IStatsCollector.java
index 8930d34..c903c68 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/profiling/IStatsCollector.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/profiling/IStatsCollector.java
@@ -43,6 +43,13 @@
IOperatorStats getOrAddOperatorStats(String operatorName);
/**
+ * @param operatorName
+ * @return {@link IOperatorStats} for the operator with name <code>operatorName</code>
+ * if it already exists, or {@code null} if the operator does not exist
+ */
+ IOperatorStats getOperatorStats(String operatorName);
+
+ /**
* Get every registered operator stats object
* @return All registered operators, and their collected stats, with the names as keys and stats as values
*/
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/profiling/IndexStats.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/profiling/IndexStats.java
new file mode 100644
index 0000000..0c471ef
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/profiling/IndexStats.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.api.job.profiling;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.Serializable;
+
+import org.apache.hyracks.api.com.job.profiling.counters.Counter;
+import org.apache.hyracks.api.io.IWritable;
+import org.apache.hyracks.api.job.profiling.counters.ICounter;
+
+public class IndexStats implements IWritable, Serializable {
+
+ private static final long serialVersionUID = 1L;
+
+ private final ICounter numPages;
+ private String indexName;
+
+ public IndexStats(String indexName, long numPages) {
+ this.indexName = indexName;
+ this.numPages = new Counter("numPages");
+ this.numPages.set(numPages);
+ }
+
+ public static IndexStats create(DataInput input) throws IOException {
+ String indexName = input.readUTF();
+ long numPages = input.readLong();
+ return new IndexStats(indexName, numPages);
+ }
+
+ @Override
+ public void writeFields(DataOutput output) throws IOException {
+ output.writeUTF(indexName);
+ output.writeLong(numPages.get());
+ }
+
+ @Override
+ public void readFields(DataInput input) throws IOException {
+ indexName = input.readUTF();
+ numPages.set(input.readLong());
+ }
+
+ public void updateNumPages(long delta) {
+ numPages.update(delta);
+ }
+
+ public long getNumPages() {
+ return numPages.get();
+ }
+
+ @Override
+ public String toString() {
+ return "IndexStats{indexName='" + indexName + "', numPages=" + numPages.get() + '}';
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/profiling/OperatorStats.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/profiling/OperatorStats.java
index 08c1adc..a3a5073 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/profiling/OperatorStats.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/profiling/OperatorStats.java
@@ -21,6 +21,8 @@
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
import org.apache.hyracks.api.com.job.profiling.counters.Counter;
import org.apache.hyracks.api.job.profiling.counters.ICounter;
@@ -32,6 +34,7 @@
public final ICounter tupleCounter;
public final ICounter timeCounter;
public final ICounter diskIoCounter;
+ private final Map<String, IndexStats> indexesStats;
public OperatorStats(String operatorName) {
if (operatorName == null || operatorName.isEmpty()) {
@@ -41,6 +44,7 @@
tupleCounter = new Counter("tupleCounter");
timeCounter = new Counter("timeCounter");
diskIoCounter = new Counter("diskIoCounter");
+ indexesStats = new HashMap<>();
}
public static IOperatorStats create(DataInput input) throws IOException {
@@ -71,11 +75,42 @@
}
@Override
+ public void updateIndexesStats(Map<String, IndexStats> stats) {
+ if (stats == null) {
+ return;
+ }
+ for (Map.Entry<String, IndexStats> stat : stats.entrySet()) {
+ String indexName = stat.getKey();
+ IndexStats indexStat = stat.getValue();
+ IndexStats existingIndexStat = indexesStats.get(indexName);
+ if (existingIndexStat == null) {
+ indexesStats.put(indexName, new IndexStats(indexName, indexStat.getNumPages()));
+ } else {
+ existingIndexStat.updateNumPages(indexStat.getNumPages());
+ }
+ }
+ }
+
+ @Override
+ public Map<String, IndexStats> getIndexesStats() {
+ return indexesStats;
+ }
+
+ @Override
+ public void updateFrom(IOperatorStats stats) {
+ tupleCounter.update(stats.getTupleCounter().get());
+ timeCounter.update(stats.getTimeCounter().get());
+ diskIoCounter.update(stats.getDiskIoCounter().get());
+ updateIndexesStats(stats.getIndexesStats());
+ }
+
+ @Override
public void writeFields(DataOutput output) throws IOException {
output.writeUTF(operatorName);
output.writeLong(tupleCounter.get());
output.writeLong(timeCounter.get());
output.writeLong(diskIoCounter.get());
+ writeIndexesStats(output);
}
@Override
@@ -83,11 +118,30 @@
tupleCounter.set(input.readLong());
timeCounter.set(input.readLong());
diskIoCounter.set(input.readLong());
+ readIndexesStats(input);
+ }
+
+ private void writeIndexesStats(DataOutput output) throws IOException {
+ output.writeInt(indexesStats.size());
+ for (Map.Entry<String, IndexStats> indexStat : indexesStats.entrySet()) {
+ output.writeUTF(indexStat.getKey());
+ indexStat.getValue().writeFields(output);
+ }
+ }
+
+ private void readIndexesStats(DataInput input) throws IOException {
+ int numIndexes = input.readInt();
+ for (int i = 0; i < numIndexes; i++) {
+ String indexName = input.readUTF();
+ IndexStats indexStats = IndexStats.create(input);
+ indexesStats.put(indexName, indexStats);
+ }
}
@Override
public String toString() {
return "{ " + "\"operatorName\": \"" + operatorName + "\", " + "\"" + tupleCounter.getName() + "\": "
- + tupleCounter.get() + ", \"" + timeCounter.getName() + "\": " + timeCounter.get() + " }";
+ + tupleCounter.get() + ", \"" + timeCounter.getName() + "\": " + timeCounter.get() + ", \""
+ + ", \"indexStats\": \"" + indexesStats + "\" }";
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/resource/DefaultJobCapacityController.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/resource/DefaultJobCapacityController.java
index 9e38a20..b18bcb1 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/resource/DefaultJobCapacityController.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/resource/DefaultJobCapacityController.java
@@ -24,6 +24,11 @@
public class DefaultJobCapacityController implements IJobCapacityController {
public static final DefaultJobCapacityController INSTANCE = new DefaultJobCapacityController();
+ private static final IClusterCapacity CAPACITY = new ClusterCapacity();
+ static {
+ CAPACITY.setAggregatedCores(Integer.MAX_VALUE);
+ CAPACITY.setAggregatedMemoryByteSize(Long.MAX_VALUE);
+ }
private DefaultJobCapacityController() {
}
@@ -37,4 +42,9 @@
public void release(JobSpecification job) {
// No operation here.
}
+
+ @Override
+ public IReadOnlyClusterCapacity getClusterCapacity() {
+ return CAPACITY;
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/resource/IJobCapacityController.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/resource/IJobCapacityController.java
index 5fa4bd9..f88baa2 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/resource/IJobCapacityController.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/resource/IJobCapacityController.java
@@ -57,4 +57,10 @@
*/
void release(JobSpecification job);
+ /**
+ * The cluster current capacity.
+ *
+ * @return the cluster current capacity.
+ */
+ IReadOnlyClusterCapacity getClusterCapacity();
}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java
index e43d72a..10af2fd 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java
@@ -30,8 +30,10 @@
import java.util.Queue;
import java.util.Set;
import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CancellationException;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Future;
+import java.util.concurrent.RejectedExecutionException;
import java.util.concurrent.Semaphore;
import org.apache.commons.collections4.MapUtils;
@@ -240,6 +242,12 @@
}
} catch (ExecutionException e) {
root = e.getCause();
+ } catch (CancellationException | RejectedExecutionException e) {
+ root = e;
+ // if a task has been cancelled or was rejected for execution, the executor has shut down and will no longer
+ // start tasks; adjust the semaphores accordingly to allow cancelTasks() to run without getting blocked.
+ completeSemaphore.release(-startSemaphore.drainPermits() + 1);
+ startSemaphore.release();
} catch (Throwable e) { // NOSONAR: Must catch all causes of failure
root = e;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/CleanupUtils.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/CleanupUtils.java
index 220311e..43fa1a7 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/CleanupUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/CleanupUtils.java
@@ -102,6 +102,41 @@
}
}
+ public static Throwable close(AutoCloseable[] closables, Throwable root) {
+ return close(closables, root, false);
+ }
+
+ public static Throwable closeSilently(AutoCloseable[] closables, Throwable root) {
+ return close(closables, root, true);
+ }
+
+ /**
+ * Close the AutoCloseable and suppress any Throwable thrown by the close call.
+ * This method must NEVER throw any Throwable
+ *
+ * @param closables
+ * the resource to close
+ * @param root
+ * the first exception encountered during release of resources
+ * @return the root Throwable if not null or a new Throwable if any was thrown, otherwise, it returns null
+ */
+ private static Throwable close(AutoCloseable[] closables, Throwable root, boolean silent) {
+ if (closables != null) {
+ for (AutoCloseable closable : closables) {
+ root = close(closable, root, silent);
+ }
+ }
+ return root;
+ }
+
+ public static Throwable close(AutoCloseable closable, Throwable root) {
+ return close(closable, root, false);
+ }
+
+ public static Throwable closeSilently(AutoCloseable closable, Throwable root) {
+ return close(closable, root, true);
+ }
+
/**
* Close the AutoCloseable and suppress any Throwable thrown by the close call.
* This method must NEVER throw any Throwable
@@ -112,16 +147,18 @@
* the first exception encountered during release of resources
* @return the root Throwable if not null or a new Throwable if any was thrown, otherwise, it returns null
*/
- public static Throwable close(AutoCloseable closable, Throwable root) {
+ private static Throwable close(AutoCloseable closable, Throwable root, boolean silent) {
if (closable != null) {
try {
closable.close();
} catch (Throwable th) { // NOSONAR Will be suppressed
- try {
- LOGGER.log(ExceptionUtils.causedByInterrupt(th) ? Level.DEBUG : Level.WARN,
- "Failure closing a closeable resource", th);
- } catch (Throwable loggingFailure) { // NOSONAR: Ignore catching Throwable
- // NOSONAR ignore logging failure
+ if (!silent) {
+ try {
+ LOGGER.log(ExceptionUtils.causedByInterrupt(th) ? Level.DEBUG : Level.WARN,
+ "Failure closing a closeable resource", th);
+ } catch (Throwable loggingFailure) { // NOSONAR: Ignore catching Throwable
+ // NOSONAR ignore logging failure
+ }
}
root = ExceptionUtils.suppress(root, th); // NOSONAR
}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ErrorMessageUtil.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ErrorMessageUtil.java
index 70b13fa..cb0d579 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ErrorMessageUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ErrorMessageUtil.java
@@ -125,6 +125,13 @@
}
}
+ public static String getMessageNoCode(String component, String message) {
+ if (NONE.equals(component)) {
+ return message;
+ }
+ return message.substring(message.indexOf(":") + 2);
+ }
+
public static String getCauseMessage(Throwable t) {
if (t instanceof IFormattedException) {
return t.getMessage();
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ExceptionUtils.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ExceptionUtils.java
index 5c1946a..d616eb5 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ExceptionUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ExceptionUtils.java
@@ -207,4 +207,16 @@
public static String getMessageOrToString(Throwable e) {
return e instanceof IFormattedException ? e.getMessage() : e.toString();
}
+
+ /**
+ * Checks if the error code of the throwable is of the provided type
+ *
+ * @param throwable throwable with error code
+ * @param code error code to match against
+ *
+ * @return true if error code matches, false otherwise
+ */
+ public static boolean isErrorCode(HyracksDataException throwable, ErrorCode code) {
+ return throwable.getError().isPresent() && throwable.getError().get() == code;
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/HyracksThrowingAction.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/HyracksThrowingAction.java
new file mode 100644
index 0000000..7e3d599
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/HyracksThrowingAction.java
@@ -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.
+ */
+package org.apache.hyracks.api.util;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+@FunctionalInterface
+public interface HyracksThrowingAction {
+ void run() throws HyracksDataException; // NOSONAR
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/InvokeUtil.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/InvokeUtil.java
index 7d04cf2..d331ab2 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/InvokeUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/InvokeUtil.java
@@ -29,8 +29,10 @@
import org.apache.hyracks.util.ComputingAction;
import org.apache.hyracks.util.IDelay;
import org.apache.hyracks.util.IOInterruptibleAction;
+import org.apache.hyracks.util.IOThrowingAction;
import org.apache.hyracks.util.IRetryPolicy;
import org.apache.hyracks.util.InterruptibleAction;
+import org.apache.hyracks.util.InterruptibleSupplier;
import org.apache.hyracks.util.Span;
import org.apache.hyracks.util.ThrowingAction;
import org.apache.logging.log4j.Level;
@@ -93,6 +95,27 @@
}
/**
+ * Executes the passed interruptible supplier, retrying if the operation is interrupted. Once the interruptible
+ * supplier completes, the current thread will be re-interrupted, if the original operation was interrupted.
+ */
+ public static <T> T getUninterruptibly(InterruptibleSupplier<T> interruptible) {
+ boolean interrupted = Thread.interrupted();
+ try {
+ while (true) {
+ try {
+ return interruptible.get();
+ } catch (InterruptedException e) { // NOSONAR- we will re-interrupt the thread during unwind
+ interrupted = true;
+ }
+ }
+ } finally {
+ if (interrupted) {
+ Thread.currentThread().interrupt();
+ }
+ }
+ }
+
+ /**
* Executes the passed interruptible, retrying if the operation is interrupted.
*
* @return true if the original operation was interrupted, otherwise false
@@ -188,7 +211,7 @@
}
}
- @SuppressWarnings({ "squid:S1181", "squid:S1193" }) // catching Throwable, instanceof of exception
+ @SuppressWarnings({ "squid:S1181", "squid:S1193", "ConstantConditions" }) // catching Throwable, instanceofs
public static void tryWithCleanups(ThrowingAction action, ThrowingAction... cleanups) throws Exception {
Throwable savedT = null;
boolean suppressedInterrupted = false;
@@ -225,6 +248,79 @@
}
}
+ @SuppressWarnings({ "squid:S1181", "squid:S1193", "ConstantConditions" }) // catching Throwable, instanceofs
+ public static void tryIoWithCleanups(IOThrowingAction action, IOThrowingAction... cleanups) throws IOException {
+ Throwable savedT = null;
+ boolean suppressedInterrupted = false;
+ try {
+ action.run();
+ } catch (Throwable t) {
+ savedT = t;
+ } finally {
+ for (IOThrowingAction cleanup : cleanups) {
+ try {
+ cleanup.run();
+ } catch (Throwable t) {
+ if (savedT != null) {
+ savedT.addSuppressed(t);
+ suppressedInterrupted = suppressedInterrupted || t instanceof InterruptedException;
+ } else {
+ savedT = t;
+ }
+ }
+ }
+ }
+ if (savedT == null) {
+ return;
+ }
+ if (suppressedInterrupted) {
+ Thread.currentThread().interrupt();
+ }
+ if (savedT instanceof Error) {
+ throw (Error) savedT;
+ } else if (savedT instanceof IOException) {
+ throw (IOException) savedT;
+ } else {
+ throw HyracksDataException.create(savedT);
+ }
+ }
+
+ @SuppressWarnings({ "squid:S1181", "squid:S1193", "ConstantConditions" }) // catching Throwable, instanceofs
+ public static void tryHyracksWithCleanups(HyracksThrowingAction action, HyracksThrowingAction... cleanups)
+ throws HyracksDataException {
+ Throwable savedT = null;
+ boolean suppressedInterrupted = false;
+ try {
+ action.run();
+ } catch (Throwable t) {
+ savedT = t;
+ } finally {
+ for (HyracksThrowingAction cleanup : cleanups) {
+ try {
+ cleanup.run();
+ } catch (Throwable t) {
+ if (savedT != null) {
+ savedT.addSuppressed(t);
+ suppressedInterrupted = suppressedInterrupted || t instanceof InterruptedException;
+ } else {
+ savedT = t;
+ }
+ }
+ }
+ }
+ if (savedT == null) {
+ return;
+ }
+ if (suppressedInterrupted) {
+ Thread.currentThread().interrupt();
+ }
+ if (savedT instanceof Error) {
+ throw (Error) savedT;
+ } else {
+ throw HyracksDataException.create(savedT);
+ }
+ }
+
/**
* Runs the supplied action, after suspending any pending interruption. An error will be logged if
* the action is itself interrupted.
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/SingleThreadEventProcessor.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/SingleThreadEventProcessor.java
index 85019c5..b07cb71 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/SingleThreadEventProcessor.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/SingleThreadEventProcessor.java
@@ -31,7 +31,7 @@
private static final Logger LOGGER = LogManager.getLogger();
protected final String name;
private final LinkedBlockingQueue<T> eventInbox;
- private volatile Thread executorThread;
+ private final Thread executorThread;
private volatile boolean stopped = false;
public SingleThreadEventProcessor(String threadName) {
@@ -43,18 +43,22 @@
@Override
public final void run() {
- LOGGER.log(Level.INFO, "Started " + Thread.currentThread().getName());
+ LOGGER.info("Started {}", name);
while (!stopped) {
try {
T event = eventInbox.take();
handle(event);
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
+ if (!stopped) {
+ LOGGER.warn("Interrupt while waiting for an event and !stopped");
+ }
+ break;
} catch (Exception e) {
- LOGGER.log(Level.ERROR, "Error handling an event", e);
+ LOGGER.error("Error handling an event", e);
}
}
- LOGGER.log(Level.WARN, "Stopped " + Thread.currentThread().getName());
+ LOGGER.info("Stopped {}", name);
}
protected abstract void handle(T event) throws Exception; //NOSONAR
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/string/UTF8StringReader.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/util/string/UTF8StringReader.java
similarity index 100%
rename from hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/string/UTF8StringReader.java
rename to hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/util/string/UTF8StringReader.java
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/util/string/UTF8StringUtil.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/util/string/UTF8StringUtil.java
new file mode 100644
index 0000000..4fc503d
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/util/string/UTF8StringUtil.java
@@ -0,0 +1,711 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.util.string;
+
+import static org.apache.hyracks.api.exceptions.ErrorCode.INVALID_STRING_UNICODE;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.UTFDataFormatException;
+import java.lang.ref.SoftReference;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.util.encoding.VarLenIntEncoderDecoder;
+
+/**
+ * A helper package to operate the UTF8String in Hyracks.
+ * Most of the codes were migrated from asterix-fuzzyjoin and hyracks-storage-am-invertedindex
+ */
+public class UTF8StringUtil {
+
+ public static final String MALFORMED_BYTES = "malformed bytes";
+ public static final String LOW_SURROGATE_WITHOUT_HIGH_SURROGATE =
+ "got a low surrogate without a leading high surrogate";
+ public static final String HIGH_SURROGATE_WITHOUT_LOW_SURROGATE =
+ "got a high surrogate without a following low surrogate";
+
+ private UTF8StringUtil() {
+ }
+
+ public static char charAt(byte[] b, int s) {
+ if (s >= b.length) {
+ throw new ArrayIndexOutOfBoundsException(s);
+ }
+ int c = b[s] & 0xff;
+ switch (c >> 4) {
+ case 0:
+ case 1:
+ case 2:
+ case 3:
+ case 4:
+ case 5:
+ case 6:
+ case 7:
+ return (char) c;
+
+ case 12:
+ case 13:
+ return (char) (((c & 0x1F) << 6) | ((b[s + 1]) & 0x3F));
+
+ case 14:
+ return (char) (((c & 0x0F) << 12) | (((b[s + 1]) & 0x3F) << 6) | (b[s + 2] & 0x3F));
+
+ default:
+ throw new IllegalArgumentException();
+ }
+ }
+
+ public static int charSize(byte[] b, int s) {
+ int c = b[s] & 0xff;
+ switch (c >> 4) {
+ case 0:
+ case 1:
+ case 2:
+ case 3:
+ case 4:
+ case 5:
+ case 6:
+ case 7:
+ return 1;
+
+ case 12:
+ case 13:
+ return 2;
+
+ case 14:
+ return 3;
+
+ default:
+ throw new IllegalStateException();
+ }
+ }
+
+ public static int codePointAt(byte[] b, int s) throws HyracksDataException {
+ char c1 = charAt(b, s);
+
+ if (Character.isLowSurrogate(c1)) {
+ // In this case, the index s doesn't point to a correct position
+ throw HyracksDataException.create(INVALID_STRING_UNICODE, LOW_SURROGATE_WITHOUT_HIGH_SURROGATE);
+ }
+
+ if (Character.isHighSurrogate(c1)) {
+ // If c1 is the a high surrogate and also the last char in the byte array (that means the byte array is somehow illegal),
+ // then an exception will be thrown because there is no low surrogate (c2) available in the byte array
+ s += charSize(b, s);
+ char c2 = charAt(b, s);
+ if (Character.isLowSurrogate(c2)) {
+ return Character.toCodePoint(c1, c2);
+ } else {
+ throw HyracksDataException.create(INVALID_STRING_UNICODE, HIGH_SURROGATE_WITHOUT_LOW_SURROGATE);
+ }
+ }
+
+ return c1;
+ }
+
+ public static int codePointSize(byte[] b, int s) throws HyracksDataException {
+ char c1 = charAt(b, s);
+ int size1 = charSize(b, s);
+
+ if (Character.isLowSurrogate(c1)) {
+ throw HyracksDataException.create(INVALID_STRING_UNICODE, LOW_SURROGATE_WITHOUT_HIGH_SURROGATE);
+ }
+
+ if (Character.isHighSurrogate(c1)) {
+ // Similar to the above codePointAt(),
+ // if c1 is the a high surrogate and also the last char in the byte array (that means the byte array is somehow illegal),
+ // then an exception will be thrown because there is no low surrogate available in the byte array
+ s += size1;
+ int size2 = charSize(b, s);
+ return size1 + size2;
+ }
+
+ return size1;
+ }
+
+ public static boolean isCharStart(byte[] b, int s) {
+ int c = b[s] & 0xff;
+ return (c >> 6) != 2;
+ }
+
+ public static int getModifiedUTF8Len(char c) {
+ if (c >= 0x0001 && c <= 0x007F) {
+ return 1;
+ } else if (c <= 0x07FF) {
+ return 2;
+ } else {
+ return 3;
+ }
+ }
+
+ public static int writeCharAsModifiedUTF8(char c, DataOutput dos) throws IOException {
+ if (c >= 0x0001 && c <= 0x007F) {
+ dos.writeByte(c);
+ return 1;
+ } else if (c <= 0x07FF) {
+ dos.writeByte((byte) (0xC0 | ((c >> 6) & 0x3F)));
+ dos.writeByte((byte) (0x80 | (c & 0x3F)));
+ return 2;
+ } else {
+ dos.writeByte((byte) (0xE0 | ((c >> 12) & 0x0F)));
+ dos.writeByte((byte) (0x80 | ((c >> 6) & 0x3F)));
+ dos.writeByte((byte) (0x80 | (c & 0x3F)));
+ return 3;
+ }
+ }
+
+ public static int writeCharAsModifiedUTF8(char c, OutputStream dos) throws IOException {
+ if (c >= 0x0001 && c <= 0x007F) {
+ dos.write(c);
+ return 1;
+ } else if (c <= 0x07FF) {
+ dos.write((byte) (0xC0 | ((c >> 6) & 0x3F)));
+ dos.write((byte) (0x80 | (c & 0x3F)));
+ return 2;
+ } else {
+ dos.write((byte) (0xE0 | ((c >> 12) & 0x0F)));
+ dos.write((byte) (0x80 | ((c >> 6) & 0x3F)));
+ dos.write((byte) (0x80 | (c & 0x3F)));
+ return 3;
+ }
+ }
+
+ // The result is the number of Java Chars (8 bytes) in the string
+ public static int getStringLength(byte[] b, int s) {
+ int len = getUTFLength(b, s);
+ int pos = s + getNumBytesToStoreLength(len);
+ return getStringLength(b, pos, len);
+ }
+
+ public static int getStringLength(byte[] b, int offs, int len) {
+ int pos = offs;
+ int end = pos + len;
+ int charCount = 0;
+ while (pos < end) {
+ charCount++;
+ pos += charSize(b, pos);
+ }
+ return charCount;
+ }
+
+ public static int getNumCodePoint(byte[] b, int s) throws HyracksDataException {
+ int len = getUTFLength(b, s);
+ int pos = s + getNumBytesToStoreLength(len);
+ int end = pos + len;
+ int codePointCount = 0;
+ while (pos < end) {
+ codePointCount++;
+ pos += codePointSize(b, pos);
+ }
+
+ return codePointCount;
+ }
+
+ public static int getUTFLength(byte[] b, int s) {
+ return VarLenIntEncoderDecoder.decode(b, s);
+ }
+
+ public static int getNumBytesToStoreLength(int strlen) {
+ return VarLenIntEncoderDecoder.getBytesRequired(strlen);
+ }
+
+ public static int codePointToUTF8(int codePoint, char[] tempChars, byte[] outputUTF8) {
+ int len = 0;
+ int numChars = Character.toChars(codePoint, tempChars, 0);
+ for (int i = 0; i < numChars; i++) {
+ len += writeToBytes(outputUTF8, len, tempChars[i]);
+ }
+
+ return len;
+ }
+
+ /**
+ * Compute the normalized key of the UTF8 string.
+ * The normalized key in Hyracks is mainly used to speedup the comparison between pointable data.
+ * In the UTF8StringPTR case, we compute the integer value by using the first 2 chars.
+ * The comparator will first use this integer to get the result ( <,>, or =), it will check
+ * the actual bytes only if the normalized key is equal. Thus this normalized key must be
+ * consistent with the comparison result.
+ */
+ public static int normalize(byte[] bytes, int start) {
+ int len = getUTFLength(bytes, start);
+ long nk = 0;
+ int offset = start + getNumBytesToStoreLength(len);
+ for (int i = 0; i < 2; ++i) {
+ nk <<= 16;
+ if (i < len) {
+ nk += (charAt(bytes, offset)) & 0xffff;
+ offset += charSize(bytes, offset);
+ }
+ }
+ return (int) (nk >> 1); // make it always positive.
+ }
+
+ public static int compareTo(byte[] thisBytes, int thisStart, byte[] thatBytes, int thatStart) {
+ return compareTo(thisBytes, thisStart, thatBytes, thatStart, false, false);
+ }
+
+ // the start and length of each are the ones calculated by UTF8StringPointable. caller should provide proper values
+ public static int compareTo(byte[] thisBytes, int thisStart, int thisLength, byte[] thatBytes, int thatStart,
+ int thatLength) {
+ return compareTo(thisBytes, thisStart, thisLength, thatBytes, thatStart, thatLength, false, false);
+ }
+
+ /**
+ * This function provides the raw bytes-based comparison for UTF8 strings.
+ * Note that the comparison may not deliver the correct ordering for certain languages that include 2 or 3 bytes characters.
+ * But it works for single-byte character languages.
+ */
+ public static int rawByteCompareTo(byte[] thisBytes, int thisStart, byte[] thatBytes, int thatStart) {
+ return compareTo(thisBytes, thisStart, thatBytes, thatStart, false, true);
+ }
+
+ public static int lowerCaseCompareTo(byte[] thisBytes, int thisStart, byte[] thatBytes, int thatStart) {
+ return compareTo(thisBytes, thisStart, thatBytes, thatStart, true, false);
+ }
+
+ // Certain type of string does not include lengthByte in the beginning and
+ // the length of the given string is given explicitly as a parameter. (e.g., token in a string)
+ public static int lowerCaseCompareTo(byte[] thisBytes, int thisStart, int thisLength, byte[] thatBytes,
+ int thatStart, int thatLength) {
+ return compareTo(thisBytes, thisStart, thisLength, thatBytes, thatStart, thatLength, true, false);
+ }
+
+ public static int hash(byte[] bytes, int start, int coefficient, int r) {
+ return hash(bytes, start, false, false, coefficient, r);
+ }
+
+ public static int hash(byte[] bytes, int start) {
+ return hash(bytes, start, false, false, 31, Integer.MAX_VALUE);
+ }
+
+ private static int hash(byte[] bytes, int start, boolean useLowerCase, boolean useRawByte, int coefficient, int r) {
+ int utflen = getUTFLength(bytes, start);
+ int sStart = start + getNumBytesToStoreLength(utflen);
+ return hash(bytes, sStart, utflen, useLowerCase, useRawByte, coefficient, r);
+ }
+
+ /**
+ * This function provides the raw bytes-based hash function for UTF8 strings.
+ * Note that the hash values may not deliver the correct ordering for certain languages that include 2 or 3 bytes characters.
+ * But it works for single-byte character languages.
+ */
+ public static int rawBytehash(byte[] bytes, int start) {
+ return hash(bytes, start, false, true, 31, Integer.MAX_VALUE);
+ }
+
+ public static int lowerCaseHash(byte[] bytes, int start) {
+ return hash(bytes, start, true, false, 31, Integer.MAX_VALUE);
+ }
+
+ // Certain type of string does not include lengthByte in the beginning and
+ // the length of the given string is given explicitly as a parameter.
+ public static int lowerCaseHash(byte[] bytes, int start, int length) {
+ return hash(bytes, start, length, true, false, 31, Integer.MAX_VALUE);
+ }
+
+ public static String toString(byte[] bytes, int start) {
+ StringBuilder builder = new StringBuilder();
+ return toString(builder, bytes, start).toString();
+ }
+
+ public static StringBuilder toString(StringBuilder builder, byte[] bytes, int start) {
+ int utfLen = getUTFLength(bytes, start);
+ int offset = getNumBytesToStoreLength(utfLen);
+ while (utfLen > 0) {
+ char c = charAt(bytes, start + offset);
+ builder.append(c);
+ int cLen = getModifiedUTF8Len(c);
+ offset += cLen;
+ utfLen -= cLen;
+ }
+ return builder;
+ }
+
+ // Different from the above toString() methods, here we assume the byte[] doesn't contain NumBytesToStoreLength
+ // In fact, this is used for string tokenizer: get "hello" and "world" from the bytes of "hello world"
+ public static String getUTF8StringInArray(byte[] b, int start, int len) {
+ StringBuilder builder = new StringBuilder();
+
+ for (int i = start; i < start + len;) {
+ char c = UTF8StringUtil.charAt(b, i);
+ builder.append(c);
+ i += UTF8StringUtil.charSize(b, i);
+ }
+
+ return builder.toString();
+ }
+
+ public static void printUTF8StringWithQuotes(byte[] b, int s, int l, OutputStream os) throws IOException {
+ printUTF8String(b, s, l, os, true);
+ }
+
+ public static void printUTF8StringNoQuotes(byte[] b, int s, int l, OutputStream os) throws IOException {
+ printUTF8String(b, s, l, os, false);
+ }
+
+ public static void printUTF8StringWithQuotes(String str, OutputStream os) throws IOException {
+ printUTF8String(str, os, true);
+ }
+
+ public static void printUTF8StringNoQuotes(String str, OutputStream os) throws IOException {
+ printUTF8String(str, os, false);
+ }
+
+ public static int encodeUTF8Length(int length, byte[] bytes, int start) {
+ return VarLenIntEncoderDecoder.encode(length, bytes, start);
+ }
+
+ public static int writeUTF8Length(int length, byte[] bytes, DataOutput out) throws IOException {
+ int nbytes = encodeUTF8Length(length, bytes, 0);
+ out.write(bytes, 0, nbytes);
+ return nbytes;
+ }
+
+ private static void printUTF8String(byte[] b, int s, int l, OutputStream os, boolean useQuotes) throws IOException {
+ int stringLength = getUTFLength(b, s);
+ int position = s + getNumBytesToStoreLength(stringLength);
+ int maxPosition = position + stringLength;
+ if (useQuotes) {
+ os.write('\"');
+ }
+ while (position < maxPosition) {
+ char c = charAt(b, position);
+ if (c == '\\' || c == '"') {
+ // escape
+ os.write('\\');
+ }
+ int sz = charSize(b, position);
+ while (sz > 0) {
+ os.write(b[position]);
+ position++;
+ sz--;
+ }
+ }
+ if (useQuotes) {
+ os.write('\"');
+ }
+ }
+
+ private static void printUTF8String(String string, OutputStream os, boolean useQuotes) throws IOException {
+ if (useQuotes) {
+ os.write('\"');
+ }
+ for (int i = 0; i < string.length(); i++) {
+ char ch = string.charAt(i);
+ writeCharAsModifiedUTF8(ch, os);
+ }
+ if (useQuotes) {
+ os.write('\"');
+ }
+ }
+
+ private static int compareTo(byte[] thisBytes, int thisStart, byte[] thatBytes, int thatStart, boolean useLowerCase,
+ boolean useRawByte) {
+ int thisLength = getUTFLength(thisBytes, thisStart);
+ int thatLength = getUTFLength(thatBytes, thatStart);
+ int thisActualStart = thisStart + getNumBytesToStoreLength(thisLength);
+ int thatActualStart = thatStart + getNumBytesToStoreLength(thatLength);
+ return compareTo(thisBytes, thisActualStart, thisLength, thatBytes, thatActualStart, thatLength, useLowerCase,
+ useRawByte);
+ }
+
+ private static int compareTo(byte[] thisBytes, int thisActualStart, int thisLength, byte[] thatBytes,
+ int thatActualStart, int thatLength, boolean useLowerCase, boolean useRawByte) {
+ int c1 = 0;
+ int c2 = 0;
+
+ while (c1 < thisLength && c2 < thatLength) {
+ char ch1, ch2;
+ if (useRawByte) {
+ ch1 = (char) thisBytes[thisActualStart + c1];
+ ch2 = (char) thatBytes[thatActualStart + c2];
+ } else {
+ ch1 = charAt(thisBytes, thisActualStart + c1);
+ ch2 = charAt(thatBytes, thatActualStart + c2);
+
+ if (useLowerCase) {
+ ch1 = Character.toLowerCase(ch1);
+ ch2 = Character.toLowerCase(ch2);
+ }
+ }
+
+ if (ch1 != ch2) {
+ return ch1 - ch2;
+ }
+ c1 += charSize(thisBytes, thisActualStart + c1);
+ c2 += charSize(thatBytes, thatActualStart + c2);
+ }
+ return thisLength - thatLength;
+ }
+
+ private static int hash(byte[] bytes, int start, int length, boolean useLowerCase, boolean useRawByte,
+ int coefficient, int r) {
+ int h = 0;
+ int c = 0;
+
+ while (c < length) {
+ char ch;
+ if (useRawByte) {
+ ch = (char) bytes[start + c];
+ } else {
+ ch = charAt(bytes, start + c);
+ if (useLowerCase) {
+ ch = Character.toLowerCase(ch);
+ }
+ }
+ h = (coefficient * h + ch) % r;
+ c += charSize(bytes, start + c);
+ }
+ return h;
+ }
+
+ public static byte[] writeStringToBytes(String string) {
+ UTF8StringWriter writer = new UTF8StringWriter();
+ ByteArrayOutputStream bos = new ByteArrayOutputStream();
+ DataOutputStream dos = new DataOutputStream(bos);
+ try {
+ writer.writeUTF8(string, dos);
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ return bos.toByteArray();
+ }
+
+ /**
+ * Reads from the
+ * stream <code>in</code> a representation
+ * of a Unicode character string encoded in
+ * <a href="DataInput.html#modified-utf-8">modified UTF-8</a> format;
+ * this string of characters is then returned as a <code>String</code>.
+ * The details of the modified UTF-8 representation
+ * are exactly the same as for the <code>readUTF</code>
+ * method of <code>DataInput</code>.
+ *
+ * @param in
+ * a data input stream.
+ * @return a Unicode string.
+ * @throws EOFException
+ * if the input stream reaches the end
+ * before all the bytes.
+ * @throws IOException
+ * the stream has been closed and the contained
+ * input stream does not support reading after close, or
+ * another I/O error occurs.
+ * @throws UTFDataFormatException
+ * if the bytes do not represent a
+ * valid modified UTF-8 encoding of a Unicode string.
+ * @see java.io.DataInputStream#readUnsignedShort()
+ */
+ public static String readUTF8(DataInput in) throws IOException {
+ return readUTF8(in, null);
+ }
+
+ public static String readUTF8(DataInput in, UTF8StringReader reader) throws IOException {
+ int utflen = VarLenIntEncoderDecoder.decode(in);
+ byte[] bytearr;
+ char[] chararr;
+
+ if (reader == null) {
+ bytearr = new byte[utflen * 2];
+ chararr = new char[utflen * 2];
+ } else {
+ if (reader.bytearr == null || reader.bytearr.length < utflen) {
+ reader.bytearr = new byte[utflen * 2];
+ reader.chararr = new char[utflen * 2];
+ }
+ bytearr = reader.bytearr;
+ chararr = reader.chararr;
+ }
+
+ int c, char2, char3;
+ int count = 0;
+ int chararr_count = 0;
+
+ in.readFully(bytearr, 0, utflen);
+
+ while (count < utflen) {
+ c = bytearr[count] & 0xff;
+ if (c > 127) {
+ break;
+ }
+ count++;
+ chararr[chararr_count++] = (char) c;
+ }
+
+ while (count < utflen) {
+ c = bytearr[count] & 0xff;
+ switch (c >> 4) {
+ case 0:
+ case 1:
+ case 2:
+ case 3:
+ case 4:
+ case 5:
+ case 6:
+ case 7:
+ /* 0xxxxxxx*/
+ count++;
+ chararr[chararr_count++] = (char) c;
+ break;
+ case 12:
+ case 13:
+ /* 110x xxxx 10xx xxxx*/
+ count += 2;
+ if (count > utflen) {
+ throw new UTFDataFormatException("malformed input: partial character at end");
+ }
+ char2 = bytearr[count - 1];
+ if ((char2 & 0xC0) != 0x80) {
+ throw new UTFDataFormatException("malformed input around byte " + count);
+ }
+ chararr[chararr_count++] = (char) (((c & 0x1F) << 6) | (char2 & 0x3F));
+ break;
+ case 14:
+ /* 1110 xxxx 10xx xxxx 10xx xxxx */
+ count += 3;
+ if (count > utflen) {
+ throw new UTFDataFormatException("malformed input: partial character at end");
+ }
+ char2 = bytearr[count - 2];
+ char3 = bytearr[count - 1];
+ if (((char2 & 0xC0) != 0x80) || ((char3 & 0xC0) != 0x80)) {
+ throw new UTFDataFormatException("malformed input around byte " + (count - 1));
+ }
+ chararr[chararr_count++] = (char) (((c & 0x0F) << 12) | ((char2 & 0x3F) << 6) | (char3 & 0x3F));
+ break;
+ default:
+ /* 10xx xxxx, 1111 xxxx */
+ throw new UTFDataFormatException("malformed input around byte " + count);
+ }
+ }
+ // The number of chars produced may be less than utflen
+ return new String(chararr, 0, chararr_count);
+ }
+
+ /**
+ * Write a UTF8 String <code>str</code> into the DataOutput <code>out</code>
+ *
+ * @param str,
+ * a Unicode string;
+ * @param out,
+ * a Data output stream.
+ * @throws IOException
+ */
+ public static void writeUTF8(CharSequence str, DataOutput out) throws IOException {
+ writeUTF8(str, out, null);
+ }
+
+ public static void writeUTF8(CharSequence str, DataOutput out, UTF8StringWriter writer) throws IOException {
+ int strlen = str.length();
+ int utflen = 0;
+ char c;
+ int count = 0;
+
+ for (int i = 0; i < strlen; i++) {
+ // ToDo: we shouldn't use str.charAt(i) to convert raw byte array to UTF-8 chars
+ // one UTF-8 char has at most four bytes, and one Java char we get via str.charAt(i) has 2 bytes
+ // In this case, a UTF-8 char may be consistent of 2 Java chars, and 1 Java char can be converted into 3 UTF-8 bytes
+ // For the emoji, it can be 6 bytes after encoded to UTF-8
+ c = str.charAt(i);
+ utflen += UTF8StringUtil.getModifiedUTF8Len(c);
+ }
+
+ byte[] tempBytes = getTempBytes(writer, utflen);
+ count += VarLenIntEncoderDecoder.encode(utflen, tempBytes, count);
+ int i = 0;
+ for (; i < strlen; i++) {
+ c = str.charAt(i);
+ if (!((c >= 0x0001) && (c <= 0x007F))) {
+ break;
+ }
+ tempBytes[count++] = (byte) c;
+ }
+
+ for (; i < strlen; i++) {
+ c = str.charAt(i);
+ count += writeToBytes(tempBytes, count, c);
+ }
+ out.write(tempBytes, 0, count);
+ }
+
+ public static void writeUTF8(char[] buffer, int start, int length, DataOutput out, UTF8StringWriter writer)
+ throws IOException {
+ int utflen = 0;
+ int count = 0;
+ char c;
+
+ for (int i = 0; i < length; i++) {
+ c = buffer[i + start];
+ utflen += UTF8StringUtil.getModifiedUTF8Len(c);
+ }
+
+ byte[] tempBytes = getTempBytes(writer, utflen);
+ count += VarLenIntEncoderDecoder.encode(utflen, tempBytes, count);
+
+ int i = 0;
+ for (; i < length; i++) {
+ c = buffer[i + start];
+ if (!((c >= 0x0001) && (c <= 0x007F))) {
+ break;
+ }
+ tempBytes[count++] = (byte) c;
+ }
+
+ for (; i < length; i++) {
+ c = buffer[i + start];
+ count += writeToBytes(tempBytes, count, c);
+ }
+ out.write(tempBytes, 0, count);
+ }
+
+ private static int writeToBytes(byte[] tempBytes, int count, char c) {
+ int orig = count;
+ if ((c >= 0x0001) && (c <= 0x007F)) {
+ tempBytes[count++] = (byte) c;
+ } else if (c > 0x07FF) {
+ tempBytes[count++] = (byte) (0xE0 | ((c >> 12) & 0x0F));
+ tempBytes[count++] = (byte) (0x80 | ((c >> 6) & 0x3F));
+ tempBytes[count++] = (byte) (0x80 | (c & 0x3F));
+ } else {
+ tempBytes[count++] = (byte) (0xC0 | ((c >> 6) & 0x1F));
+ tempBytes[count++] = (byte) (0x80 | (c & 0x3F));
+ }
+ return count - orig;
+ }
+
+ private static byte[] getTempBytes(UTF8StringWriter writer, int utflen) {
+ byte[] tempBytes;
+ if (writer == null) {
+ tempBytes = new byte[utflen + 5];
+ } else {
+ byte[] writerTempBytes = writer.tempBytesRef != null ? writer.tempBytesRef.get() : null;
+ if (writerTempBytes == null || writerTempBytes.length < utflen + 5) {
+ writerTempBytes = new byte[utflen + 5];
+ writer.tempBytesRef = new SoftReference<>(writerTempBytes);
+ }
+ tempBytes = writerTempBytes;
+ }
+ return tempBytes;
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/string/UTF8StringWriter.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/util/string/UTF8StringWriter.java
similarity index 100%
rename from hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/string/UTF8StringWriter.java
rename to hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/util/string/UTF8StringWriter.java
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties b/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties
index 4d9c60b..7db5d49 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties
@@ -144,6 +144,8 @@
124 = Parsing error %s: %s
125 = Invalid inverted list type traits: %1$s
126 = Illegal state. %1$s
+127 = Decoding error - %1$s
+
10000 = The given rule collection %1$s is not an instance of the List class.
10001 = Cannot compose partition constraint %1$s with %2$s
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/test/java/org/apache/hyracks/api/string/UTF8StringReaderWriterTest.java b/hyracks-fullstack/hyracks/hyracks-api/src/test/java/org/apache/hyracks/api/string/UTF8StringReaderWriterTest.java
new file mode 100644
index 0000000..abba958
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/test/java/org/apache/hyracks/api/string/UTF8StringReaderWriterTest.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.api.string;
+
+import static org.apache.hyracks.util.string.UTF8StringSample.EMPTY_STRING;
+import static org.apache.hyracks.util.string.UTF8StringSample.STRING_LEN_127;
+import static org.apache.hyracks.util.string.UTF8StringSample.STRING_LEN_128;
+import static org.apache.hyracks.util.string.UTF8StringSample.STRING_LEN_3;
+import static org.apache.hyracks.util.string.UTF8StringSample.STRING_LEN_LARGE;
+import static org.apache.hyracks.util.string.UTF8StringSample.STRING_LEN_LARGE_SUB_1;
+import static org.apache.hyracks.util.string.UTF8StringSample.STRING_LEN_MEDIUM;
+import static org.apache.hyracks.util.string.UTF8StringSample.STRING_LEN_MEDIUM_SUB_1;
+import static org.apache.hyracks.util.string.UTF8StringSample.STRING_UTF8_3;
+import static org.apache.hyracks.util.string.UTF8StringSample.STRING_UTF8_MIX;
+import static org.junit.Assert.assertEquals;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import org.apache.hyracks.util.string.UTF8StringReader;
+import org.apache.hyracks.util.string.UTF8StringWriter;
+import org.junit.Test;
+
+public class UTF8StringReaderWriterTest {
+
+ UTF8StringWriter writer = new UTF8StringWriter();
+ UTF8StringReader reader = new UTF8StringReader();
+
+ @Test
+ public void testWriterReader() throws IOException {
+ writeAndReadOneString(EMPTY_STRING);
+ writeAndReadOneString(STRING_LEN_3);
+
+ writeAndReadOneString(STRING_LEN_127);
+ writeAndReadOneString(STRING_LEN_128);
+ writeAndReadOneString(STRING_LEN_MEDIUM_SUB_1);
+ }
+
+ @Test
+ public void testMedium() throws IOException {
+ writeAndReadOneString(STRING_LEN_MEDIUM);
+ writeAndReadOneString(STRING_LEN_LARGE_SUB_1);
+ }
+
+ @Test
+ public void testLarge() throws IOException {
+ writeAndReadOneString(STRING_LEN_LARGE);
+ }
+
+ @Test
+ public void testUTF8() throws IOException {
+ writeAndReadOneString(STRING_UTF8_3);
+ writeAndReadOneString(STRING_UTF8_MIX);
+ }
+
+ private void writeAndReadOneString(String testString) throws IOException {
+ ByteArrayOutputStream bos = new ByteArrayOutputStream();
+ DataOutputStream dos = new DataOutputStream(bos);
+ writer.writeUTF8(testString, dos);
+
+ ByteArrayInputStream bis = new ByteArrayInputStream(bos.toByteArray(), 0, bos.size());
+ assertEquals(testString, reader.readUTF(new DataInputStream(bis)));
+
+ int lastOffset = bos.size();
+ char[] charArray = testString.toCharArray();
+ writer.writeUTF8(charArray, 0, charArray.length, dos);
+
+ bis = new ByteArrayInputStream(bos.toByteArray(), lastOffset, bos.size());
+ assertEquals(testString, reader.readUTF(new DataInputStream(bis)));
+ }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/test/java/org/apache/hyracks/api/string/UTF8StringUtilTest.java b/hyracks-fullstack/hyracks/hyracks-api/src/test/java/org/apache/hyracks/api/string/UTF8StringUtilTest.java
new file mode 100644
index 0000000..6f3782b
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/test/java/org/apache/hyracks/api/string/UTF8StringUtilTest.java
@@ -0,0 +1,191 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.api.string;
+
+import static org.apache.hyracks.util.string.UTF8StringSample.STRING_LEN_127;
+import static org.apache.hyracks.util.string.UTF8StringSample.STRING_LEN_128;
+import static org.apache.hyracks.util.string.UTF8StringSample.STRING_LEN_MEDIUM;
+import static org.apache.hyracks.util.string.UTF8StringSample.STRING_UTF8_3;
+import static org.apache.hyracks.util.string.UTF8StringSample.STRING_UTF8_MIX;
+import static org.apache.hyracks.util.string.UTF8StringSample.STRING_UTF8_MIX_LOWERCASE;
+import static org.apache.hyracks.util.string.UTF8StringUtil.charAt;
+import static org.apache.hyracks.util.string.UTF8StringUtil.charSize;
+import static org.apache.hyracks.util.string.UTF8StringUtil.compareTo;
+import static org.apache.hyracks.util.string.UTF8StringUtil.getModifiedUTF8Len;
+import static org.apache.hyracks.util.string.UTF8StringUtil.getNumBytesToStoreLength;
+import static org.apache.hyracks.util.string.UTF8StringUtil.getNumCodePoint;
+import static org.apache.hyracks.util.string.UTF8StringUtil.getStringLength;
+import static org.apache.hyracks.util.string.UTF8StringUtil.getUTF8StringInArray;
+import static org.apache.hyracks.util.string.UTF8StringUtil.getUTFLength;
+import static org.apache.hyracks.util.string.UTF8StringUtil.hash;
+import static org.apache.hyracks.util.string.UTF8StringUtil.lowerCaseCompareTo;
+import static org.apache.hyracks.util.string.UTF8StringUtil.lowerCaseHash;
+import static org.apache.hyracks.util.string.UTF8StringUtil.normalize;
+import static org.apache.hyracks.util.string.UTF8StringUtil.rawByteCompareTo;
+import static org.apache.hyracks.util.string.UTF8StringUtil.writeStringToBytes;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.util.string.UTF8StringUtil;
+import org.junit.Test;
+
+public class UTF8StringUtilTest {
+
+ @Test
+ public void testCharAtCharSizeGetLen() throws Exception {
+ char[] utf8Mix = STRING_UTF8_MIX.toCharArray();
+ byte[] buffer = writeStringToBytes(STRING_UTF8_MIX);
+ int pos = getNumBytesToStoreLength(getUTFLength(buffer, 0));
+ for (char c : utf8Mix) {
+ assertEquals(c, charAt(buffer, pos));
+ assertEquals(getModifiedUTF8Len(c), charSize(buffer, pos));
+ pos += charSize(buffer, pos);
+ }
+ }
+
+ @Test
+ public void testGetStringLength() throws Exception {
+ byte[] buffer = writeStringToBytes(STRING_UTF8_MIX);
+ assertEquals(STRING_UTF8_MIX.length(), getStringLength(buffer, 0));
+ }
+
+ @Test
+ public void testChinese() {
+ byte[] bufferDe = writeStringToBytes("的");
+ byte[] bufferLi = writeStringToBytes("离");
+ int ret = compareTo(bufferDe, 0, bufferLi, 0);
+ assertTrue(ret != 0);
+ }
+
+ @Test
+ public void testCompareToAndNormolize() throws Exception {
+ testCompare(STRING_UTF8_MIX, STRING_UTF8_MIX, OPTION.STANDARD);
+ testCompare(STRING_UTF8_3, STRING_UTF8_MIX, OPTION.STANDARD);
+ testCompare(STRING_LEN_MEDIUM, STRING_UTF8_MIX, OPTION.STANDARD);
+ }
+
+ public boolean isSameSign(int r1, int r2) {
+ if (r1 > 0) {
+ return r2 > 0;
+ }
+ if (r1 < 0) {
+ return r2 < 0;
+ }
+ return r2 == 0;
+ }
+
+ enum OPTION {
+ STANDARD,
+ RAW_BYTE,
+ LOWERCASE
+ }
+
+ public void testCompare(String str1, String str2, OPTION option) throws IOException {
+ byte[] buffer1 = writeStringToBytes(str1);
+ byte[] buffer2 = writeStringToBytes(str2);
+
+ switch (option) {
+ case STANDARD:
+ assertEquals(str1.compareTo(str2), compareTo(buffer1, 0, buffer2, 0));
+ int n1 = normalize(buffer1, 0);
+ int n2 = normalize(buffer2, 0);
+ assertTrue(isSameSign(str1.compareTo(str2), n1 - n2));
+ break;
+ case RAW_BYTE:
+ assertEquals(str1.compareTo(str2), rawByteCompareTo(buffer1, 0, buffer2, 0));
+ break;
+ case LOWERCASE:
+ assertEquals(str1.compareToIgnoreCase(str2), lowerCaseCompareTo(buffer1, 0, buffer2, 0));
+ break;
+ }
+
+ }
+
+ @Test
+ public void testRawByteCompareTo() throws Exception {
+ testCompare(STRING_LEN_MEDIUM, STRING_LEN_MEDIUM, OPTION.RAW_BYTE);
+ testCompare(STRING_LEN_127, STRING_LEN_128, OPTION.RAW_BYTE);
+ }
+
+ @Test
+ public void testLowerCaseCompareTo() throws Exception {
+ testCompare(STRING_LEN_127, STRING_LEN_128, OPTION.LOWERCASE);
+ testCompare(STRING_LEN_127, STRING_UTF8_MIX, OPTION.LOWERCASE);
+ testCompare(STRING_UTF8_MIX, STRING_UTF8_MIX_LOWERCASE, OPTION.LOWERCASE);
+ testCompare(STRING_UTF8_MIX_LOWERCASE, STRING_UTF8_MIX, OPTION.LOWERCASE);
+ }
+
+ @Test
+ public void testToString() throws Exception {
+
+ StringBuilder sb = new StringBuilder();
+ byte[] buffer = writeStringToBytes(STRING_UTF8_MIX);
+ assertEquals(STRING_UTF8_MIX, UTF8StringUtil.toString(sb, buffer, 0).toString());
+ }
+
+ @Test
+ public void testHash() throws IOException {
+ byte[] buffer = writeStringToBytes(STRING_UTF8_MIX_LOWERCASE);
+ int lowerHash = hash(buffer, 0);
+
+ buffer = writeStringToBytes(STRING_UTF8_MIX_LOWERCASE);
+ int upperHash = lowerCaseHash(buffer, 0);
+ assertEquals(lowerHash, upperHash);
+
+ int familyOne = hash(buffer, 0, 7, 297);
+ int familyTwo = hash(buffer, 0, 8, 297);
+ assertTrue(familyOne != familyTwo);
+ }
+
+ @Test
+ public void testGetUTF8StringInArray() {
+ String str = null;
+ byte[] bytes = null;
+ List<String> answer = null;
+
+ str = "database group at university of California, Irvine 23333";
+ bytes = writeStringToBytes(str);
+ // First byte in bytes is for the number of bytes of the entire string,
+ // and it should be skipped in getUTF8StringInArray
+ assertEquals("database", getUTF8StringInArray(bytes, 1, 8));
+ assertEquals("at", getUTF8StringInArray(bytes, 16, 2));
+ // test upper case
+ assertEquals("California", getUTF8StringInArray(bytes, 33, 10));
+ // test non-english char
+ assertEquals(",", getUTF8StringInArray(bytes, 43, 1));
+ assertEquals("Irvine", getUTF8StringInArray(bytes, 45, 6));
+ // test number
+ assertEquals("23333", getUTF8StringInArray(bytes, 52, 5));
+ }
+
+ @Test
+ public void testGetNumCodePoint() throws HyracksDataException {
+ String str = "\uD83D\uDC69\u200D\uD83D\uDC69\u200D\uD83D\uDC67\u200D\uD83D\uDC66";
+ assertEquals(getNumCodePoint(writeStringToBytes(str), 0), 7);
+
+ str = "\uD83D\uDC69\u200D\uD83D\uDC69\u200D\uD83D\uDC67\u200D\uD83D\uDC66\uD83C\uDDE8\uD83C\uDDF3";
+ assertEquals(getNumCodePoint(writeStringToBytes(str), 0), 9);
+ }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-client/pom.xml b/hyracks-fullstack/hyracks/hyracks-client/pom.xml
index 409da1f8..9d4ac27 100644
--- a/hyracks-fullstack/hyracks/hyracks-client/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-client/pom.xml
@@ -23,7 +23,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<licenses>
@@ -101,6 +101,11 @@
<version>${project.version}</version>
</dependency>
<dependency>
+ <groupId>org.apache.hyracks</groupId>
+ <artifactId>hyracks-util</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
<groupId>junit</groupId>
<artifactId>junit</artifactId>
<scope>test</scope>
diff --git a/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/result/ResultDirectory.java b/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/result/ResultDirectory.java
index b335a93..5dd5fff 100644
--- a/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/result/ResultDirectory.java
+++ b/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/result/ResultDirectory.java
@@ -18,6 +18,7 @@
*/
package org.apache.hyracks.client.result;
+import java.io.Closeable;
import java.io.IOException;
import java.net.InetSocketAddress;
@@ -35,7 +36,7 @@
import org.apache.hyracks.ipc.impl.JavaSerializationBasedPayloadSerializerDeserializer;
//TODO(madhusudancs): Should this implementation be moved to org.apache.hyracks.client?
-public class ResultDirectory implements IResultDirectory {
+public class ResultDirectory implements IResultDirectory, Closeable {
private final IPCSystem ipc;
private final IResultDirectory remoteResultDirectory;
@@ -64,4 +65,9 @@
public IResultMetadata getResultMetadata(JobId jobId, ResultSetId rsId) throws Exception {
return remoteResultDirectory.getResultMetadata(jobId, rsId);
}
+
+ @Override
+ public void close() throws IOException {
+ ipc.stop();
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/result/ResultSet.java b/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/result/ResultSet.java
index 4d8767f..8a88045 100644
--- a/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/result/ResultSet.java
+++ b/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/result/ResultSet.java
@@ -18,6 +18,9 @@
*/
package org.apache.hyracks.client.result;
+import java.io.Closeable;
+import java.io.IOException;
+
import org.apache.hyracks.api.client.IHyracksClientConnection;
import org.apache.hyracks.api.comm.NetworkAddress;
import org.apache.hyracks.api.context.IHyracksCommonContext;
@@ -25,15 +28,15 @@
import org.apache.hyracks.api.io.IIOManager;
import org.apache.hyracks.api.job.JobId;
import org.apache.hyracks.api.network.ISocketChannelFactory;
-import org.apache.hyracks.api.result.IResultDirectory;
import org.apache.hyracks.api.result.IResultSet;
import org.apache.hyracks.api.result.IResultSetReader;
import org.apache.hyracks.api.result.ResultSetId;
import org.apache.hyracks.client.net.ClientNetworkManager;
import org.apache.hyracks.control.nc.resources.memory.FrameManager;
+import org.apache.hyracks.util.NetworkUtil;
-public class ResultSet implements IResultSet {
- private final IResultDirectory resultDirectory;
+public class ResultSet implements IResultSet, Closeable {
+ private final ResultDirectory resultDirectory;
private final ClientNetworkManager netManager;
@@ -51,6 +54,15 @@
}
@Override
+ public void close() throws IOException {
+ try {
+ netManager.stop();
+ } finally {
+ NetworkUtil.closeQuietly(resultDirectory);
+ }
+ }
+
+ @Override
public IResultSetReader createReader(JobId jobId, ResultSetId resultSetId) throws HyracksDataException {
IResultSetReader reader = null;
try {
diff --git a/hyracks-fullstack/hyracks/hyracks-comm/pom.xml b/hyracks-fullstack/hyracks/hyracks-comm/pom.xml
index f73060f..c833638 100644
--- a/hyracks-fullstack/hyracks/hyracks-comm/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-comm/pom.xml
@@ -24,7 +24,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/hyracks-fullstack/hyracks/hyracks-comm/src/main/java/org/apache/hyracks/comm/channels/FileNetworkInputChannel.java b/hyracks-fullstack/hyracks/hyracks-comm/src/main/java/org/apache/hyracks/comm/channels/FileNetworkInputChannel.java
new file mode 100644
index 0000000..542fda1
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-comm/src/main/java/org/apache/hyracks/comm/channels/FileNetworkInputChannel.java
@@ -0,0 +1,144 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.comm.channels;
+
+import java.net.SocketAddress;
+import java.nio.ByteBuffer;
+import java.util.ArrayDeque;
+import java.util.Queue;
+
+import org.apache.hyracks.api.channels.IInputChannel;
+import org.apache.hyracks.api.channels.IInputChannelMonitor;
+import org.apache.hyracks.api.comm.IBufferAcceptor;
+import org.apache.hyracks.api.comm.IChannelControlBlock;
+import org.apache.hyracks.api.comm.ICloseableBufferAcceptor;
+import org.apache.hyracks.api.context.IHyracksCommonContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class FileNetworkInputChannel implements IInputChannel {
+
+ private static final int NUM_READ_BUFFERS = 1;
+ public static final long FILE_CHANNEL_CODE = -1;
+
+ private final IChannelConnectionFactory netManager;
+ private final SocketAddress remoteAddress;
+ private final long jobId;
+ private final long fileId;
+ private final Queue<ByteBuffer> fullQueue;
+ private final int nBuffers;
+ private IChannelControlBlock ccb;
+ private IInputChannelMonitor monitor;
+ private Object attachment;
+
+ public FileNetworkInputChannel(IChannelConnectionFactory netManager, SocketAddress remoteAddress, long jobId,
+ long fileId) {
+ this.netManager = netManager;
+ this.remoteAddress = remoteAddress;
+ this.jobId = jobId;
+ this.fileId = fileId;
+ this.fullQueue = new ArrayDeque<>(NUM_READ_BUFFERS);
+ this.nBuffers = NUM_READ_BUFFERS;
+ }
+
+ @Override
+ public void registerMonitor(IInputChannelMonitor monitor) {
+ this.monitor = monitor;
+ }
+
+ @Override
+ public void setAttachment(Object attachment) {
+ this.attachment = attachment;
+ }
+
+ @Override
+ public Object getAttachment() {
+ return attachment;
+ }
+
+ @Override
+ public synchronized ByteBuffer getNextBuffer() {
+ return fullQueue.poll();
+ }
+
+ @Override
+ public void recycleBuffer(ByteBuffer buffer) {
+ buffer.clear();
+ ccb.getReadInterface().getEmptyBufferAcceptor().accept(buffer);
+ }
+
+ @Override
+ public void open(IHyracksCommonContext ctx) throws HyracksDataException {
+ try {
+ ccb = netManager.connect(remoteAddress);
+ } catch (Exception e) {
+ throw HyracksDataException.create(e);
+ }
+ ccb.getReadInterface().setFullBufferAcceptor(new ReadFullBufferAcceptor());
+ ccb.getWriteInterface().setEmptyBufferAcceptor(WriteEmptyBufferAcceptor.INSTANCE);
+ ccb.getReadInterface().setBufferFactory(new ReadBufferFactory(nBuffers, ctx), nBuffers,
+ ctx.getInitialFrameSize());
+
+ ByteBuffer writeBuffer = ByteBuffer.allocate(NetworkInputChannel.INITIAL_MESSAGE_SIZE);
+ writeBuffer.putLong(FILE_CHANNEL_CODE);
+ writeBuffer.putLong(jobId);
+ writeBuffer.putLong(fileId);
+ writeBuffer.flip();
+
+ ccb.getWriteInterface().getFullBufferAcceptor().accept(writeBuffer);
+ ccb.getWriteInterface().getFullBufferAcceptor().close();
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+
+ }
+
+ @Override
+ public void fail() {
+ // do nothing (covered by job lifecycle)
+ }
+
+ private class ReadFullBufferAcceptor implements ICloseableBufferAcceptor {
+ @Override
+ public void accept(ByteBuffer buffer) {
+ fullQueue.add(buffer);
+ monitor.notifyDataAvailability(FileNetworkInputChannel.this, 1);
+ }
+
+ @Override
+ public void close() {
+ monitor.notifyEndOfStream(FileNetworkInputChannel.this);
+ }
+
+ @Override
+ public void error(int ecode) {
+ monitor.notifyFailure(FileNetworkInputChannel.this, ecode);
+ }
+ }
+
+ private static class WriteEmptyBufferAcceptor implements IBufferAcceptor {
+
+ static final WriteEmptyBufferAcceptor INSTANCE = new WriteEmptyBufferAcceptor();
+
+ @Override
+ public void accept(ByteBuffer buffer) {
+ // do nothing
+ }
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-comm/src/main/java/org/apache/hyracks/comm/channels/NetworkInputChannel.java b/hyracks-fullstack/hyracks/hyracks-comm/src/main/java/org/apache/hyracks/comm/channels/NetworkInputChannel.java
index 53bb7cd..18cab49 100644
--- a/hyracks-fullstack/hyracks/hyracks-comm/src/main/java/org/apache/hyracks/comm/channels/NetworkInputChannel.java
+++ b/hyracks-fullstack/hyracks/hyracks-comm/src/main/java/org/apache/hyracks/comm/channels/NetworkInputChannel.java
@@ -37,7 +37,8 @@
public class NetworkInputChannel implements IInputChannel {
private static final Logger LOGGER = LogManager.getLogger();
- static final int INITIAL_MESSAGE_SIZE = 20;
+ private static final int INITIAL_MSG_FILLER = -1;
+ public static final int INITIAL_MESSAGE_SIZE = 24;
private final IChannelConnectionFactory netManager;
@@ -106,6 +107,7 @@
writeBuffer.putInt(partitionId.getConnectorDescriptorId().getId());
writeBuffer.putInt(partitionId.getSenderIndex());
writeBuffer.putInt(partitionId.getReceiverIndex());
+ writeBuffer.putInt(INITIAL_MSG_FILLER);
writeBuffer.flip();
if (LOGGER.isTraceEnabled()) {
LOGGER.trace("Sending partition request: " + partitionId + " on channel: " + ccb);
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 57e22e7..71f7b44 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/pom.xml
@@ -24,7 +24,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks-control</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<properties>
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClientInterfaceIPCI.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClientInterfaceIPCI.java
index e6973dd..6da8fdf 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClientInterfaceIPCI.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClientInterfaceIPCI.java
@@ -147,8 +147,8 @@
case WAIT_FOR_COMPLETION:
HyracksClientInterfaceFunctions.WaitForCompletionFunction wfcf =
(HyracksClientInterfaceFunctions.WaitForCompletionFunction) fn;
- ccs.getWorkQueue()
- .schedule(new WaitForJobCompletionWork(ccs, wfcf.getJobId(), new IPCResponder<>(handle, mid)));
+ ccs.getWorkQueue().schedule(new WaitForJobCompletionWork(ccs, wfcf.getJobId(),
+ wfcf.getStatOperatorNames(), new IPCResponder<>(handle, mid)));
break;
case GET_NODE_CONTROLLERS_INFO:
ccs.getWorkQueue().schedule(
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerService.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerService.java
index f11e7ff..c9ac4ef 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerService.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerService.java
@@ -34,6 +34,7 @@
import java.util.TimerTask;
import java.util.TreeMap;
import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
import org.apache.hyracks.api.application.ICCApplication;
import org.apache.hyracks.api.client.ClusterControllerInfo;
@@ -78,6 +79,7 @@
import org.apache.hyracks.control.common.ipc.CCNCFunctions;
import org.apache.hyracks.control.common.logs.LogFile;
import org.apache.hyracks.control.common.shutdown.ShutdownRun;
+import org.apache.hyracks.control.common.utils.HyracksThreadFactory;
import org.apache.hyracks.control.common.work.WorkQueue;
import org.apache.hyracks.ipc.api.IIPCI;
import org.apache.hyracks.ipc.impl.IPCSystem;
@@ -118,7 +120,7 @@
private final WorkQueue workQueue;
- private ExecutorService executor;
+ private ExecutorService executor = Executors.newCachedThreadPool(new HyracksThreadFactory("<bootstrap>"));
private final Timer timer;
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/application/CCServiceContext.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/application/CCServiceContext.java
index de166dd..555d37e 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/application/CCServiceContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/application/CCServiceContext.java
@@ -37,6 +37,7 @@
import org.apache.hyracks.api.job.JobId;
import org.apache.hyracks.api.job.JobSpecification;
import org.apache.hyracks.api.job.JobStatus;
+import org.apache.hyracks.api.job.resource.IJobCapacityController;
import org.apache.hyracks.api.service.IControllerService;
import org.apache.hyracks.control.cc.ClusterControllerService;
import org.apache.hyracks.control.common.application.ServiceContext;
@@ -79,22 +80,23 @@
jobLifecycleListeners.add(jobLifecycleListener);
}
- public synchronized void notifyJobStart(JobId jobId) throws HyracksException {
+ public synchronized void notifyJobStart(JobId jobId, JobSpecification spec) throws HyracksException {
for (IJobLifecycleListener l : jobLifecycleListeners) {
- l.notifyJobStart(jobId);
+ l.notifyJobStart(jobId, spec);
}
}
- public synchronized void notifyJobFinish(JobId jobId, JobStatus jobStatus, List<Exception> exceptions)
- throws HyracksException {
+ public synchronized void notifyJobFinish(JobId jobId, JobSpecification spec, JobStatus jobStatus,
+ List<Exception> exceptions) throws HyracksException {
for (IJobLifecycleListener l : jobLifecycleListeners) {
- l.notifyJobFinish(jobId, jobStatus, exceptions);
+ l.notifyJobFinish(jobId, spec, jobStatus, exceptions);
}
}
- public synchronized void notifyJobCreation(JobId jobId, JobSpecification spec) throws HyracksException {
+ public synchronized void notifyJobCreation(JobId jobId, JobSpecification spec,
+ IJobCapacityController.JobSubmissionStatus status) throws HyracksException {
for (IJobLifecycleListener l : jobLifecycleListeners) {
- l.notifyJobCreation(jobId, spec);
+ l.notifyJobCreation(jobId, spec, status);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/executor/JobExecutor.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/executor/JobExecutor.java
index 3574acd..567d20c 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/executor/JobExecutor.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/executor/JobExecutor.java
@@ -115,7 +115,7 @@
public void startJob() throws HyracksException {
startRunnableActivityClusters();
- ccs.getContext().notifyJobStart(jobRun.getJobId());
+ ccs.getContext().notifyJobStart(jobRun.getJobId(), jobRun.getJobSpecification());
}
public void cancelJob(IResultCallback<Void> callback) throws HyracksException {
@@ -539,7 +539,7 @@
private void abortTaskCluster(TaskClusterAttempt tcAttempt,
TaskClusterAttempt.TaskClusterStatus failedOrAbortedStatus) {
- LOGGER.trace(() -> "Aborting task cluster: " + tcAttempt.getAttempt());
+ LOGGER.trace("Aborting task cluster: {}", tcAttempt.getAttempt());
Set<TaskAttemptId> abortTaskIds = new HashSet<>();
Map<String, List<TaskAttemptId>> abortTaskAttemptMap = new HashMap<>();
for (TaskAttempt ta : tcAttempt.getTaskAttempts().values()) {
@@ -561,14 +561,12 @@
}
}
final JobId jobId = jobRun.getJobId();
- LOGGER.trace(() -> "Abort map for job: " + jobId + ": " + abortTaskAttemptMap);
+ LOGGER.trace("Abort map for job: {}: {}", jobId, abortTaskAttemptMap);
INodeManager nodeManager = ccs.getNodeManager();
abortTaskAttemptMap.forEach((key, abortTaskAttempts) -> {
final NodeControllerState node = nodeManager.getNodeControllerState(key);
if (node != null) {
- if (LOGGER.isTraceEnabled()) {
- LOGGER.trace("Aborting: " + abortTaskAttempts + " at " + key);
- }
+ LOGGER.trace("Aborting: {} at {}", abortTaskAttempts, key);
try {
node.getNodeController().abortTasks(jobId, abortTaskAttempts);
} catch (Exception e) {
@@ -579,8 +577,8 @@
inProgressTaskClusters.remove(tcAttempt.getTaskCluster());
TaskCluster tc = tcAttempt.getTaskCluster();
PartitionMatchMaker pmm = jobRun.getPartitionMatchMaker();
- pmm.removeUncommittedPartitions(tc.getProducedPartitions(), abortTaskIds);
- pmm.removePartitionRequests(tc.getRequiredPartitions(), abortTaskIds);
+ pmm.removeUncommittedPartitions(tc.getProducedPartitions(), abortTaskIds, jobId);
+ pmm.removePartitionRequests(tc.getRequiredPartitions(), abortTaskIds, jobId);
tcAttempt.setStatus(failedOrAbortedStatus);
tcAttempt.setEndTime(System.currentTimeMillis());
@@ -656,7 +654,11 @@
}
TaskAttempt.TaskStatus taStatus = ta.getStatus();
if (taStatus != TaskAttempt.TaskStatus.RUNNING) {
- LOGGER.warn(() -> "Spurious task complete notification: " + taId + " Current state = " + taStatus);
+ // don't log if aborted/failed because a task could complete just before the job was aborted/failed
+ if (taStatus != TaskAttempt.TaskStatus.ABORTED && taStatus != TaskAttempt.TaskStatus.FAILED) {
+ LOGGER.warn("spurious task complete notification {}:{}. current state {}", jobRun.getJobId(), taId,
+ taStatus);
+ }
return;
}
ta.setStatus(TaskAttempt.TaskStatus.COMPLETED, null);
@@ -683,7 +685,6 @@
*/
public void notifyTaskFailure(TaskAttempt ta, List<Exception> exceptions) {
try {
- LOGGER.debug("Received failure notification for TaskAttempt " + ta.getTaskAttemptId());
TaskAttemptId taId = ta.getTaskAttemptId();
TaskCluster tc = ta.getTask().getTaskCluster();
TaskClusterAttempt lastAttempt = findLastTaskClusterAttempt(tc);
@@ -696,7 +697,7 @@
LOGGER.trace(() -> "Marking TaskAttempt " + ta.getTaskAttemptId()
+ " as failed and the number of max re-attempts = " + maxReattempts);
if (lastAttempt.getAttempt() >= maxReattempts || isCancelled()) {
- LOGGER.debug(() -> "Aborting the job of " + ta.getTaskAttemptId());
+ LOGGER.debug("Aborting the job:{} of {}", jobRun.getJobId(), ta.getTaskAttemptId());
abortJob(exceptions, NoOpCallback.INSTANCE);
return;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/IJobManager.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/IJobManager.java
index a9ddee3..b2795d4 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/IJobManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/IJobManager.java
@@ -112,6 +112,11 @@
Collection<JobRun> getRunningJobs();
/**
+ * @return number of jobs that are currently running.
+ */
+ int getRunningJobsCount();
+
+ /**
* @return all jobs that are currently waiting in the job queue.
*/
Collection<JobRun> getPendingJobs();
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/JobManager.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/JobManager.java
index 9850ee2..d803c88 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/JobManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/JobManager.java
@@ -38,7 +38,9 @@
import org.apache.hyracks.api.job.JobId;
import org.apache.hyracks.api.job.JobSpecification;
import org.apache.hyracks.api.job.JobStatus;
+import org.apache.hyracks.api.job.resource.IClusterCapacity;
import org.apache.hyracks.api.job.resource.IJobCapacityController;
+import org.apache.hyracks.api.job.resource.IReadOnlyClusterCapacity;
import org.apache.hyracks.api.util.ExceptionUtils;
import org.apache.hyracks.control.cc.ClusterControllerService;
import org.apache.hyracks.control.cc.NodeControllerState;
@@ -110,7 +112,7 @@
JobSpecification job = jobRun.getJobSpecification();
IJobCapacityController.JobSubmissionStatus status = jobCapacityController.allocate(job);
CCServiceContext serviceCtx = ccs.getContext();
- serviceCtx.notifyJobCreation(jobRun.getJobId(), job);
+ serviceCtx.notifyJobCreation(jobRun.getJobId(), job, status);
switch (status) {
case QUEUE:
queueJob(jobRun);
@@ -129,7 +131,7 @@
if (activeRunMap.containsKey(jobId)) {
JobRun jobRun = activeRunMap.get(jobId);
// The following call will abort all ongoing tasks and then consequently
- // trigger JobCleanupWork and JobCleanupNotificationWork which will update the lifecyle of the job.
+ // trigger JobCleanupWork and JobCleanupNotificationWork which will update the lifecycle of the job.
// Therefore, we do not remove the job out of activeRunMap here.
jobRun.getExecutor().cancelJob(callback);
return;
@@ -139,14 +141,15 @@
if (jobRun != null) {
List<Exception> exceptions =
Collections.singletonList(HyracksException.create(ErrorCode.JOB_CANCELED, jobId));
- // Since the job has not been executed, we only need to update its status and lifecyle here.
+ // Since the job has not been executed, we only need to update its status and lifecycle here.
jobRun.setStatus(JobStatus.FAILURE_BEFORE_EXECUTION, exceptions);
runMapArchive.put(jobId, jobRun);
runMapHistory.put(jobId, exceptions);
CCServiceContext serviceCtx = ccs.getContext();
if (serviceCtx != null) {
try {
- serviceCtx.notifyJobFinish(jobId, JobStatus.FAILURE_BEFORE_EXECUTION, exceptions);
+ serviceCtx.notifyJobFinish(jobId, jobRun.getJobSpecification(), JobStatus.FAILURE_BEFORE_EXECUTION,
+ exceptions);
} catch (Exception e) {
LOGGER.error("Exception notifying cancel on pending job {}", jobId, e);
throw HyracksDataException.create(e);
@@ -170,7 +173,6 @@
return;
}
if (run.getPendingStatus() != null) {
- LOGGER.warn("Ignoring duplicate cleanup for JobRun with id: {}", run::getJobId);
return;
}
Set<String> targetNodes = run.getParticipatingNodeIds();
@@ -223,7 +225,8 @@
Throwable caughtException = null;
CCServiceContext serviceCtx = ccs.getContext();
try {
- serviceCtx.notifyJobFinish(jobId, run.getPendingStatus(), run.getPendingExceptions());
+ serviceCtx.notifyJobFinish(jobId, run.getJobSpecification(), run.getPendingStatus(),
+ run.getPendingExceptions());
} catch (Exception e) {
LOGGER.error("Exception notifying job finish {}", jobId, e);
caughtException = e;
@@ -264,6 +267,11 @@
}
@Override
+ public int getRunningJobsCount() {
+ return activeRunMap.size();
+ }
+
+ @Override
public Collection<JobRun> getPendingJobs() {
return jobQueue.jobs();
}
@@ -308,6 +316,7 @@
run.setStartTime(System.currentTimeMillis());
run.setStartTimeZoneId(ZoneId.systemDefault().getId());
JobId jobId = run.getJobId();
+ logJobCapacity(run, "running", Level.DEBUG);
activeRunMap.put(jobId, run);
run.setStatus(JobStatus.RUNNING, null);
executeJobInternal(run);
@@ -315,6 +324,7 @@
// Queue a job when the required capacity for the job is not met.
private void queueJob(JobRun jobRun) throws HyracksException {
+ logJobCapacity(jobRun, "queueing", Level.INFO);
jobRun.setStatus(JobStatus.PENDING, null);
jobQueue.add(jobRun);
}
@@ -350,5 +360,23 @@
private void releaseJobCapacity(JobRun jobRun) {
final JobSpecification job = jobRun.getJobSpecification();
jobCapacityController.release(job);
+ logJobCapacity(jobRun, "released", Level.DEBUG);
+ }
+
+ private void logJobCapacity(JobRun jobRun, String jobStateDesc, Level lvl) {
+ IClusterCapacity requiredResources = jobRun.getJobSpecification().getRequiredClusterCapacity();
+ if (requiredResources == null) {
+ return;
+ }
+ long requiredMemory = requiredResources.getAggregatedMemoryByteSize();
+ int requiredCPUs = requiredResources.getAggregatedCores();
+ if (requiredMemory == 0 && requiredCPUs == 0) {
+ return;
+ }
+ IReadOnlyClusterCapacity clusterCapacity = jobCapacityController.getClusterCapacity();
+ LOGGER.log(lvl, "{} {}, memory={}, cpu={}, (new) cluster memory={}, cpu={}, currently running={}, queued={}",
+ jobStateDesc, jobRun.getJobId(), requiredMemory, requiredCPUs,
+ clusterCapacity.getAggregatedMemoryByteSize(), clusterCapacity.getAggregatedCores(),
+ getRunningJobsCount(), jobQueue.size());
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/JobRun.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/JobRun.java
index 0cc09b4..06e955a 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/JobRun.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/JobRun.java
@@ -82,13 +82,13 @@
private final Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicyMap;
- private long createTime;
+ private final long createTime;
- private long startTime;
+ private volatile long startTime;
private String startTimeZoneId;
- private long endTime;
+ private volatile long endTime;
private JobStatus status;
@@ -98,7 +98,7 @@
private List<Exception> pendingExceptions;
- private Map<OperatorDescriptorId, Map<Integer, String>> operatorLocations;
+ private final Map<OperatorDescriptorId, Map<Integer, String>> operatorLocations;
private JobRun(DeploymentId deploymentId, JobId jobId, Set<JobFlag> jobFlags, JobSpecification spec,
ActivityClusterGraph acg) {
@@ -218,6 +218,10 @@
this.endTime = endTime;
}
+ public long getQueueWaitTimeInMillis() {
+ return startTime > 0 ? startTime - createTime : System.currentTimeMillis() - createTime;
+ }
+
public void registerOperatorLocation(OperatorDescriptorId op, int partition, String location) {
operatorLocations.computeIfAbsent(op, k -> new HashMap<>()).put(partition, location);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/partitions/PartitionMatchMaker.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/partitions/PartitionMatchMaker.java
index ac29b53..6278693 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/partitions/PartitionMatchMaker.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/partitions/PartitionMatchMaker.java
@@ -28,6 +28,7 @@
import org.apache.commons.lang3.tuple.Pair;
import org.apache.hyracks.api.dataflow.TaskAttemptId;
+import org.apache.hyracks.api.job.JobId;
import org.apache.hyracks.api.partitions.PartitionId;
import org.apache.hyracks.control.common.job.PartitionDescriptor;
import org.apache.hyracks.control.common.job.PartitionRequest;
@@ -43,14 +44,13 @@
private final Map<PartitionId, List<PartitionRequest>> partitionRequests;
public PartitionMatchMaker() {
- partitionDescriptors = new HashMap<PartitionId, List<PartitionDescriptor>>();
- partitionRequests = new HashMap<PartitionId, List<PartitionRequest>>();
+ partitionDescriptors = new HashMap<>();
+ partitionRequests = new HashMap<>();
}
public List<Pair<PartitionDescriptor, PartitionRequest>> registerPartitionDescriptor(
PartitionDescriptor partitionDescriptor) {
- List<Pair<PartitionDescriptor, PartitionRequest>> matches =
- new ArrayList<Pair<PartitionDescriptor, PartitionRequest>>();
+ List<Pair<PartitionDescriptor, PartitionRequest>> matches = new ArrayList<>();
PartitionId pid = partitionDescriptor.getPartitionId();
boolean matched = false;
List<PartitionRequest> requests = partitionRequests.get(pid);
@@ -73,11 +73,7 @@
}
if (!matched) {
- List<PartitionDescriptor> descriptors = partitionDescriptors.get(pid);
- if (descriptors == null) {
- descriptors = new ArrayList<PartitionDescriptor>();
- partitionDescriptors.put(pid, descriptors);
- }
+ List<PartitionDescriptor> descriptors = partitionDescriptors.computeIfAbsent(pid, k -> new ArrayList<>());
descriptors.add(partitionDescriptor);
}
@@ -108,11 +104,7 @@
}
if (match == null) {
- List<PartitionRequest> requests = partitionRequests.get(pid);
- if (requests == null) {
- requests = new ArrayList<PartitionRequest>();
- partitionRequests.put(pid, requests);
- }
+ List<PartitionRequest> requests = partitionRequests.computeIfAbsent(pid, k -> new ArrayList<>());
requests.add(partitionRequest);
}
@@ -133,17 +125,11 @@
}
private interface IEntryFilter<T> {
- public boolean matches(T o);
+ boolean matches(T o);
}
private static <T> void removeEntries(List<T> list, IEntryFilter<T> filter) {
- Iterator<T> j = list.iterator();
- while (j.hasNext()) {
- T o = j.next();
- if (filter.matches(o)) {
- j.remove();
- }
- }
+ list.removeIf(filter::matches);
}
private static <T> void removeEntries(Map<PartitionId, List<T>> map, IEntryFilter<T> filter) {
@@ -159,30 +145,16 @@
}
public void notifyNodeFailures(final Collection<String> deadNodes) {
- removeEntries(partitionDescriptors, new IEntryFilter<PartitionDescriptor>() {
- @Override
- public boolean matches(PartitionDescriptor o) {
- return deadNodes.contains(o.getNodeId());
- }
- });
- removeEntries(partitionRequests, new IEntryFilter<PartitionRequest>() {
- @Override
- public boolean matches(PartitionRequest o) {
- return deadNodes.contains(o.getNodeId());
- }
- });
+ removeEntries(partitionDescriptors, o -> deadNodes.contains(o.getNodeId()));
+ removeEntries(partitionRequests, o -> deadNodes.contains(o.getNodeId()));
}
- public void removeUncommittedPartitions(Set<PartitionId> partitionIds, final Set<TaskAttemptId> taIds) {
- if (LOGGER.isDebugEnabled()) {
- LOGGER.debug("Removing uncommitted partitions: " + partitionIds);
+ public void removeUncommittedPartitions(Set<PartitionId> partitionIds, Set<TaskAttemptId> taIds, JobId jobId) {
+ if (partitionIds != null && !partitionIds.isEmpty()) {
+ LOGGER.debug("Removing uncommitted partitions {}: {}", jobId, partitionIds);
}
- IEntryFilter<PartitionDescriptor> filter = new IEntryFilter<PartitionDescriptor>() {
- @Override
- public boolean matches(PartitionDescriptor o) {
- return o.getState() != PartitionState.COMMITTED && taIds.contains(o.getProducingTaskAttemptId());
- }
- };
+ IEntryFilter<PartitionDescriptor> filter =
+ o -> o.getState() != PartitionState.COMMITTED && taIds.contains(o.getProducingTaskAttemptId());
for (PartitionId pid : partitionIds) {
List<PartitionDescriptor> descriptors = partitionDescriptors.get(pid);
if (descriptors != null) {
@@ -194,16 +166,11 @@
}
}
- public void removePartitionRequests(Set<PartitionId> partitionIds, final Set<TaskAttemptId> taIds) {
- if (LOGGER.isDebugEnabled()) {
- LOGGER.debug("Removing partition requests: " + partitionIds);
+ public void removePartitionRequests(Set<PartitionId> partitionIds, Set<TaskAttemptId> taIds, JobId jobId) {
+ if (partitionIds != null && !partitionIds.isEmpty()) {
+ LOGGER.debug("Removing partition requests {}: {}", jobId, partitionIds);
}
- IEntryFilter<PartitionRequest> filter = new IEntryFilter<PartitionRequest>() {
- @Override
- public boolean matches(PartitionRequest o) {
- return taIds.contains(o.getRequestingTaskAttemptId());
- }
- };
+ IEntryFilter<PartitionRequest> filter = o -> taIds.contains(o.getRequestingTaskAttemptId());
for (PartitionId pid : partitionIds) {
List<PartitionRequest> requests = partitionRequests.get(pid);
if (requests != null) {
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/result/ResultDirectoryService.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/result/ResultDirectoryService.java
index 9f8a7e2..b6274d9 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/result/ResultDirectoryService.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/result/ResultDirectoryService.java
@@ -34,6 +34,7 @@
import org.apache.hyracks.api.job.JobId;
import org.apache.hyracks.api.job.JobSpecification;
import org.apache.hyracks.api.job.JobStatus;
+import org.apache.hyracks.api.job.resource.IJobCapacityController;
import org.apache.hyracks.api.result.IJobResultCallback;
import org.apache.hyracks.api.result.IResultMetadata;
import org.apache.hyracks.api.result.IResultStateRecord;
@@ -45,7 +46,6 @@
import org.apache.hyracks.control.common.result.AbstractResultManager;
import org.apache.hyracks.control.common.result.ResultStateSweeper;
import org.apache.hyracks.control.common.work.IResultCallback;
-import org.apache.logging.log4j.Level;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
@@ -78,10 +78,8 @@
}
@Override
- public synchronized void notifyJobCreation(JobId jobId, JobSpecification spec) throws HyracksException {
- if (LOGGER.isDebugEnabled()) {
- LOGGER.debug(getClass().getSimpleName() + " notified of new job " + jobId);
- }
+ public synchronized void notifyJobCreation(JobId jobId, JobSpecification spec,
+ IJobCapacityController.JobSubmissionStatus status) throws HyracksException {
if (jobResultLocations.get(jobId) != null) {
throw HyracksDataException.create(ErrorCode.MORE_THAN_ONE_RESULT, jobId);
}
@@ -89,12 +87,13 @@
}
@Override
- public synchronized void notifyJobStart(JobId jobId) throws HyracksException {
+ public synchronized void notifyJobStart(JobId jobId, JobSpecification spec) throws HyracksException {
jobResultLocations.get(jobId).getRecord().start();
}
@Override
- public void notifyJobFinish(JobId jobId, JobStatus jobStatus, List<Exception> exceptions) throws HyracksException {
+ public void notifyJobFinish(JobId jobId, JobSpecification spec, JobStatus jobStatus, List<Exception> exceptions)
+ throws HyracksException {
if (exceptions == null || exceptions.isEmpty()) {
final ResultJobRecord resultJobRecord = getResultJobRecord(jobId);
if (resultJobRecord == null) {
@@ -157,15 +156,14 @@
@Override
public synchronized void reportJobFailure(JobId jobId, List<Exception> exceptions) {
- Exception ex = exceptions.isEmpty() ? null : exceptions.get(0);
- Level logLevel = Level.DEBUG;
- if (LOGGER.isEnabled(logLevel)) {
- LOGGER.log(logLevel, "job " + jobId + " failed and is being reported to " + getClass().getSimpleName(), ex);
- }
ResultJobRecord rjr = getResultJobRecord(jobId);
+ if (logFailure(rjr)) {
+ LOGGER.debug("job {} failed and is being reported to {}", jobId, getClass().getSimpleName());
+ }
if (rjr != null) {
rjr.fail(exceptions);
}
+ Exception ex = exceptions.isEmpty() ? null : exceptions.get(0);
final JobResultInfo jobResultInfo = jobResultLocations.get(jobId);
if (jobResultInfo != null) {
jobResultInfo.setException(ex);
@@ -211,6 +209,15 @@
}
}
+ private static boolean logFailure(ResultJobRecord rjr) {
+ if (rjr == null) {
+ return true;
+ }
+ // don't re-log if the state is already failed
+ ResultJobRecord.Status status = rjr.getStatus();
+ return status == null || status.getState() != State.FAILED;
+ }
+
/**
* Compares the records already known by the client for the given job's result set id with the records that the
* result directory service knows and if there are any newly discovered records returns a whole array with the
@@ -264,7 +271,7 @@
class JobResultInfo {
- private ResultJobRecord record;
+ private final ResultJobRecord record;
private Waiters waiters;
private Exception exception;
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/scheduler/FIFOJobQueue.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/scheduler/FIFOJobQueue.java
index 260c6b9..38277c2 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/scheduler/FIFOJobQueue.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/scheduler/FIFOJobQueue.java
@@ -121,4 +121,9 @@
public void clear() {
jobListMap.clear();
}
+
+ @Override
+ public int size() {
+ return jobListMap.size();
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/scheduler/IJobQueue.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/scheduler/IJobQueue.java
index be40883..1f2c29a 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/scheduler/IJobQueue.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/scheduler/IJobQueue.java
@@ -73,4 +73,11 @@
* Clears the job queue
*/
void clear();
+
+ /**
+ * Returns the number of queued jobs.
+ *
+ * @return the number of queued jobs.
+ */
+ int size();
}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/ApplicationMessageWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/ApplicationMessageWork.java
index 771832e..6630ba7 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/ApplicationMessageWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/ApplicationMessageWork.java
@@ -25,6 +25,7 @@
import org.apache.hyracks.api.messages.IMessage;
import org.apache.hyracks.control.cc.ClusterControllerService;
import org.apache.hyracks.control.common.deployment.DeploymentUtils;
+import org.apache.logging.log4j.Level;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
@@ -34,8 +35,8 @@
public class ApplicationMessageWork extends AbstractHeartbeatWork {
private static final Logger LOGGER = LogManager.getLogger();
- private byte[] message;
- private DeploymentId deploymentId;
+ private final byte[] message;
+ private final DeploymentId deploymentId;
public ApplicationMessageWork(ClusterControllerService ccs, byte[] message, DeploymentId deploymentId,
String nodeId) {
@@ -57,6 +58,11 @@
}
@Override
+ public Level logLevel() {
+ return Level.TRACE;
+ }
+
+ @Override
public String toString() {
return getName() + ": nodeID: " + nodeId;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetNodeControllersInfoWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetNodeControllersInfoWork.java
index c36b887..f08e209 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetNodeControllersInfoWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetNodeControllersInfoWork.java
@@ -24,10 +24,12 @@
import org.apache.hyracks.control.cc.cluster.INodeManager;
import org.apache.hyracks.control.common.work.AbstractWork;
import org.apache.hyracks.control.common.work.IResultCallback;
+import org.apache.logging.log4j.Level;
public class GetNodeControllersInfoWork extends AbstractWork {
+
private final INodeManager nodeManager;
- private IResultCallback<Map<String, NodeControllerInfo>> callback;
+ private final IResultCallback<Map<String, NodeControllerInfo>> callback;
public GetNodeControllersInfoWork(INodeManager nodeManager,
IResultCallback<Map<String, NodeControllerInfo>> callback) {
@@ -39,4 +41,9 @@
public void run() {
callback.setValue(nodeManager.getNodeControllerInfoMap());
}
+
+ @Override
+ public Level logLevel() {
+ return Level.TRACE;
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetResultDirectoryAddressWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetResultDirectoryAddressWork.java
index bf95ff2..b7dbd75 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetResultDirectoryAddressWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetResultDirectoryAddressWork.java
@@ -22,8 +22,10 @@
import org.apache.hyracks.control.cc.ClusterControllerService;
import org.apache.hyracks.control.common.work.IResultCallback;
import org.apache.hyracks.control.common.work.SynchronizableWork;
+import org.apache.logging.log4j.Level;
public class GetResultDirectoryAddressWork extends SynchronizableWork {
+
private final ClusterControllerService ccs;
private final IResultCallback<NetworkAddress> callback;
@@ -42,4 +44,9 @@
callback.setException(e);
}
}
+
+ @Override
+ public Level logLevel() {
+ return Level.TRACE;
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetResultPartitionLocationsWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetResultPartitionLocationsWork.java
index d1d2269..1e34b96 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetResultPartitionLocationsWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetResultPartitionLocationsWork.java
@@ -28,6 +28,7 @@
import org.apache.hyracks.control.cc.result.IResultDirectoryService;
import org.apache.hyracks.control.common.work.IResultCallback;
import org.apache.hyracks.control.common.work.SynchronizableWork;
+import org.apache.logging.log4j.Level;
public class GetResultPartitionLocationsWork extends SynchronizableWork {
private final ClusterControllerService ccs;
@@ -68,4 +69,9 @@
public String toString() {
return getName() + ": JobId@" + jobId + " ResultSetId@" + rsId + " Known@" + Arrays.toString(knownRecords);
}
+
+ @Override
+ public Level logLevel() {
+ return Level.TRACE;
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobCleanupWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobCleanupWork.java
index 77d2f82..6fe9909 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobCleanupWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobCleanupWork.java
@@ -28,17 +28,19 @@
import org.apache.hyracks.control.cc.job.JobRun;
import org.apache.hyracks.control.common.work.AbstractWork;
import org.apache.hyracks.control.common.work.IResultCallback;
+import org.apache.logging.log4j.Level;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
public class JobCleanupWork extends AbstractWork {
+
private static final Logger LOGGER = LogManager.getLogger();
- private IJobManager jobManager;
- private JobId jobId;
- private JobStatus status;
- private List<Exception> exceptions;
- private IResultCallback<Void> callback;
+ private final IJobManager jobManager;
+ private final JobId jobId;
+ private final JobStatus status;
+ private final List<Exception> exceptions;
+ private final IResultCallback<Void> callback;
public JobCleanupWork(IJobManager jobManager, JobId jobId, JobStatus status, List<Exception> exceptions,
IResultCallback<Void> callback) {
@@ -51,12 +53,10 @@
@Override
public void run() {
- if (LOGGER.isInfoEnabled()) {
- LOGGER.info("Cleanup for job: {}", jobId);
- }
+ LOGGER.info("cleaning up {} on NCs, status={}", jobId, status);
final JobRun jobRun = jobManager.get(jobId);
if (jobRun == null) {
- LOGGER.debug("Ignoring cleanup for unknown job: {}", jobId);
+ LOGGER.debug("ignoring cleanup for unknown {}", jobId);
return;
}
try {
@@ -80,4 +80,9 @@
return getName() + ": JobId@" + jobId + " Status@" + status
+ (exceptions == null ? "" : " Exceptions@" + exceptions);
}
+
+ @Override
+ public Level logLevel() {
+ return Level.TRACE;
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobStartWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobStartWork.java
index cfedfc9..7606dc9 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobStartWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobStartWork.java
@@ -35,6 +35,7 @@
import org.apache.hyracks.control.common.deployment.DeploymentUtils;
import org.apache.hyracks.control.common.work.IResultCallback;
import org.apache.hyracks.control.common.work.SynchronizableWork;
+import org.apache.logging.log4j.Level;
public class JobStartWork extends SynchronizableWork {
private final ClusterControllerService ccs;
@@ -85,4 +86,9 @@
callback.setException(e);
}
}
+
+ @Override
+ public Level logLevel() {
+ return Level.TRACE;
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobletCleanupNotificationWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobletCleanupNotificationWork.java
index c3a09f9..76a72c0 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobletCleanupNotificationWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobletCleanupNotificationWork.java
@@ -35,6 +35,7 @@
import org.apache.logging.log4j.Logger;
public class JobletCleanupNotificationWork extends AbstractHeartbeatWork {
+
private static final Logger LOGGER = LogManager.getLogger();
private final JobId jobId;
@@ -46,6 +47,7 @@
@Override
public void runWork() {
+ LOGGER.debug("node {} finished job clean-up {}", nodeId, jobId);
IJobManager jobManager = ccs.getJobManager();
final JobRun run = jobManager.get(jobId);
if (run == null) {
@@ -82,4 +84,9 @@
public String toString() {
return getName() + " jobId:" + jobId + ", nodeId:" + nodeId;
}
+
+ @Override
+ public Level logLevel() {
+ return Level.TRACE;
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RegisterNodeWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RegisterNodeWork.java
index ec21785..810fda2 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RegisterNodeWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RegisterNodeWork.java
@@ -31,6 +31,7 @@
import org.apache.hyracks.control.common.controllers.NodeRegistration;
import org.apache.hyracks.control.common.ipc.NodeControllerRemoteProxy;
import org.apache.hyracks.control.common.work.SynchronizableWork;
+import org.apache.logging.log4j.Level;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
@@ -83,4 +84,9 @@
nc.sendRegistrationResult(params, e);
}
}
+
+ @Override
+ public Level logLevel() {
+ return Level.TRACE;
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RegisterResultPartitionLocationWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RegisterResultPartitionLocationWork.java
index b788e27..d517761 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RegisterResultPartitionLocationWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RegisterResultPartitionLocationWork.java
@@ -90,4 +90,9 @@
+ nPartitions + " ResultPartitionLocation@" + networkAddress + " metadata@" + metadata + " EmptyResult@"
+ emptyResult;
}
+
+ @Override
+ public Level logLevel() {
+ return Level.TRACE;
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RemoveDeadNodesWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RemoveDeadNodesWork.java
index ee10669..9f740ef 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RemoveDeadNodesWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RemoveDeadNodesWork.java
@@ -50,9 +50,7 @@
Collection<JobId> affectedJobIds = result.getRight();
int size = affectedJobIds.size();
if (size > 0) {
- if (LOGGER.isInfoEnabled()) {
- LOGGER.info("Number of affected jobs: " + size);
- }
+ LOGGER.info("number of affected jobs due to dead nodes removal {}", size);
IJobManager jobManager = ccs.getJobManager();
for (JobId jobId : affectedJobIds) {
JobRun run = jobManager.get(jobId);
@@ -71,6 +69,6 @@
@Override
public Level logLevel() {
- return Level.DEBUG;
+ return Level.TRACE;
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/TaskCompleteWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/TaskCompleteWork.java
index 0c53142..80dbd2e 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/TaskCompleteWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/TaskCompleteWork.java
@@ -29,8 +29,13 @@
import org.apache.hyracks.control.common.job.profiling.om.JobProfile;
import org.apache.hyracks.control.common.job.profiling.om.JobletProfile;
import org.apache.hyracks.control.common.job.profiling.om.TaskProfile;
+import org.apache.logging.log4j.Level;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
public class TaskCompleteWork extends AbstractTaskLifecycleWork {
+
+ private static final Logger LOGGER = LogManager.getLogger();
private final TaskProfile statistics;
public TaskCompleteWork(ClusterControllerService ccs, JobId jobId, TaskAttemptId taId, String nodeId,
@@ -46,6 +51,7 @@
if (run == null) {
return;
}
+ LOGGER.debug("node completed task {}:{}:{}", nodeId, jobId, taId);
if (statistics != null) {
JobProfile jobProfile = run.getJobProfile();
Map<String, JobletProfile> jobletProfiles = jobProfile.getJobletProfiles();
@@ -63,4 +69,9 @@
public String toString() {
return getName() + ": [" + nodeId + "[" + jobId + ":" + taId + "]";
}
+
+ @Override
+ public Level logLevel() {
+ return Level.TRACE;
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/TaskFailureWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/TaskFailureWork.java
index 833066e..48fd403 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/TaskFailureWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/TaskFailureWork.java
@@ -22,17 +22,14 @@
import org.apache.hyracks.api.dataflow.TaskAttemptId;
import org.apache.hyracks.api.job.JobId;
-import org.apache.hyracks.api.util.ExceptionUtils;
+import org.apache.hyracks.api.util.ErrorMessageUtil;
import org.apache.hyracks.control.cc.ClusterControllerService;
import org.apache.hyracks.control.cc.job.IJobManager;
import org.apache.hyracks.control.cc.job.JobRun;
import org.apache.hyracks.control.cc.job.TaskAttempt;
-import org.apache.logging.log4j.Level;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
public class TaskFailureWork extends AbstractTaskLifecycleWork {
- private static final Logger LOGGER = LogManager.getLogger();
+
private final List<Exception> exceptions;
public TaskFailureWork(ClusterControllerService ccs, JobId jobId, TaskAttemptId taId, String nodeId,
@@ -43,9 +40,6 @@
@Override
protected void performEvent(TaskAttempt ta) {
- Exception ex = exceptions.get(0);
- LOGGER.log(ExceptionUtils.causedByInterrupt(ex) ? Level.DEBUG : Level.WARN,
- "Executing task failure work for " + this, ex);
IJobManager jobManager = ccs.getJobManager();
JobRun run = jobManager.get(jobId);
if (run == null) {
@@ -57,6 +51,7 @@
@Override
public String toString() {
- return getName() + ": [" + jobId + ":" + taId + ":" + nodeId + "]";
+ return getName() + ": [" + jobId + ":" + taId + ":" + nodeId + "] "
+ + ErrorMessageUtil.getCauseMessage(exceptions.get(0));
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/WaitForJobCompletionWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/WaitForJobCompletionWork.java
index 4ffe9a0..63d5340 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/WaitForJobCompletionWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/WaitForJobCompletionWork.java
@@ -29,15 +29,19 @@
import org.apache.hyracks.control.cc.job.JobRun;
import org.apache.hyracks.control.common.work.IResultCallback;
import org.apache.hyracks.control.common.work.SynchronizableWork;
+import org.apache.logging.log4j.Level;
public class WaitForJobCompletionWork extends SynchronizableWork {
private final ClusterControllerService ccs;
private final JobId jobId;
+ private final List<String> statOperatorNames;
private final IResultCallback<Object> callback;
- public WaitForJobCompletionWork(ClusterControllerService ccs, JobId jobId, IResultCallback<Object> callback) {
+ public WaitForJobCompletionWork(ClusterControllerService ccs, JobId jobId, List<String> statOperatorNames,
+ IResultCallback<Object> callback) {
this.ccs = ccs;
this.jobId = jobId;
+ this.statOperatorNames = statOperatorNames;
this.callback = callback;
}
@@ -53,7 +57,7 @@
Thread.currentThread()
.setName(Thread.currentThread().getName() + " : WaitForCompletionForJobId: " + jobId);
jobRun.waitForCompletion();
- callback.setValue(null);
+ callback.setValue(jobRun.getJobProfile().getAggregatedStats(statOperatorNames));
} catch (Exception e) {
callback.setException(e);
}
@@ -89,4 +93,9 @@
public String toString() {
return getName() + " jobId:" + jobId;
}
+
+ @Override
+ public Level logLevel() {
+ return Level.TRACE;
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/pom.xml b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/pom.xml
index 65152a9..2455f76 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/pom.xml
@@ -24,7 +24,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks-control</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<properties>
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/config/ConfigManager.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/config/ConfigManager.java
index 85661fe..9c5a9fa 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/config/ConfigManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/config/ConfigManager.java
@@ -148,7 +148,7 @@
if (configured) {
throw new IllegalStateException("configuration already processed");
}
- LOGGER.debug("registering option: " + option.toIniString());
+ LOGGER.trace("registering option: {}", option::toIniString);
Map<String, IOption> optionMap = sectionMap.computeIfAbsent(option.section(), section -> new HashMap<>());
IOption prev = optionMap.put(option.ini(), option);
if (prev != null) {
@@ -160,8 +160,13 @@
registeredOptions.add(option);
optionSetters.put(option, (node, value, isDefault) -> correctedMap(node, isDefault).put(option, value));
if (LOGGER.isDebugEnabled()) {
- optionSetters.put(option, (node, value, isDefault) -> LOGGER.debug("{} {} to {} for node {}",
- isDefault ? "defaulting" : "setting", option.toIniString(), value, node));
+ optionSetters.put(option, (node, value, isDefault) -> {
+ if (isDefault) {
+ LOGGER.trace("defaulting {} to {} for node {}", option.toIniString(), value, node);
+ } else {
+ LOGGER.debug("setting {} to {} for node {}", option.toIniString(), value, node);
+ }
+ });
}
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/StatsCollector.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/StatsCollector.java
index 878d7c3..580a560 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/StatsCollector.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/StatsCollector.java
@@ -52,6 +52,11 @@
}
@Override
+ public IOperatorStats getOperatorStats(String operatorName) {
+ return operatorStatsMap.get(operatorName);
+ }
+
+ @Override
public Map<String, IOperatorStats> getAllOperatorStats() {
return Collections.unmodifiableMap(operatorStatsMap);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/om/JobProfile.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/om/JobProfile.java
index c4eff85..ee49908 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/om/JobProfile.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/om/JobProfile.java
@@ -21,18 +21,25 @@
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
+import java.util.Arrays;
import java.util.HashMap;
+import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
+import org.apache.hyracks.api.dataflow.TaskAttemptId;
+import org.apache.hyracks.api.dataflow.TaskId;
import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
+import org.apache.hyracks.api.job.profiling.IStatsCollector;
+import org.apache.hyracks.api.job.profiling.OperatorStats;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.node.ArrayNode;
import com.fasterxml.jackson.databind.node.ObjectNode;
public class JobProfile extends AbstractProfile {
- private static final long serialVersionUID = 1L;
+ private static final long serialVersionUID = 2L;
private JobId jobId;
@@ -109,4 +116,42 @@
entry.getValue().writeFields(output);
}
}
+
+ public List<IOperatorStats> getAggregatedStats(List<String> operatorNames) {
+ if (jobletProfiles == null || operatorNames == null || operatorNames.isEmpty()) {
+ return null;
+ }
+ // gather final task attempts for each task
+ Map<TaskId, TaskProfile> taskProfileMap = new HashMap<>();
+ for (JobletProfile jobletProfile : jobletProfiles.values()) {
+ for (TaskProfile taskProfile : jobletProfile.getTaskProfiles().values()) {
+ TaskAttemptId taskAttemptId = taskProfile.getTaskId();
+ TaskId taskId = taskAttemptId.getTaskId();
+ TaskProfile existingProfile = taskProfileMap.get(taskId);
+ if (existingProfile == null || taskAttemptId.getAttempt() > existingProfile.getTaskId().getAttempt()) {
+ taskProfileMap.put(taskId, taskProfile);
+ }
+ }
+ }
+ // compute aggregated counts
+ int n = operatorNames.size();
+ IOperatorStats[] outStats = new IOperatorStats[n];
+ for (TaskProfile taskProfile : taskProfileMap.values()) {
+ IStatsCollector statsCollector = taskProfile.getStatsCollector();
+ for (int i = 0; i < n; i++) {
+ String operatorName = operatorNames.get(i);
+ IOperatorStats opTaskStats = statsCollector.getOperatorStats(operatorName);
+ if (opTaskStats == null) {
+ continue;
+ }
+ IOperatorStats opOutStats = outStats[i];
+ if (opOutStats == null) {
+ opOutStats = new OperatorStats(operatorName);
+ outStats[i] = opOutStats;
+ }
+ opOutStats.updateFrom(opTaskStats);
+ }
+ }
+ return Arrays.asList(outStats);
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/pom.xml b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/pom.xml
index 083d268..64b7145 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/pom.xml
@@ -24,7 +24,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks-control</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<properties>
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/CcConnection.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/CcConnection.java
index d1f7d5a..c99898d 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/CcConnection.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/CcConnection.java
@@ -70,6 +70,8 @@
InvokeUtil.runWithTimeout(() -> {
this.wait(REGISTRATION_RESPONSE_POLL_PERIOD); // NOSONAR while loop in timeout call
}, () -> !registrationPending, 1, TimeUnit.MINUTES);
+ } catch (InterruptedException e) {
+ throw e;
} catch (Exception e) {
registrationException = e;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/Joblet.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/Joblet.java
index f179c36..a96dfe5 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/Joblet.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/Joblet.java
@@ -108,6 +108,7 @@
private final String jobStartTimeZoneId;
private final long maxWarnings;
+ private final AtomicLong uniqueIds;
public Joblet(NodeControllerService nodeController, DeploymentId deploymentId, JobId jobId,
INCServiceContext serviceCtx, ActivityClusterGraph acg,
@@ -140,6 +141,7 @@
this.jobStartTime = jobStartTime;
this.jobStartTimeZoneId = jobStartTimeZoneId;
this.maxWarnings = acg.getMaxWarnings();
+ this.uniqueIds = new AtomicLong();
}
@Override
@@ -161,6 +163,11 @@
}
@Override
+ public long nextUniqueId() {
+ return uniqueIds.getAndIncrement();
+ }
+
+ @Override
public long getJobStartTime() {
return jobStartTime;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
index f69d106..cc84aeb 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
@@ -35,6 +35,7 @@
import java.util.TimerTask;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
@@ -72,6 +73,7 @@
import org.apache.hyracks.control.common.ipc.CCNCFunctions;
import org.apache.hyracks.control.common.ipc.ClusterControllerRemoteProxy;
import org.apache.hyracks.control.common.job.profiling.om.JobProfile;
+import org.apache.hyracks.control.common.utils.HyracksThreadFactory;
import org.apache.hyracks.control.common.work.FutureValue;
import org.apache.hyracks.control.common.work.WorkQueue;
import org.apache.hyracks.control.nc.application.NCServiceContext;
@@ -145,7 +147,7 @@
private final Map<JobId, JobParameterByteStore> jobParameterByteStoreMap = new HashMap<>();
- private ExecutorService executor;
+ private ExecutorService executor = Executors.newCachedThreadPool(new HyracksThreadFactory("<bootstrap>"));
private Map<CcId, HeartbeatManager> heartbeatManagers = new ConcurrentHashMap<>();
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/Task.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/Task.java
index 158e24e..8a1bbda 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/Task.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/Task.java
@@ -63,6 +63,7 @@
import org.apache.hyracks.api.partitions.PartitionId;
import org.apache.hyracks.api.resources.IDeallocatable;
import org.apache.hyracks.api.result.IResultPartitionManager;
+import org.apache.hyracks.api.util.CleanupUtils;
import org.apache.hyracks.api.util.ExceptionUtils;
import org.apache.hyracks.api.util.JavaSerializationUtils;
import org.apache.hyracks.control.common.job.PartitionState;
@@ -87,6 +88,8 @@
private final TaskAttemptId taskAttemptId;
+ private final int partitionCount;
+
private final String displayName;
private final ExecutorService executorService;
@@ -131,12 +134,13 @@
private final Map<Long, IThreadStats> perThreadStats = new HashMap<>();
- public Task(Joblet joblet, Set<JobFlag> jobFlags, TaskAttemptId taskId, String displayName,
+ public Task(Joblet joblet, Set<JobFlag> jobFlags, TaskAttemptId taskId, int partitionCount, String displayName,
ExecutorService executor, NodeControllerService ncs,
List<List<PartitionChannel>> inputChannelsFromConnectors) {
this.joblet = joblet;
this.jobFlags = jobFlags;
this.taskAttemptId = taskId;
+ this.partitionCount = partitionCount;
this.displayName = displayName;
this.executorService = executor;
fileFactory = new WorkspaceFileFactory(this, joblet.getIoManager());
@@ -225,6 +229,11 @@
}
@Override
+ public int getPartitionCount() {
+ return partitionCount;
+ }
+
+ @Override
public ICounter getCounter(String name, boolean create) {
Counter counter = counterMap.get(name);
if (counter == null && create) {
@@ -398,6 +407,7 @@
if (aborted) {
return;
}
+ Throwable originalEx = null;
try {
collector.open();
try {
@@ -422,23 +432,24 @@
buffer.compact();
}
} catch (Exception e) {
- try {
- writer.fail();
- } catch (HyracksDataException e1) {
- e.addSuppressed(e1);
- }
- throw e;
+ originalEx = e;
+ CleanupUtils.fail(writer, originalEx);
} finally {
- writer.close();
+ originalEx = CleanupUtils.closeSilently(writer, originalEx);
}
} finally {
- reader.close();
+ originalEx = CleanupUtils.closeSilently(reader, originalEx);
}
+ } catch (Exception e) {
+ originalEx = ExceptionUtils.suppress(originalEx, e);
} finally {
- collector.close();
+ originalEx = CleanupUtils.closeSilently(collector, originalEx);
}
} catch (Exception e) {
- throw HyracksDataException.create(e);
+ originalEx = ExceptionUtils.suppress(originalEx, e);
+ }
+ if (originalEx != null) {
+ throw HyracksDataException.create(originalEx);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/net/NetworkManager.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/net/NetworkManager.java
index 6876618..468a969 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/net/NetworkManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/net/NetworkManager.java
@@ -31,8 +31,12 @@
import org.apache.hyracks.api.job.JobId;
import org.apache.hyracks.api.network.ISocketChannelFactory;
import org.apache.hyracks.api.partitions.PartitionId;
+import org.apache.hyracks.comm.channels.FileNetworkInputChannel;
import org.apache.hyracks.comm.channels.IChannelConnectionFactory;
+import org.apache.hyracks.comm.channels.NetworkInputChannel;
import org.apache.hyracks.comm.channels.NetworkOutputChannel;
+import org.apache.hyracks.control.nc.NodeControllerService;
+import org.apache.hyracks.control.nc.partitions.PartitionFileReaderUtil;
import org.apache.hyracks.control.nc.partitions.PartitionManager;
import org.apache.hyracks.net.protocols.muxdemux.ChannelControlBlock;
import org.apache.hyracks.net.protocols.muxdemux.IChannelOpenListener;
@@ -47,8 +51,6 @@
private static final int MAX_CONNECTION_ATTEMPTS = 5;
- static final int INITIAL_MESSAGE_SIZE = 20;
-
private final PartitionManager partitionManager;
private final int nBuffers;
@@ -113,7 +115,8 @@
@Override
public void channelOpened(ChannelControlBlock channel) {
channel.getReadInterface().setFullBufferAcceptor(new InitialBufferAcceptor(channel));
- channel.getReadInterface().getEmptyBufferAcceptor().accept(ByteBuffer.allocate(INITIAL_MESSAGE_SIZE));
+ channel.getReadInterface().getEmptyBufferAcceptor()
+ .accept(ByteBuffer.allocate(NetworkInputChannel.INITIAL_MESSAGE_SIZE));
}
}
@@ -128,12 +131,13 @@
@Override
public void accept(ByteBuffer buffer) {
- PartitionId pid = readInitialMessage(buffer);
- if (LOGGER.isTraceEnabled()) {
- LOGGER.trace("Received initial partition request: " + pid + " on channel: " + ccb);
- }
noc = new NetworkOutputChannel(ccb, nBuffers);
- partitionManager.registerPartitionRequest(pid, noc);
+ long id = buffer.getLong();
+ if (id == FileNetworkInputChannel.FILE_CHANNEL_CODE) {
+ handleFileRequest(buffer);
+ } else {
+ handlePartitionRequest(buffer, id);
+ }
}
@Override
@@ -147,16 +151,37 @@
noc.abort(ecode);
}
}
+
+ private void handlePartitionRequest(ByteBuffer buffer, long jid) {
+ PartitionId pid = readInitialMessage(buffer, jid);
+ if (LOGGER.isTraceEnabled()) {
+ LOGGER.trace("Received initial partition request: " + pid + " on channel: " + ccb);
+ }
+ partitionManager.registerPartitionRequest(pid, noc);
+ }
+
+ private void handleFileRequest(ByteBuffer buffer) {
+ JobId jobId = new JobId(buffer.getLong());
+ long fileId = buffer.getLong();
+ if (partitionManager.registerFileRequest(jobId, noc)) {
+ writeFileToChannel(partitionManager.getNodeControllerService(), noc, jobId, fileId);
+ }
+ }
}
- private static PartitionId readInitialMessage(ByteBuffer buffer) {
- JobId jobId = new JobId(buffer.getLong());
+ private static PartitionId readInitialMessage(ByteBuffer buffer, long jid) {
+ JobId jobId = new JobId(jid);
ConnectorDescriptorId cdid = new ConnectorDescriptorId(buffer.getInt());
int senderIndex = buffer.getInt();
int receiverIndex = buffer.getInt();
return new PartitionId(jobId, cdid, senderIndex, receiverIndex);
}
+ private static void writeFileToChannel(NodeControllerService ncs, NetworkOutputChannel noc, JobId jobId,
+ long fileId) {
+ PartitionFileReaderUtil.writeFileToChannel(ncs, noc, jobId, fileId);
+ }
+
public MuxDemuxPerformanceCounters getPerformanceCounters() {
return md.getPerformanceCounters();
}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/partitions/JobFileState.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/partitions/JobFileState.java
new file mode 100644
index 0000000..e4e6f09
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/partitions/JobFileState.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.control.nc.partitions;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hyracks.api.dataflow.state.IStateObject;
+import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.api.job.JobId;
+
+public class JobFileState implements IStateObject {
+
+ private final FileReference fileRef;
+ private final JobId jobId;
+ private final JobFileId jobFileId;
+
+ public JobFileState(FileReference fileRef, JobId jobId, long fileId) {
+ this.fileRef = fileRef;
+ this.jobFileId = new JobFileId(fileId);
+ this.jobId = jobId;
+ }
+
+ @Override
+ public JobId getJobId() {
+ return jobId;
+ }
+
+ @Override
+ public Object getId() {
+ return jobFileId;
+ }
+
+ @Override
+ public long getMemoryOccupancy() {
+ return 0;
+ }
+
+ @Override
+ public void toBytes(DataOutput out) throws IOException {
+
+ }
+
+ @Override
+ public void fromBytes(DataInput in) throws IOException {
+
+ }
+
+ public FileReference getFileRef() {
+ return fileRef;
+ }
+
+ public static class JobFileId {
+
+ private final long fileId;
+
+ public JobFileId(long fileId) {
+ this.fileId = fileId;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (!(o instanceof JobFileId)) {
+ return false;
+ }
+ JobFileId otherFileId = (JobFileId) o;
+ return otherFileId.fileId == fileId;
+ }
+
+ @Override
+ public int hashCode() {
+ return Long.hashCode(fileId);
+ }
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/partitions/MaterializedPartition.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/partitions/MaterializedPartition.java
index 3e36946..1471453 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/partitions/MaterializedPartition.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/partitions/MaterializedPartition.java
@@ -18,14 +18,11 @@
*/
package org.apache.hyracks.control.nc.partitions;
-import java.nio.ByteBuffer;
import java.util.concurrent.Executor;
import org.apache.hyracks.api.comm.IFrameWriter;
import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.io.FileReference;
-import org.apache.hyracks.api.io.IFileHandle;
import org.apache.hyracks.api.io.IIOManager;
import org.apache.hyracks.api.partitions.IPartition;
@@ -60,45 +57,7 @@
@Override
public void writeTo(final IFrameWriter writer) {
- executor.execute(new Runnable() {
- @Override
- public void run() {
- try {
- if (partitionFile == null) {
- writer.open();
- writer.close();
- return;
- }
- IFileHandle fh = ioManager.open(partitionFile, IIOManager.FileReadWriteMode.READ_ONLY,
- IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
- try {
- writer.open();
- try {
- long offset = 0;
- ByteBuffer buffer = ctx.allocateFrame();
- while (true) {
- buffer.clear();
- long size = ioManager.syncRead(fh, offset, buffer);
- if (size < 0) {
- break;
- } else if (size < buffer.capacity()) {
- throw new HyracksDataException("Premature end of file");
- }
- offset += size;
- buffer.flip();
- writer.nextFrame(buffer);
- }
- } finally {
- writer.close();
- }
- } finally {
- ioManager.close(fh);
- }
- } catch (HyracksDataException e) {
- throw new RuntimeException(e);
- }
- }
- });
+ executor.execute(new PartitionFileReader(ctx, partitionFile, ioManager, writer, false));
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/partitions/MaterializingPipelinedPartition.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/partitions/MaterializingPipelinedPartition.java
index eee8950..e52e3ac 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/partitions/MaterializingPipelinedPartition.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/partitions/MaterializingPipelinedPartition.java
@@ -52,7 +52,6 @@
private boolean failed;
protected boolean flushRequest;
private boolean deallocated;
- private Level openCloseLevel = Level.DEBUG;
private Thread dataConsumerThread;
public MaterializingPipelinedPartition(IHyracksTaskContext ctx, PartitionManager manager, PartitionId pid,
@@ -181,9 +180,6 @@
@Override
public void open() throws HyracksDataException {
- if (LOGGER.isEnabled(openCloseLevel)) {
- LOGGER.log(openCloseLevel, "open(" + pid + " by " + taId);
- }
size = 0;
eos = false;
failed = false;
@@ -215,9 +211,6 @@
@Override
public void close() throws HyracksDataException {
- if (LOGGER.isEnabled(openCloseLevel)) {
- LOGGER.log(openCloseLevel, "close(" + pid + " by " + taId);
- }
if (writeHandle != null) {
ctx.getIoManager().close(writeHandle);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/partitions/PartitionFileReader.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/partitions/PartitionFileReader.java
new file mode 100644
index 0000000..609b32a
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/partitions/PartitionFileReader.java
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.control.nc.partitions;
+
+import java.nio.ByteBuffer;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.context.IHyracksCommonContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.api.io.IFileHandle;
+import org.apache.hyracks.api.io.IIOManager;
+
+public class PartitionFileReader implements Runnable {
+
+ private final IHyracksCommonContext ctx;
+ private final FileReference partitionFile;
+ private final IIOManager ioManager;
+ private final IFrameWriter writer;
+ private final boolean deleteFile;
+
+ public PartitionFileReader(IHyracksCommonContext ctx, FileReference partitionFile, IIOManager ioManager,
+ IFrameWriter writer, boolean deleteFile) {
+ this.ctx = ctx;
+ this.partitionFile = partitionFile;
+ this.ioManager = ioManager;
+ this.writer = writer;
+ this.deleteFile = deleteFile;
+ }
+
+ @Override
+ public void run() {
+ try {
+ if (partitionFile == null) {
+ writer.open();
+ writer.close();
+ return;
+ }
+ IFileHandle fh = ioManager.open(partitionFile, IIOManager.FileReadWriteMode.READ_ONLY,
+ IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
+ try {
+ writer.open();
+ try {
+ long offset = 0;
+ ByteBuffer buffer = ctx.allocateFrame();
+ while (true) {
+ buffer.clear();
+ long size = ioManager.syncRead(fh, offset, buffer);
+ if (size < 0) {
+ break;
+ } else if (size < buffer.capacity()) {
+ throw new HyracksDataException("Premature end of file");
+ }
+ offset += size;
+ buffer.flip();
+ writer.nextFrame(buffer);
+ }
+ } finally {
+ writer.close();
+ }
+ } finally {
+ try {
+ ioManager.close(fh);
+ } finally {
+ if (deleteFile) {
+ FileUtils.deleteQuietly(partitionFile.getFile());
+ }
+ }
+ }
+ } catch (HyracksDataException e) {
+ throw new RuntimeException(e);
+ }
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/partitions/PartitionFileReaderUtil.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/partitions/PartitionFileReaderUtil.java
new file mode 100644
index 0000000..2a105d9
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/partitions/PartitionFileReaderUtil.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.control.nc.partitions;
+
+import java.util.concurrent.ExecutorService;
+
+import org.apache.hyracks.api.dataflow.state.IStateObject;
+import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.comm.channels.NetworkOutputChannel;
+import org.apache.hyracks.control.nc.Joblet;
+import org.apache.hyracks.control.nc.NodeControllerService;
+import org.apache.hyracks.net.protocols.muxdemux.AbstractChannelWriteInterface;
+
+public class PartitionFileReaderUtil {
+
+ private PartitionFileReaderUtil() {
+ }
+
+ public static void writeFileToChannel(NodeControllerService ncs, NetworkOutputChannel noc, JobId jobId,
+ long fileId) {
+ Joblet joblet = ncs.getJobletMap().get(jobId);
+ if (joblet == null) {
+ noc.abort(AbstractChannelWriteInterface.REMOTE_ERROR_CODE);
+ return;
+ }
+ IStateObject stateObject = joblet.getEnvironment().getStateObject(new JobFileState.JobFileId(fileId));
+ if (!(stateObject instanceof JobFileState)) {
+ noc.abort(AbstractChannelWriteInterface.REMOTE_ERROR_CODE);
+ return;
+ }
+ JobFileState fileState = (JobFileState) stateObject;
+ FileReference fileRef = fileState.getFileRef();
+ if (!fileRef.getFile().exists()) {
+ noc.abort(AbstractChannelWriteInterface.REMOTE_ERROR_CODE);
+ return;
+ }
+ ExecutorService executor = ncs.getExecutor();
+ noc.setFrameSize(joblet.getInitialFrameSize());
+ executor.execute(new PartitionFileReader(joblet, fileRef, ncs.getIoManager(), noc, true));
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/partitions/PartitionManager.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/partitions/PartitionManager.java
index c082b71..927dd7c 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/partitions/PartitionManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/partitions/PartitionManager.java
@@ -58,6 +58,8 @@
private final Map<PartitionId, NetworkOutputChannel> partitionRequests = new HashMap<>();
+ private final Map<JobId, List<NetworkOutputChannel>> fileRequests = new HashMap<>();
+
private final Cache<JobId, JobId> failedJobsCache;
public PartitionManager(NodeControllerService ncs) {
@@ -122,10 +124,24 @@
}
}
+ public synchronized boolean registerFileRequest(JobId jid, NetworkOutputChannel noc) {
+ if (failedJobsCache.getIfPresent(jid) != null) {
+ noc.abort(AbstractChannelWriteInterface.REMOTE_ERROR_CODE);
+ return false;
+ }
+ List<NetworkOutputChannel> netOutChannels = fileRequests.computeIfAbsent(jid, k -> new ArrayList<>());
+ netOutChannels.add(noc);
+ return true;
+ }
+
public IWorkspaceFileFactory getFileFactory() {
return fileFactory;
}
+ public NodeControllerService getNodeControllerService() {
+ return ncs;
+ }
+
public void close() {
deallocatableRegistry.close();
}
@@ -175,6 +191,10 @@
}
private List<NetworkOutputChannel> removePendingRequests(JobId jobId, JobStatus status) {
+ List<NetworkOutputChannel> jobFileRequests = null;
+ if (!fileRequests.isEmpty()) {
+ jobFileRequests = fileRequests.remove(jobId);
+ }
if (status != JobStatus.FAILURE) {
return Collections.emptyList();
}
@@ -189,6 +209,9 @@
requestsIterator.remove();
}
}
+ if (jobFileRequests != null && !jobFileRequests.isEmpty()) {
+ pendingRequests.addAll(jobFileRequests);
+ }
return pendingRequests;
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/ApplicationMessageWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/ApplicationMessageWork.java
index 6d4f173..f386a89 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/ApplicationMessageWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/ApplicationMessageWork.java
@@ -29,11 +29,12 @@
import org.apache.logging.log4j.Logger;
public class ApplicationMessageWork extends AbstractWork {
+
private static final Logger LOGGER = LogManager.getLogger();
- private byte[] message;
- private DeploymentId deploymentId;
- private String nodeId;
- private NodeControllerService ncs;
+ private final byte[] message;
+ private final DeploymentId deploymentId;
+ private final String nodeId;
+ private final NodeControllerService ncs;
public ApplicationMessageWork(NodeControllerService ncs, byte[] message, DeploymentId deploymentId, String nodeId) {
this.ncs = ncs;
@@ -59,6 +60,11 @@
}
@Override
+ public Level logLevel() {
+ return Level.TRACE;
+ }
+
+ @Override
public String toString() {
return getName() + ": nodeId: " + nodeId;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/CleanupJobletWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/CleanupJobletWork.java
index 75edd38..2036d72 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/CleanupJobletWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/CleanupJobletWork.java
@@ -25,10 +25,12 @@
import org.apache.hyracks.control.common.work.AbstractWork;
import org.apache.hyracks.control.nc.Joblet;
import org.apache.hyracks.control.nc.NodeControllerService;
+import org.apache.logging.log4j.Level;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
public class CleanupJobletWork extends AbstractWork {
+
private static final Logger LOGGER = LogManager.getLogger();
private final NodeControllerService ncs;
@@ -45,7 +47,7 @@
@Override
public void run() {
- LOGGER.debug("cleaning up after job: {}", jobId);
+ LOGGER.debug("cleaning up {}, status:{}", jobId, status);
ncs.removeJobParameterByteStore(jobId);
ncs.getPartitionManager().jobCompleted(jobId, status);
Map<JobId, Joblet> jobletMap = ncs.getJobletMap();
@@ -59,4 +61,9 @@
public String toString() {
return getName() + " jobId:" + jobId + ", status:" + status;
}
+
+ @Override
+ public Level logLevel() {
+ return Level.TRACE;
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/NotifyTaskCompleteWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/NotifyTaskCompleteWork.java
index 60860c5..52469dc 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/NotifyTaskCompleteWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/NotifyTaskCompleteWork.java
@@ -18,6 +18,8 @@
*/
package org.apache.hyracks.control.nc.work;
+import org.apache.hyracks.api.dataflow.TaskAttemptId;
+import org.apache.hyracks.api.job.JobId;
import org.apache.hyracks.control.common.job.profiling.om.TaskProfile;
import org.apache.hyracks.control.common.work.AbstractWork;
import org.apache.hyracks.control.nc.NodeControllerService;
@@ -38,13 +40,16 @@
@Override
public void run() {
- TaskProfile taskProfile = new TaskProfile(task.getTaskAttemptId(), task.getPartitionSendProfile(),
+ JobId jobId = task.getJoblet().getJobId();
+ TaskAttemptId taskAttemptId = task.getTaskAttemptId();
+ LOGGER.debug("notifying CC of task complete {}:{}", jobId, taskAttemptId);
+ TaskProfile taskProfile = new TaskProfile(taskAttemptId, task.getPartitionSendProfile(),
task.getStatsCollector(), task.getWarnings(), task.getWarningCollector().getTotalWarningsCount());
try {
- ncs.getClusterController(task.getJobletContext().getJobId().getCcId()).notifyTaskComplete(
- task.getJobletContext().getJobId(), task.getTaskAttemptId(), ncs.getId(), taskProfile);
+ ncs.getClusterController(task.getJobletContext().getJobId().getCcId())
+ .notifyTaskComplete(task.getJobletContext().getJobId(), taskAttemptId, ncs.getId(), taskProfile);
} catch (Exception e) {
- LOGGER.log(Level.ERROR, "Failed notifying task complete for " + task.getTaskAttemptId(), e);
+ LOGGER.log(Level.ERROR, "Failed notifying task complete for {}", taskAttemptId, e);
}
task.getJoblet().removeTask(task);
}
@@ -54,4 +59,9 @@
return getName() + ": [" + ncs.getId() + "[" + task.getJoblet().getJobId() + ":" + task.getTaskAttemptId()
+ "]";
}
+
+ @Override
+ public Level logLevel() {
+ return Level.TRACE;
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/NotifyTaskFailureWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/NotifyTaskFailureWork.java
index b0c60aa..cd79da7 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/NotifyTaskFailureWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/NotifyTaskFailureWork.java
@@ -23,7 +23,7 @@
import org.apache.hyracks.api.dataflow.TaskAttemptId;
import org.apache.hyracks.api.job.JobId;
import org.apache.hyracks.api.result.IResultPartitionManager;
-import org.apache.hyracks.api.util.ExceptionUtils;
+import org.apache.hyracks.api.util.ErrorMessageUtil;
import org.apache.hyracks.control.common.work.AbstractWork;
import org.apache.hyracks.control.nc.NodeControllerService;
import org.apache.hyracks.control.nc.Task;
@@ -50,9 +50,6 @@
@Override
public void run() {
- Exception ex = exceptions.get(0);
- LOGGER.log(ExceptionUtils.causedByInterrupt(ex) ? Level.DEBUG : Level.WARN, "task " + taskId + " has failed",
- ex);
try {
IResultPartitionManager resultPartitionManager = ncs.getResultPartitionManager();
if (resultPartitionManager != null) {
@@ -69,6 +66,8 @@
@Override
public String toString() {
- return getName() + ": [" + ncs.getId() + "[" + jobId + ":" + taskId + "]";
+ return getName() + ": [" + ncs.getId() + "[" + jobId + ":" + taskId + "]"
+ + ((exceptions != null && !exceptions.isEmpty())
+ ? " " + ErrorMessageUtil.getCauseMessage(exceptions.get(0)) : "");
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/StartTasksWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/StartTasksWork.java
index 442f0d0..dd4a956 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/StartTasksWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/StartTasksWork.java
@@ -152,11 +152,12 @@
ActivityCluster ac = acg.getActivityMap().get(aid);
IActivity han = ac.getActivityMap().get(aid);
LOGGER.trace("Initializing {} -> {} for {}", taId, han, jobId);
+ int partitionCount = td.getPartitionCount();
final int partition = tid.getPartition();
List<IConnectorDescriptor> inputs = ac.getActivityInputMap().get(aid);
- task = new Task(joblet, flags, taId, han.getDisplayName(), ncs.getExecutor(), ncs,
+ task = new Task(joblet, flags, taId, partitionCount, han.getDisplayName(), ncs.getExecutor(), ncs,
createInputChannels(td, inputs));
- IOperatorNodePushable operator = han.createPushRuntime(task, rdp, partition, td.getPartitionCount());
+ IOperatorNodePushable operator = han.createPushRuntime(task, rdp, partition, partitionCount);
List<IPartitionCollector> collectors = new ArrayList<>();
if (inputs != null) {
for (int i = 0; i < inputs.size(); ++i) {
@@ -182,7 +183,7 @@
createPartitionWriterFactory(task, cPolicy, jobId, conn, partition, taId, flags);
LOGGER.trace("input: {}: {}", i, conn.getConnectorId());
IFrameWriter writer = conn.createPartitioner(task, recordDesc, pwFactory, partition,
- td.getPartitionCount(), td.getOutputPartitionCounts()[i]);
+ partitionCount, td.getOutputPartitionCounts()[i]);
writer = (enforce && !profile) ? EnforceFrameWriter.enforce(writer) : writer;
operator.setOutputFrameWriter(i, writer, recordDesc);
}
@@ -317,6 +318,6 @@
@Override
public String toString() {
- return getName() + " jobId:" + jobId;
+ return getName() + " jobId:" + jobId + " tasks:" + taskDescriptors.size();
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-nc-service/pom.xml b/hyracks-fullstack/hyracks/hyracks-control/hyracks-nc-service/pom.xml
index db09f40..d73aec4 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-nc-service/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-nc-service/pom.xml
@@ -21,7 +21,7 @@
<parent>
<artifactId>hyracks-control</artifactId>
<groupId>org.apache.hyracks</groupId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<modelVersion>4.0.0</modelVersion>
diff --git a/hyracks-fullstack/hyracks/hyracks-control/pom.xml b/hyracks-fullstack/hyracks/hyracks-control/pom.xml
index bf8bb52..8105c24 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-control/pom.xml
@@ -26,7 +26,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/pom.xml b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/pom.xml
index dfc894f..f136618 100644
--- a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/pom.xml
@@ -25,7 +25,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks-data</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<properties>
<root.dir>${basedir}/../../..</root.dir>
diff --git a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/UTF8StringPointable.java b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/UTF8StringPointable.java
index 49f6221..47d2488 100644
--- a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/UTF8StringPointable.java
+++ b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/UTF8StringPointable.java
@@ -18,8 +18,10 @@
*/
package org.apache.hyracks.data.std.primitive;
+import static org.apache.hyracks.api.exceptions.ErrorCode.INVALID_STRING_UNICODE;
import static org.apache.hyracks.util.string.UTF8StringUtil.HIGH_SURROGATE_WITHOUT_LOW_SURROGATE;
import static org.apache.hyracks.util.string.UTF8StringUtil.LOW_SURROGATE_WITHOUT_HIGH_SURROGATE;
+import static org.apache.hyracks.util.string.UTF8StringUtil.MALFORMED_BYTES;
import java.io.IOException;
import java.nio.charset.Charset;
@@ -120,15 +122,15 @@
return UTF8StringUtil.charSize(bytes, start + offset);
}
- public int codePointAt(int offset) {
+ public int codePointAt(int offset) throws HyracksDataException {
return UTF8StringUtil.codePointAt(bytes, start + offset);
}
- public int codePointSize(int offset) {
+ public int codePointSize(int offset) throws HyracksDataException {
return UTF8StringUtil.codePointSize(bytes, start + offset);
}
- public void getCodePoints(IntCollection codePointSet) {
+ public void getCodePoints(IntCollection codePointSet) throws HyracksDataException {
int byteIdx = 0;
while (byteIdx < utf8Length) {
codePointSet.add(codePointAt(metaLength + byteIdx));
@@ -136,7 +138,7 @@
}
if (byteIdx != utf8Length) {
- throw new IllegalArgumentException("Decoding error: malformed bytes");
+ throw HyracksDataException.create(INVALID_STRING_UNICODE, MALFORMED_BYTES);
}
}
@@ -202,7 +204,7 @@
other.getStartOffset());
}
- public int find(UTF8StringPointable pattern, boolean ignoreCase) {
+ public int find(UTF8StringPointable pattern, boolean ignoreCase) throws HyracksDataException {
return find(this, pattern, ignoreCase);
}
@@ -227,7 +229,8 @@
* to ignore case or not.
* @return the byte offset of the first character of the matching string. Not including the MetaLength.
*/
- public static int find(UTF8StringPointable src, UTF8StringPointable pattern, boolean ignoreCase) {
+ public static int find(UTF8StringPointable src, UTF8StringPointable pattern, boolean ignoreCase)
+ throws HyracksDataException {
return find(src, pattern, ignoreCase, 0);
}
@@ -240,7 +243,8 @@
* to ignore case or not.
* @return the offset in the unit of code point of the first character of the matching string. Not including the MetaLength.
*/
- public static int findInCodePoint(UTF8StringPointable src, UTF8StringPointable pattern, boolean ignoreCase) {
+ public static int findInCodePoint(UTF8StringPointable src, UTF8StringPointable pattern, boolean ignoreCase)
+ throws HyracksDataException {
return findInByteOrCodePoint(src, pattern, ignoreCase, 0, false);
}
@@ -256,7 +260,8 @@
* @return the byte offset of the first character of the matching string after <code>startMatchPos}</code>.
* Not including the MetaLength.
*/
- public static int find(UTF8StringPointable src, UTF8StringPointable pattern, boolean ignoreCase, int startMatch) {
+ public static int find(UTF8StringPointable src, UTF8StringPointable pattern, boolean ignoreCase, int startMatch)
+ throws HyracksDataException {
return findInByteOrCodePoint(src, pattern, ignoreCase, startMatch, true);
}
@@ -272,13 +277,13 @@
* @return the offset in the unit of code point of the first character of the matching string. Not including the MetaLength.
*/
public static int findInCodePoint(UTF8StringPointable src, UTF8StringPointable pattern, boolean ignoreCase,
- int startMatch) {
+ int startMatch) throws HyracksDataException {
return findInByteOrCodePoint(src, pattern, ignoreCase, startMatch, false);
}
// If resultInByte is true, then return the position in bytes, otherwise return the position in code points
private static int findInByteOrCodePoint(UTF8StringPointable src, UTF8StringPointable pattern, boolean ignoreCase,
- int startMatch, boolean resultInByte) {
+ int startMatch, boolean resultInByte) throws HyracksDataException {
int startMatchPos = startMatch;
final int srcUtfLen = src.getUTF8Length();
final int pttnUtfLen = pattern.getUTF8Length();
@@ -317,7 +322,7 @@
return startMatchPos;
} else {
if (prevHighSurrogate) {
- throw new IllegalArgumentException(HIGH_SURROGATE_WITHOUT_LOW_SURROGATE);
+ throw HyracksDataException.create(INVALID_STRING_UNICODE, HIGH_SURROGATE_WITHOUT_LOW_SURROGATE);
}
return codePointCount;
}
@@ -333,7 +338,7 @@
codePointCount++;
prevHighSurrogate = false;
} else {
- throw new IllegalArgumentException(LOW_SURROGATE_WITHOUT_HIGH_SURROGATE);
+ throw HyracksDataException.create(INVALID_STRING_UNICODE, LOW_SURROGATE_WITHOUT_HIGH_SURROGATE);
}
} else {
codePointCount++;
@@ -345,11 +350,12 @@
return -1;
}
- public boolean contains(UTF8StringPointable pattern, boolean ignoreCase) {
+ public boolean contains(UTF8StringPointable pattern, boolean ignoreCase) throws HyracksDataException {
return contains(this, pattern, ignoreCase);
}
- public static boolean contains(UTF8StringPointable src, UTF8StringPointable pattern, boolean ignoreCase) {
+ public static boolean contains(UTF8StringPointable src, UTF8StringPointable pattern, boolean ignoreCase)
+ throws HyracksDataException {
return find(src, pattern, ignoreCase) >= 0;
}
@@ -678,9 +684,9 @@
endIndex = startIndex;
int cursorIndex = startIndex;
while (cursorIndex < srcUtfLen) {
- int codePioint = srcPtr.codePointAt(srcStart + cursorIndex);
+ int codePoint = srcPtr.codePointAt(srcStart + cursorIndex);
cursorIndex += srcPtr.codePointSize(srcStart + cursorIndex);
- if (!codePointSet.contains(codePioint)) {
+ if (!codePointSet.contains(codePoint)) {
endIndex = cursorIndex;
}
}
@@ -739,9 +745,9 @@
cursorIndex--;
if (UTF8StringUtil.isCharStart(srcPtr.bytes, cursorIndex)) {
ch = UTF8StringUtil.charAt(srcPtr.bytes, cursorIndex);
- if (Character.isHighSurrogate(ch) == false) {
- throw new IllegalArgumentException(
- "Decoding Error: no corresponding high surrogate found for the following low surrogate");
+ if (!Character.isHighSurrogate(ch)) {
+ throw HyracksDataException.create(INVALID_STRING_UNICODE,
+ LOW_SURROGATE_WITHOUT_HIGH_SURROGATE);
}
charSize += UTF8StringUtil.charSize(srcPtr.bytes, cursorIndex);
@@ -749,7 +755,7 @@
}
}
} else if (Character.isHighSurrogate(ch)) {
- throw new IllegalArgumentException("Decoding Error: get a high surrogate without low surrogate");
+ throw HyracksDataException.create(INVALID_STRING_UNICODE, HIGH_SURROGATE_WITHOUT_LOW_SURROGATE);
}
builder.appendUtf8StringPointable(srcPtr, cursorIndex, charSize);
diff --git a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/test/java/org/apache/hyracks/data/std/primitive/UTF8StringPointableTest.java b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/test/java/org/apache/hyracks/data/std/primitive/UTF8StringPointableTest.java
index f088c7e..45a5ba3 100644
--- a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/test/java/org/apache/hyracks/data/std/primitive/UTF8StringPointableTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/test/java/org/apache/hyracks/data/std/primitive/UTF8StringPointableTest.java
@@ -30,6 +30,7 @@
import java.io.IOException;
import java.util.Arrays;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.util.GrowableArray;
import org.apache.hyracks.data.std.util.UTF8StringBuilder;
import org.apache.hyracks.util.string.UTF8StringSample;
@@ -68,7 +69,7 @@
}
@Test
- public void testFindInCodePoint() {
+ public void testFindInCodePoint() throws HyracksDataException {
UTF8StringPointable strp = generateUTF8Pointable(STRING_EMOJI_FAMILY_OF_4 + EMOJI_BASKETBALL);
UTF8StringPointable pattern = generateUTF8Pointable(EMOJI_BASKETBALL);
diff --git a/hyracks-fullstack/hyracks/hyracks-data/pom.xml b/hyracks-fullstack/hyracks/hyracks-data/pom.xml
index 3d6f62f..b5f04bb 100644
--- a/hyracks-fullstack/hyracks/hyracks-data/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-data/pom.xml
@@ -26,7 +26,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/pom.xml b/hyracks-fullstack/hyracks/hyracks-dataflow-common/pom.xml
index 274cad7..6bb86ef 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/pom.xml
@@ -24,7 +24,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/io/MessagingFrameTupleAppender.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/io/MessagingFrameTupleAppender.java
index c27a7e6..6c073f3 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/io/MessagingFrameTupleAppender.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/io/MessagingFrameTupleAppender.java
@@ -172,13 +172,4 @@
++tupleCount;
IntSerDeUtils.putInt(getBuffer().array(), FrameHelper.getTupleCountOffset(frame.getFrameSize()), tupleCount);
}
-
- /*
- * Always write and then flush to send out the message if exists
- */
- @Override
- public void flush(IFrameWriter writer) throws HyracksDataException {
- write(writer, true);
- writer.flush();
- }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/io/RunFileReader.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/io/RunFileReader.java
index 50cacb2..7051e1d 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/io/RunFileReader.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/io/RunFileReader.java
@@ -115,6 +115,10 @@
return size;
}
+ public FileReference getFile() {
+ return file;
+ }
+
public void setDeleteAfterClose(boolean deleteAfterClose) {
this.deleteAfterClose = deleteAfterClose;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/io/RunFileWriter.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/io/RunFileWriter.java
index c370b58..26b9f10 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/io/RunFileWriter.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/io/RunFileWriter.java
@@ -69,7 +69,7 @@
@Override
public void close() throws HyracksDataException {
- if (!failed) {
+ if (!failed && handle != null) {
ioManager.close(handle);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/utils/TaskUtil.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/utils/TaskUtil.java
index 75c95b0..b77883d 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/utils/TaskUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/utils/TaskUtil.java
@@ -74,4 +74,18 @@
Map<String, Object> sharedMap = TaskUtil.getSharedMap(ctx, false);
return sharedMap == null ? null : (T) sharedMap.get(key);
}
+
+ /**
+ * get a <T> object from the shared map of the task, or returns the default value
+ *
+ * @param key
+ * @param ctx
+ * @param defaultValue
+ * @return the value associated with the key casted as T
+ */
+ @SuppressWarnings("unchecked")
+ public static <T> T getOrDefault(String key, IHyracksTaskContext ctx, T defaultValue) {
+ Map<String, T> sharedMap = (Map<String, T>) TaskUtil.getSharedMap(ctx, false);
+ return sharedMap == null ? defaultValue : sharedMap.getOrDefault(key, defaultValue);
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/pom.xml b/hyracks-fullstack/hyracks/hyracks-dataflow-std/pom.xml
index 284aa0d..8ac7a12 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/pom.xml
@@ -25,7 +25,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
index 555e8fb..0f31491 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
@@ -319,9 +319,6 @@
if (!failed) {
state.hybridHJ.closeBuild();
ctx.setStateObject(state);
- if (LOGGER.isTraceEnabled()) {
- LOGGER.trace("OptimizedHybridHashJoin closed its build phase");
- }
} else {
state.hybridHJ.clearBuildTempFiles();
}
@@ -402,10 +399,6 @@
writer.open();
state.hybridHJ.initProbe(probComp);
-
- if (LOGGER.isDebugEnabled()) {
- LOGGER.debug("OptimizedHybridHashJoin is starting the probe phase.");
- }
}
@Override
@@ -416,7 +409,7 @@
@Override
public void fail() throws HyracksDataException {
failed = true;
- if (state.hybridHJ != null) {
+ if (state != null && state.hybridHJ != null) {
state.hybridHJ.fail();
}
writer.fail();
@@ -427,12 +420,13 @@
if (failed) {
try {
// Clear temp files if fail() was called.
- state.hybridHJ.clearBuildTempFiles();
- state.hybridHJ.clearProbeTempFiles();
+ if (state != null && state.hybridHJ != null) {
+ state.hybridHJ.clearBuildTempFiles();
+ state.hybridHJ.clearProbeTempFiles();
+ }
} finally {
writer.close(); // writer should always be closed.
}
- logProbeComplete();
return;
}
try {
@@ -477,17 +471,7 @@
// Re-throw the whatever is caught.
throw e;
} finally {
- try {
- logProbeComplete();
- } finally {
- writer.close();
- }
- }
- }
-
- private void logProbeComplete() {
- if (LOGGER.isDebugEnabled()) {
- LOGGER.debug("OptimizedHybridHashJoin closed its probe phase");
+ writer.close();
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/TupleSorterHeapSort.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/TupleSorterHeapSort.java
index 08f15b3..a1704ec 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/TupleSorterHeapSort.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/TupleSorterHeapSort.java
@@ -41,13 +41,9 @@
import org.apache.hyracks.dataflow.std.structures.IResetableComparableFactory;
import org.apache.hyracks.dataflow.std.structures.MaxHeap;
import org.apache.hyracks.dataflow.std.structures.TuplePointer;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
public class TupleSorterHeapSort implements ITupleSorter {
- private static final Logger LOGGER = LogManager.getLogger();
-
class HeapEntryFactory implements IResetableComparableFactory<HeapEntry> {
@Override
public IResetableComparable<HeapEntry> createResetableComparable() {
@@ -288,7 +284,6 @@
int maxFrameSize = outputFrame.getFrameSize();
int numEntries = heap.getNumEntries();
IResetableComparable[] entries = heap.getEntries();
- int io = 0;
for (int i = 0; i < numEntries; i++) {
HeapEntry minEntry = (HeapEntry) entries[i];
bufferAccessor1.reset(minEntry.tuplePointer);
@@ -296,14 +291,10 @@
bufferAccessor1.getTupleStartOffset(), bufferAccessor1.getTupleLength());
if (flushed > 0) {
maxFrameSize = Math.max(maxFrameSize, flushed);
- io++;
}
}
maxFrameSize = Math.max(maxFrameSize, outputFrame.getFrameSize());
outputAppender.write(writer, true);
- if (LOGGER.isInfoEnabled()) {
- LOGGER.info("Flushed records:" + numEntries + "; Flushed through " + (io + 1) + " frames");
- }
return maxFrameSize;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-dist/pom.xml b/hyracks-fullstack/hyracks/hyracks-dist/pom.xml
index efed967..90f3c15 100644
--- a/hyracks-fullstack/hyracks/hyracks-dist/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-dist/pom.xml
@@ -22,7 +22,7 @@
<parent>
<artifactId>hyracks</artifactId>
<groupId>org.apache.hyracks</groupId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/hyracks-fullstack/hyracks/hyracks-documentation/pom.xml b/hyracks-fullstack/hyracks/hyracks-documentation/pom.xml
index 624639c..03aba8f 100644
--- a/hyracks-fullstack/hyracks/hyracks-documentation/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-documentation/pom.xml
@@ -25,7 +25,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<properties>
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeclient/pom.xml b/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeclient/pom.xml
index 81a3dfc..5940c73 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeclient/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeclient/pom.xml
@@ -25,7 +25,7 @@
<parent>
<groupId>org.apache.hyracks.examples</groupId>
<artifactId>btree-example</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<properties>
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreehelper/pom.xml b/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreehelper/pom.xml
index 7735cd7..42ed74a 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreehelper/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreehelper/pom.xml
@@ -25,7 +25,7 @@
<parent>
<groupId>org.apache.hyracks.examples</groupId>
<artifactId>btree-example</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<properties>
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeserver/pom.xml b/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeserver/pom.xml
index 75f4efe..c14a0e4 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeserver/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeserver/pom.xml
@@ -26,7 +26,7 @@
<parent>
<groupId>org.apache.hyracks.examples</groupId>
<artifactId>btree-example</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<properties>
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/btree-example/pom.xml b/hyracks-fullstack/hyracks/hyracks-examples/btree-example/pom.xml
index 942f980..d438b5c 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/btree-example/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-examples/btree-example/pom.xml
@@ -27,7 +27,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks-examples</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<properties>
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 f2eb1e1..90ea341 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/pom.xml
@@ -25,7 +25,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks-examples</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<build>
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeOperatorTestHelper.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeOperatorTestHelper.java
index bcd71a3..c3bc5ca 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeOperatorTestHelper.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeOperatorTestHelper.java
@@ -49,6 +49,6 @@
getVirtualBufferCacheProvider(), SynchronousSchedulerProvider.INSTANCE, MERGE_POLICY_FACTORY,
MERGE_POLICY_PROPERTIES, DURABLE, bloomFilterKeyFields,
LSMTreeOperatorTestHelper.DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE, true, btreefields,
- NoOpCompressorDecompressorFactory.INSTANCE, bloomFilterKeyFields != null, null, null);
+ NoOpCompressorDecompressorFactory.INSTANCE, bloomFilterKeyFields != null, null, null, false);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/AbstractMultiNCIntegrationTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/AbstractMultiNCIntegrationTest.java
index be22b9c..7a75a0f 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/AbstractMultiNCIntegrationTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/AbstractMultiNCIntegrationTest.java
@@ -34,7 +34,9 @@
import org.apache.hyracks.api.job.JobId;
import org.apache.hyracks.api.job.JobSpecification;
import org.apache.hyracks.api.job.JobStatus;
+import org.apache.hyracks.api.job.resource.ClusterCapacity;
import org.apache.hyracks.api.job.resource.IJobCapacityController;
+import org.apache.hyracks.api.job.resource.IReadOnlyClusterCapacity;
import org.apache.hyracks.api.result.IResultSet;
import org.apache.hyracks.api.result.IResultSetReader;
import org.apache.hyracks.client.result.ResultSet;
@@ -254,6 +256,14 @@
public void release(JobSpecification job) {
}
+
+ @Override
+ public IReadOnlyClusterCapacity getClusterCapacity() {
+ ClusterCapacity clusterCapacity = new ClusterCapacity();
+ clusterCapacity.setAggregatedMemoryByteSize(maxRAM);
+ clusterCapacity.setAggregatedCores(Integer.MAX_VALUE);
+ return clusterCapacity;
+ }
};
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TestJobLifecycleListener.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TestJobLifecycleListener.java
index 008be29..19fdcfe 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TestJobLifecycleListener.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TestJobLifecycleListener.java
@@ -30,6 +30,7 @@
import org.apache.hyracks.api.job.JobId;
import org.apache.hyracks.api.job.JobSpecification;
import org.apache.hyracks.api.job.JobStatus;
+import org.apache.hyracks.api.job.resource.IJobCapacityController;
import org.apache.logging.log4j.Level;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
@@ -47,7 +48,8 @@
private final Set<JobId> finishWithoutStart = new HashSet<>();
@Override
- public void notifyJobCreation(JobId jobId, JobSpecification spec) throws HyracksException {
+ public void notifyJobCreation(JobId jobId, JobSpecification spec, IJobCapacityController.JobSubmissionStatus status)
+ throws HyracksException {
if (created.containsKey(jobId)) {
LOGGER.log(Level.WARN, "Job " + jobId + "has been created before");
increment(doubleCreated, jobId);
@@ -62,7 +64,7 @@
}
@Override
- public void notifyJobStart(JobId jobId) throws HyracksException {
+ public void notifyJobStart(JobId jobId, JobSpecification spec) throws HyracksException {
if (!created.containsKey(jobId)) {
LOGGER.log(Level.WARN, "Job " + jobId + "has not been created");
startWithoutCreate.add(jobId);
@@ -75,7 +77,8 @@
}
@Override
- public void notifyJobFinish(JobId jobId, JobStatus jobStatus, List<Exception> exceptions) throws HyracksException {
+ public void notifyJobFinish(JobId jobId, JobSpecification spec, JobStatus jobStatus, List<Exception> exceptions)
+ throws HyracksException {
if (!started.contains(jobId)) {
LOGGER.log(Level.WARN, "Job " + jobId + "has not been started");
finishWithoutStart.add(jobId);
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-shutdown-test/pom.xml b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-shutdown-test/pom.xml
index d2242c1..7117fee 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-shutdown-test/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-shutdown-test/pom.xml
@@ -26,7 +26,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks-examples</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<properties>
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/pom.xml b/hyracks-fullstack/hyracks/hyracks-examples/pom.xml
index 3e3bfe9..8c86526 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-examples/pom.xml
@@ -26,7 +26,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/text-example/pom.xml b/hyracks-fullstack/hyracks/hyracks-examples/text-example/pom.xml
index 0135b1a..19fabdc 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/text-example/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-examples/text-example/pom.xml
@@ -26,7 +26,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks-examples</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<properties>
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/text-example/textclient/pom.xml b/hyracks-fullstack/hyracks/hyracks-examples/text-example/textclient/pom.xml
index 57ba143..353b5ef 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/text-example/textclient/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-examples/text-example/textclient/pom.xml
@@ -25,7 +25,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>text-example</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<properties>
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/text-example/texthelper/pom.xml b/hyracks-fullstack/hyracks/hyracks-examples/text-example/texthelper/pom.xml
index 223e408..9ddddec 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/text-example/texthelper/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-examples/text-example/texthelper/pom.xml
@@ -25,7 +25,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>text-example</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<properties>
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/text-example/textserver/pom.xml b/hyracks-fullstack/hyracks/hyracks-examples/text-example/textserver/pom.xml
index 922dfa2..39d6ae8 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/text-example/textserver/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-examples/text-example/textserver/pom.xml
@@ -26,7 +26,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>text-example</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<properties>
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/tpch-example/pom.xml b/hyracks-fullstack/hyracks/hyracks-examples/tpch-example/pom.xml
index dadaef7..f27fca2 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/tpch-example/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-examples/tpch-example/pom.xml
@@ -26,7 +26,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks-examples</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<properties>
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/tpch-example/tpchclient/pom.xml b/hyracks-fullstack/hyracks/hyracks-examples/tpch-example/tpchclient/pom.xml
index 77eedd0..8c568de 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/tpch-example/tpchclient/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-examples/tpch-example/tpchclient/pom.xml
@@ -24,7 +24,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>tpch-example</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<properties>
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/tpch-example/tpchserver/pom.xml b/hyracks-fullstack/hyracks/hyracks-examples/tpch-example/tpchserver/pom.xml
index c408096..7b49f4a 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/tpch-example/tpchserver/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-examples/tpch-example/tpchserver/pom.xml
@@ -26,7 +26,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>tpch-example</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<properties>
diff --git a/hyracks-fullstack/hyracks/hyracks-hdfs/pom.xml b/hyracks-fullstack/hyracks/hyracks-hdfs/pom.xml
index 2010653..237d799 100644
--- a/hyracks-fullstack/hyracks/hyracks-hdfs/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-hdfs/pom.xml
@@ -25,7 +25,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<properties>
@@ -112,8 +112,12 @@
<artifactId>servlet-api</artifactId>
</exclusion>
<exclusion>
- <groupId>log4j</groupId>
- <artifactId>log4j</artifactId>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-reload4j</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>ch.qos.reload4j</groupId>
+ <artifactId>reload4j</artifactId>
</exclusion>
</exclusions>
</dependency>
@@ -126,16 +130,12 @@
<artifactId>jersey-test-framework-grizzly2</artifactId>
</exclusion>
<exclusion>
- <groupId>log4j</groupId>
- <artifactId>log4j</artifactId>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-reload4j</artifactId>
</exclusion>
<exclusion>
- <groupId>org.slf4j</groupId>
- <artifactId>slf4j-api</artifactId>
- </exclusion>
- <exclusion>
- <groupId>org.slf4j</groupId>
- <artifactId>slf4j-log4j12</artifactId>
+ <groupId>ch.qos.reload4j</groupId>
+ <artifactId>reload4j</artifactId>
</exclusion>
</exclusions>
</dependency>
@@ -152,16 +152,12 @@
<artifactId>netty-all</artifactId>
</exclusion>
<exclusion>
- <groupId>log4j</groupId>
- <artifactId>log4j</artifactId>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-reload4j</artifactId>
</exclusion>
<exclusion>
- <groupId>org.slf4j</groupId>
- <artifactId>slf4j-api</artifactId>
- </exclusion>
- <exclusion>
- <groupId>org.slf4j</groupId>
- <artifactId>slf4j-log4j12</artifactId>
+ <groupId>ch.qos.reload4j</groupId>
+ <artifactId>reload4j</artifactId>
</exclusion>
</exclusions>
</dependency>
@@ -182,12 +178,8 @@
<artifactId>log4j</artifactId>
</exclusion>
<exclusion>
- <groupId>org.slf4j</groupId>
- <artifactId>slf4j-api</artifactId>
- </exclusion>
- <exclusion>
- <groupId>org.slf4j</groupId>
- <artifactId>slf4j-log4j12</artifactId>
+ <groupId>ch.qos.reload4j</groupId>
+ <artifactId>reload4j</artifactId>
</exclusion>
</exclusions>
</dependency>
diff --git a/hyracks-fullstack/hyracks/hyracks-http/pom.xml b/hyracks-fullstack/hyracks/hyracks-http/pom.xml
index e06f6be..e0c8efe 100644
--- a/hyracks-fullstack/hyracks/hyracks-http/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-http/pom.xml
@@ -22,7 +22,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<artifactId>hyracks-http</artifactId>
<properties>
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/ChunkedNettyOutputStream.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/ChunkedNettyOutputStream.java
index 9a940b1..b313cbb 100644
--- a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/ChunkedNettyOutputStream.java
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/ChunkedNettyOutputStream.java
@@ -21,6 +21,7 @@
import java.io.IOException;
import java.io.OutputStream;
+import org.apache.hyracks.api.util.InvokeUtil;
import org.apache.logging.log4j.Level;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
@@ -81,20 +82,22 @@
@Override
public void close() throws IOException {
if (!closed) {
- if (response.isHeaderSent() || response.status() != HttpResponseStatus.OK) {
- try {
+ InvokeUtil.tryIoWithCleanups(() -> {
+ if (response.isHeaderSent() || response.status() != HttpResponseStatus.OK) {
flush();
- } finally {
- if (buffer != null) {
- buffer.release();
- }
+ } else {
+ response.fullResponse(buffer);
+ // The responsibility of releasing the buffer is now with the netty pipeline since it is
+ // forwarded within the http content. We must nullify buffer to avoid releasing the buffer twice.
+ buffer = null;
}
- } else {
- response.fullResponse(buffer);
- }
- super.close();
+ super.close();
+ }, () -> {
+ if (buffer != null) {
+ buffer.release();
+ }
+ }, () -> closed = true);
}
- closed = true;
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/ChunkedResponse.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/ChunkedResponse.java
index e00c519..740af2f 100644
--- a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/ChunkedResponse.java
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/ChunkedResponse.java
@@ -25,6 +25,8 @@
import java.nio.charset.Charset;
import java.nio.charset.StandardCharsets;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.util.InvokeUtil;
import org.apache.hyracks.http.api.IServletResponse;
import org.apache.hyracks.http.server.utils.HttpUtil;
import org.apache.logging.log4j.Level;
@@ -45,6 +47,7 @@
import io.netty.handler.codec.http.HttpResponseStatus;
import io.netty.handler.codec.http.HttpVersion;
import io.netty.handler.codec.http.LastHttpContent;
+import io.netty.util.ReferenceCountUtil;
/**
* A chunked http response. Here is how it is expected to work:
@@ -114,28 +117,40 @@
@Override
public void close() throws IOException {
- if (writer != null) {
- writer.close();
- } else {
- outputStream.close();
- }
- if (errorBuf == null && response.status() == HttpResponseStatus.OK) {
- if (!done) {
- respond(LastHttpContent.EMPTY_LAST_CONTENT);
- }
- } else {
- // There was an error
- if (headerSent) {
- LOGGER.log(Level.WARN, "Error after header write of chunked response");
- if (errorBuf != null) {
- errorBuf.release();
+ try {
+ InvokeUtil.tryIoWithCleanups(() -> {
+ if (writer != null) {
+ writer.close();
+ } else {
+ outputStream.close();
}
- future = ctx.channel().close().addListener(handler);
- } else {
- // we didn't send anything to the user, we need to send an non-chunked error response
- fullResponse(response.protocolVersion(), response.status(),
- errorBuf == null ? ctx.alloc().buffer(0, 0) : errorBuf, response.headers());
- }
+ if (errorBuf == null && response.status() == HttpResponseStatus.OK) {
+ if (!done) {
+ respond(LastHttpContent.EMPTY_LAST_CONTENT);
+ }
+ } else {
+ // There was an error
+ if (headerSent) {
+ LOGGER.log(Level.WARN, "Error after header write of chunked response");
+ future = ctx.channel().close().addListener(handler);
+ } else {
+ // we didn't send anything to the user, we need to send an non-chunked error response
+ fullResponse(response.protocolVersion(), response.status(),
+ errorBuf == null ? ctx.alloc().buffer(0, 0) : errorBuf, response.headers());
+ // The responsibility of releasing the error buffer is now with the netty pipeline since it is
+ // forwarded within the http content. We must nullify buffer to avoid releasing the buffer twice.
+ errorBuf = null;
+ }
+ }
+ }, outputStream::close, () -> {
+ ReferenceCountUtil.release(errorBuf);
+ // We must nullify buffer to avoid releasing the buffer twice in case of duplicate close()
+ errorBuf = null;
+ });
+ } catch (IOException e) {
+ throw e;
+ } catch (Exception e) {
+ throw HyracksDataException.create(e);
}
done = true;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-ipc/pom.xml b/hyracks-fullstack/hyracks/hyracks-ipc/pom.xml
index caf3f8f..92a75e7 100644
--- a/hyracks-fullstack/hyracks/hyracks-ipc/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-ipc/pom.xml
@@ -24,7 +24,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/HyracksClientInterfaceRemoteProxy.java b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/HyracksClientInterfaceRemoteProxy.java
index 82d7a3b..3b59a70 100644
--- a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/HyracksClientInterfaceRemoteProxy.java
+++ b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/HyracksClientInterfaceRemoteProxy.java
@@ -35,6 +35,7 @@
import org.apache.hyracks.api.job.JobId;
import org.apache.hyracks.api.job.JobInfo;
import org.apache.hyracks.api.job.JobStatus;
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
import org.apache.hyracks.api.topology.ClusterTopology;
import org.apache.hyracks.ipc.api.IIPCHandle;
import org.apache.hyracks.ipc.api.RPCInterface;
@@ -130,6 +131,13 @@
}
@Override
+ public List<IOperatorStats> waitForCompletion(JobId jobId, List<String> statOperatorNames) throws Exception {
+ HyracksClientInterfaceFunctions.WaitForCompletionFunction wfcf =
+ new HyracksClientInterfaceFunctions.WaitForCompletionFunction(jobId, statOperatorNames);
+ return (List<IOperatorStats>) rpci.call(ipcHandle, wfcf);
+ }
+
+ @Override
public Map<String, NodeControllerInfo> getNodeControllersInfo() throws Exception {
HyracksClientInterfaceFunctions.GetNodeControllersInfoFunction gncif =
new HyracksClientInterfaceFunctions.GetNodeControllersInfoFunction();
diff --git a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/HyracksConnection.java b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/HyracksConnection.java
index 9351348..1b6d8af 100644
--- a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/HyracksConnection.java
+++ b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/HyracksConnection.java
@@ -18,6 +18,7 @@
*/
package org.apache.hyracks.ipc.impl;
+import java.io.Closeable;
import java.io.File;
import java.net.InetSocketAddress;
import java.net.URL;
@@ -52,6 +53,7 @@
import org.apache.hyracks.api.job.JobInfo;
import org.apache.hyracks.api.job.JobSpecification;
import org.apache.hyracks.api.job.JobStatus;
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
import org.apache.hyracks.api.network.ISocketChannelFactory;
import org.apache.hyracks.api.topology.ClusterTopology;
import org.apache.hyracks.api.util.InvokeUtil;
@@ -70,7 +72,7 @@
*
* @author vinayakb
*/
-public final class HyracksConnection implements IHyracksClientConnection {
+public final class HyracksConnection implements Closeable, IHyracksClientConnection {
private static final Logger LOGGER = LogManager.getLogger();
@@ -123,6 +125,11 @@
}
@Override
+ public void close() {
+ ipc.stop();
+ }
+
+ @Override
public JobStatus getJobStatus(JobId jobId) throws Exception {
return hci.getJobStatus(jobId);
}
@@ -196,6 +203,17 @@
}
@Override
+ public List<IOperatorStats> waitForCompletion(JobId jobId, List<String> statOperatorNames) throws Exception {
+ try {
+ return hci.waitForCompletion(jobId, statOperatorNames);
+ } catch (InterruptedException e) {
+ // Cancels an on-going job if the current thread gets interrupted.
+ cancelJob(jobId);
+ throw e;
+ }
+ }
+
+ @Override
public Map<String, NodeControllerInfo> getNodeControllerInfos() throws HyracksException {
try {
return hci.getNodeControllersInfo();
diff --git a/hyracks-fullstack/hyracks/hyracks-maven-plugins/hyracks-virtualcluster-maven-plugin/pom.xml b/hyracks-fullstack/hyracks/hyracks-maven-plugins/hyracks-virtualcluster-maven-plugin/pom.xml
index fc9d0fd..13db6f1 100644
--- a/hyracks-fullstack/hyracks/hyracks-maven-plugins/hyracks-virtualcluster-maven-plugin/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-maven-plugins/hyracks-virtualcluster-maven-plugin/pom.xml
@@ -26,7 +26,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks-maven-plugins</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<properties>
diff --git a/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/pom.xml b/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/pom.xml
index 16bcc7c..4ff5809 100644
--- a/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/pom.xml
@@ -21,7 +21,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks-maven-plugins</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<modelVersion>4.0.0</modelVersion>
<packaging>maven-plugin</packaging>
diff --git a/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/GenerateFileMojo.java b/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/GenerateFileMojo.java
index 297160e..5c11106 100644
--- a/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/GenerateFileMojo.java
+++ b/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/GenerateFileMojo.java
@@ -124,6 +124,9 @@
private boolean stripFoundationAssertionFromNotices = false;
@Parameter
+ private boolean includeShadowedDependencies = true;
+
+ @Parameter
private boolean validateShadowLicenses = false;
private SortedMap<String, SortedSet<Project>> noticeMap;
@@ -529,6 +532,11 @@
private void gatherShadowedDependencies(Map<MavenProject, List<Pair<String, String>>> dependencyLicenseMap,
Map<String, MavenProject> dependencyGavMap) throws MojoExecutionException, ProjectBuildingException {
+ if (!includeShadowedDependencies) {
+ getLog().info("Not gathering shadowed dependencies as 'includeShadowedDependencies' is set to "
+ + includeShadowedDependencies);
+ return;
+ }
Set<MavenProject> projects = new TreeSet<>(Comparator.comparing(MavenProject::getId));
projects.addAll(dependencyLicenseMap.keySet());
for (MavenProject p : projects) {
diff --git a/hyracks-fullstack/hyracks/hyracks-maven-plugins/pom.xml b/hyracks-fullstack/hyracks/hyracks-maven-plugins/pom.xml
index 373aca3..407d0de 100644
--- a/hyracks-fullstack/hyracks/hyracks-maven-plugins/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-maven-plugins/pom.xml
@@ -26,7 +26,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/hyracks-fullstack/hyracks/hyracks-net/pom.xml b/hyracks-fullstack/hyracks/hyracks-net/pom.xml
index 506eae7..f1cdbf8 100644
--- a/hyracks-fullstack/hyracks/hyracks-net/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-net/pom.xml
@@ -24,7 +24,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/hyracks-fullstack/hyracks/hyracks-server/pom.xml b/hyracks-fullstack/hyracks/hyracks-server/pom.xml
index 12a723f..baa8e3a 100644
--- a/hyracks-fullstack/hyracks/hyracks-server/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-server/pom.xml
@@ -24,7 +24,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-bloomfilter/pom.xml b/hyracks-fullstack/hyracks/hyracks-storage-am-bloomfilter/pom.xml
index 29f1dbf..838fecd 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-bloomfilter/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-bloomfilter/pom.xml
@@ -24,7 +24,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/pom.xml b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/pom.xml
index 88b84aa..148694d 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/pom.xml
@@ -24,7 +24,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<licenses>
<license>
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java
index 78faaff..e085b5e 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java
@@ -105,7 +105,7 @@
RangePredicate diskOrderScanPred = new RangePredicate(null, null, true, true, ctx.getCmp(), ctx.getCmp());
int maxPageId = freePageManager.getMaxPageId(ctx.getMetaFrame());
int currentPageId = bulkloadLeafStart;
- ICachedPage page = bufferCache.pin(BufferedFileHandle.getDiskPageId(getFileId(), currentPageId), false);
+ final ICachedPage page = bufferCache.pin(BufferedFileHandle.getDiskPageId(getFileId(), currentPageId), false);
page.acquireReadLatch();
try {
cursor.setBufferCache(bufferCache);
@@ -116,7 +116,7 @@
ctx.getCursorInitialState().setSearchOperationCallback(ctx.getSearchCallback());
ctx.getCursorInitialState().setOriginialKeyComparator(ctx.getCmp());
cursor.open(ctx.getCursorInitialState(), diskOrderScanPred);
- } catch (Exception e) {
+ } catch (Throwable e) {
page.releaseReadLatch();
bufferCache.unpin(page);
throw HyracksDataException.create(e);
@@ -202,8 +202,7 @@
}
// we use this loop to deal with possibly multiple operation restarts
// due to ongoing structure modifications during the descent
- boolean repeatOp = true;
- while (repeatOp && ctx.getOpRestarts() < MAX_RESTARTS) {
+ while (true) {
performOp(rootPage, null, true, ctx);
// if we reach this stage then we need to restart from the (possibly
// new) root
@@ -211,7 +210,7 @@
ctx.getPageLsns().removeLast(); // pop the restart op indicator
continue;
}
- repeatOp = false;
+ break;
}
cursor.setBufferCache(bufferCache);
cursor.setFileId(getFileId());
@@ -231,12 +230,8 @@
bufferCache.unpin(smPage);
}
}
- if (ctx.getSmPages().size() > 0) {
- if (ctx.getSmoCount() == Integer.MAX_VALUE) {
- smoCounter.set(0);
- } else {
- smoCounter.incrementAndGet();
- }
+ if (!ctx.getSmPages().isEmpty()) {
+ smoCounter.updateAndGet(i -> i == Integer.MAX_VALUE ? 0 : i + 1);
treeLatch.writeLock().unlock();
ctx.getSmPages().clear();
}
@@ -599,8 +594,7 @@
// We use this loop to deal with possibly multiple operation
// restarts due to ongoing structure modifications during
// the descent.
- boolean repeatOp = true;
- while (repeatOp && ctx.getOpRestarts() < MAX_RESTARTS) {
+ while (true) {
int childPageId = ctx.getInteriorFrame().getChildPageId(ctx.getPred());
performOp(childPageId, node, isReadLatched, ctx);
node = null;
@@ -615,6 +609,10 @@
if (node != null) {
isReadLatched = true;
// Descend the tree again.
+ if (ctx.getOpRestarts() >= MAX_RESTARTS) {
+ throw HyracksDataException.create(ErrorCode.OPERATION_EXCEEDED_MAX_RESTARTS,
+ MAX_RESTARTS);
+ }
continue;
} else {
// Pop pageLsn of this page (version seen by this op during descent).
@@ -662,7 +660,7 @@
}
}
// Operation completed.
- repeatOp = false;
+ break;
} // end while
} else { // smFlag
ctx.setOpRestarts(ctx.getOpRestarts() + 1);
@@ -734,21 +732,8 @@
ctx.getPageLsns().add(FULL_RESTART_OP);
}
}
- } catch (HyracksDataException e) {
- if (!ctx.isExceptionHandled()) {
- if (node != null) {
- if (isReadLatched) {
- node.releaseReadLatch();
- } else {
- node.releaseWriteLatch(true);
- }
- bufferCache.unpin(node);
- ctx.setExceptionHandled(true);
- }
- }
- throw e;
- } catch (Exception e) {
- if (node != null) {
+ } catch (Throwable e) {
+ if (!ctx.isExceptionHandled() && node != null) {
if (isReadLatched) {
node.releaseReadLatch();
} else {
@@ -756,9 +741,8 @@
}
bufferCache.unpin(node);
}
- HyracksDataException wrappedException = HyracksDataException.create(e);
ctx.setExceptionHandled(true);
- throw wrappedException;
+ throw HyracksDataException.create(e);
}
}
@@ -785,7 +769,7 @@
ICachedPage node = bufferCache.pin(BufferedFileHandle.getDiskPageId(getFileId(), pageId), false);
node.acquireReadLatch();
try {
- if (parent != null && unpin == true) {
+ if (parent != null && unpin) {
parent.releaseReadLatch();
bufferCache.unpin(parent);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeCountingSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeCountingSearchCursor.java
index 1491424..71df593 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeCountingSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeCountingSearchCursor.java
@@ -119,13 +119,21 @@
private void fetchNextLeafPage(int nextLeafPage) throws HyracksDataException {
do {
- ICachedPage nextLeaf = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, nextLeafPage), false);
- if (exclusiveLatchNodes) {
- nextLeaf.acquireWriteLatch();
- page.releaseWriteLatch(isPageDirty);
- } else {
- nextLeaf.acquireReadLatch();
- page.releaseReadLatch();
+ final ICachedPage nextLeaf;
+ try {
+ nextLeaf = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, nextLeafPage), false);
+ if (exclusiveLatchNodes) {
+ nextLeaf.acquireWriteLatch();
+ } else {
+ nextLeaf.acquireReadLatch();
+ }
+ } finally {
+ // release latches in finally, don't leak locks on pin failure
+ if (exclusiveLatchNodes) {
+ page.releaseWriteLatch(isPageDirty);
+ } else {
+ page.releaseReadLatch();
+ }
}
bufferCache.unpin(page);
page = nextLeaf;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeRangeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeRangeSearchCursor.java
index 9ccd881..cf3727a 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeRangeSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeRangeSearchCursor.java
@@ -110,8 +110,13 @@
protected void fetchNextLeafPage(int nextLeafPage) throws HyracksDataException {
do {
- ICachedPage nextLeaf = acquirePage(nextLeafPage);
- releasePage();
+ ICachedPage nextLeaf;
+ try {
+ nextLeaf = acquirePage(nextLeafPage);
+ } finally {
+ // release page in finally, don't leak lock on pin failure
+ releasePage();
+ }
page = nextLeaf;
isPageDirty = false;
frame.setPage(page);
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTree.java
index ae6bbaa..282aad9 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTree.java
@@ -143,19 +143,12 @@
ctx.getCursorInitialState().setPageId(childPageId);
ctx.getLeafFrame().setPage(currentPage);
cursor.open(ctx.getCursorInitialState(), ctx.getPred());
- } catch (HyracksDataException e) {
+ } catch (Exception e) {
if (!ctx.isExceptionHandled() && currentPage != null) {
bufferCache.unpin(currentPage);
- ctx.setExceptionHandled(true);
}
- throw e;
- } catch (Exception e) {
- if (currentPage != null) {
- bufferCache.unpin(currentPage);
- }
- HyracksDataException wrappedException = HyracksDataException.create(e);
ctx.setExceptionHandled(true);
- throw wrappedException;
+ throw HyracksDataException.create(e);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/pom.xml b/hyracks-fullstack/hyracks/hyracks-storage-am-common/pom.xml
index 82937cf..a84c25d 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/pom.xml
@@ -24,7 +24,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<licenses>
<license>
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/pom.xml b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/pom.xml
index deafec3..217253c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/pom.xml
@@ -23,7 +23,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<licenses>
<license>
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResource.java
index e16baf9..ee4c4f7 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResource.java
@@ -56,19 +56,21 @@
ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
IMetadataPageManagerFactory metadataPageManagerFactory,
ILSMIOOperationSchedulerProvider ioSchedulerProvider, boolean durable, boolean hasBloomFilter,
- ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
+ ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector,
+ boolean isSecondaryNoIncrementalMaintenance) {
super(typeTraits, cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate, isPrimary, path,
storageManager, mergePolicyFactory, mergePolicyProperties, filterTypeTraits, filterCmpFactories,
btreeFields, filterFields, opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory,
metadataPageManagerFactory, null, ioSchedulerProvider, durable,
- NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter, nullTypeTraits, nullIntrospector);
+ NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter, nullTypeTraits, nullIntrospector,
+ isSecondaryNoIncrementalMaintenance);
}
private ExternalBTreeLocalResource(IPersistedResourceRegistry registry, JsonNode json, int[] bloomFilterKeyFields,
- double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields, boolean hasBloomFilter)
- throws HyracksDataException {
+ double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields, boolean hasBloomFilter,
+ boolean isSecondaryNoIncrementalMaintenance) throws HyracksDataException {
super(registry, json, bloomFilterKeyFields, bloomFilterFalsePositiveRate, isPrimary, btreeFields,
- NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter);
+ NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter, isSecondaryNoIncrementalMaintenance);
}
@Override
@@ -97,6 +99,6 @@
LSMBTreeLocalResource lsmBtree = (LSMBTreeLocalResource) LSMBTreeLocalResource.fromJson(registry, json);
return new ExternalBTreeLocalResource(registry, json, lsmBtree.bloomFilterKeyFields,
lsmBtree.bloomFilterFalsePositiveRate, lsmBtree.isPrimary, lsmBtree.btreeFields,
- lsmBtree.hasBloomFilter);
+ lsmBtree.hasBloomFilter, lsmBtree.isSecondaryNoIncrementalMaintenance);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResourceFactory.java
index e4c29ba..3a28057 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResourceFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResourceFactory.java
@@ -46,13 +46,13 @@
IMetadataPageManagerFactory metadataPageManagerFactory,
ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMMergePolicyFactory mergePolicyFactory,
Map<String, String> mergePolicyProperties, boolean durable, int[] bloomFilterKeyFields,
- double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields, boolean hasBloomFilter,
- ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
+ double bloomFilterFalsePositiveRate, int[] btreeFields, boolean hasBloomFilter, ITypeTraits nullTypeTraits,
+ INullIntrospector nullIntrospector) {
super(storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
opTrackerFactory, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory, null,
ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, bloomFilterKeyFields,
- bloomFilterFalsePositiveRate, isPrimary, btreeFields, NoOpCompressorDecompressorFactory.INSTANCE,
- hasBloomFilter, nullTypeTraits, nullIntrospector);
+ bloomFilterFalsePositiveRate, false, btreeFields, NoOpCompressorDecompressorFactory.INSTANCE,
+ hasBloomFilter, nullTypeTraits, nullIntrospector, true);
}
@Override
@@ -61,6 +61,7 @@
bloomFilterFalsePositiveRate, isPrimary, fileRef.getRelativePath(), storageManager, mergePolicyFactory,
mergePolicyProperties, filterTypeTraits, filterCmpFactories, btreeFields, bloomFilterKeyFields,
opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
- ioSchedulerProvider, durable, hasBloomFilter, nullTypeTraits, nullIntrospector);
+ ioSchedulerProvider, durable, hasBloomFilter, nullTypeTraits, nullIntrospector,
+ isSecondaryNoIncrementalMaintenance);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResource.java
index 863ae56..f2c5263 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResource.java
@@ -59,19 +59,21 @@
ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
IMetadataPageManagerFactory metadataPageManagerFactory,
ILSMIOOperationSchedulerProvider ioSchedulerProvider, boolean durable, boolean hasBloomFilter,
- ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
+ ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector,
+ boolean isSecondaryNoIncrementalMaintenance) {
super(typeTraits, cmpFactories, buddyBtreeFields, bloomFilterFalsePositiveRate, isPrimary, path, storageManager,
mergePolicyFactory, mergePolicyProperties, filterTypeTraits, filterCmpFactories, btreeFields,
filterFields, opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory,
metadataPageManagerFactory, null, ioSchedulerProvider, durable,
- NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter, nullTypeTraits, nullIntrospector);
+ NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter, nullTypeTraits, nullIntrospector,
+ isSecondaryNoIncrementalMaintenance);
}
private ExternalBTreeWithBuddyLocalResource(IPersistedResourceRegistry registry, JsonNode json,
int[] bloomFilterKeyFields, double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields,
- boolean hasBloomFilter) throws HyracksDataException {
+ boolean hasBloomFilter, boolean isSecondaryNoIncrementalMaintenance) throws HyracksDataException {
super(registry, json, bloomFilterKeyFields, bloomFilterFalsePositiveRate, isPrimary, btreeFields,
- NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter);
+ NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter, isSecondaryNoIncrementalMaintenance);
}
@Override
@@ -100,6 +102,6 @@
LSMBTreeLocalResource lsmBtree = (LSMBTreeLocalResource) LSMBTreeLocalResource.fromJson(registry, json);
return new ExternalBTreeWithBuddyLocalResource(registry, json, lsmBtree.bloomFilterKeyFields,
lsmBtree.bloomFilterFalsePositiveRate, lsmBtree.isPrimary, lsmBtree.btreeFields,
- lsmBtree.hasBloomFilter);
+ lsmBtree.hasBloomFilter, lsmBtree.isSecondaryNoIncrementalMaintenance);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResourceFactory.java
index 39c8aed..995a23b 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResourceFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResourceFactory.java
@@ -46,13 +46,13 @@
IMetadataPageManagerFactory metadataPageManagerFactory,
ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMMergePolicyFactory mergePolicyFactory,
Map<String, String> mergePolicyProperties, boolean durable, int[] buddyBtreeFields,
- double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields, boolean hasBloomFilter,
- ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
+ double bloomFilterFalsePositiveRate, int[] btreeFields, boolean hasBloomFilter, ITypeTraits nullTypeTraits,
+ INullIntrospector nullIntrospector) {
super(storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory, null,
ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, buddyBtreeFields,
- bloomFilterFalsePositiveRate, isPrimary, btreeFields, NoOpCompressorDecompressorFactory.INSTANCE,
- hasBloomFilter, nullTypeTraits, nullIntrospector);
+ bloomFilterFalsePositiveRate, false, btreeFields, NoOpCompressorDecompressorFactory.INSTANCE,
+ hasBloomFilter, nullTypeTraits, nullIntrospector, true);
}
@Override
@@ -61,6 +61,7 @@
bloomFilterFalsePositiveRate, isPrimary, fileRef.getRelativePath(), storageManager, mergePolicyFactory,
mergePolicyProperties, filterTypeTraits, filterCmpFactories, btreeFields, filterFields,
opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
- ioSchedulerProvider, durable, hasBloomFilter, nullTypeTraits, nullIntrospector);
+ ioSchedulerProvider, durable, hasBloomFilter, nullTypeTraits, nullIntrospector,
+ isSecondaryNoIncrementalMaintenance);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResource.java
index 656ae02..a7e433c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResource.java
@@ -59,6 +59,7 @@
protected final boolean isPrimary;
protected final int[] btreeFields;
protected final ICompressorDecompressorFactory compressorDecompressorFactory;
+ protected final boolean isSecondaryNoIncrementalMaintenance;
public LSMBTreeLocalResource(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories,
int[] bloomFilterKeyFields, double bloomFilterFalsePositiveRate, boolean isPrimary, String path,
@@ -70,7 +71,8 @@
IMetadataPageManagerFactory metadataPageManagerFactory, IVirtualBufferCacheProvider vbcProvider,
ILSMIOOperationSchedulerProvider ioSchedulerProvider, boolean durable,
ICompressorDecompressorFactory compressorDecompressorFactory, boolean hasBloomFilter,
- ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
+ ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector,
+ boolean isSecondaryNoIncrementalMaintenance) {
super(path, storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, nullTypeTraits,
@@ -81,12 +83,13 @@
this.btreeFields = btreeFields;
this.compressorDecompressorFactory = compressorDecompressorFactory;
this.hasBloomFilter = hasBloomFilter;
+ this.isSecondaryNoIncrementalMaintenance = isSecondaryNoIncrementalMaintenance;
}
protected LSMBTreeLocalResource(IPersistedResourceRegistry registry, JsonNode json, int[] bloomFilterKeyFields,
double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields,
- ICompressorDecompressorFactory compressorDecompressorFactory, boolean hasBloomFilter)
- throws HyracksDataException {
+ ICompressorDecompressorFactory compressorDecompressorFactory, boolean hasBloomFilter,
+ boolean isSecondaryNoIncrementalMaintenance) throws HyracksDataException {
super(registry, json);
this.bloomFilterKeyFields = bloomFilterKeyFields;
this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
@@ -94,6 +97,7 @@
this.btreeFields = btreeFields;
this.compressorDecompressorFactory = compressorDecompressorFactory;
this.hasBloomFilter = hasBloomFilter;
+ this.isSecondaryNoIncrementalMaintenance = isSecondaryNoIncrementalMaintenance;
}
@Override
@@ -114,6 +118,10 @@
compressorDecompressorFactory, hasBloomFilter, nullTypeTraits, nullIntrospector);
}
+ public boolean isSecondaryNoIncrementalMaintenance() {
+ return isSecondaryNoIncrementalMaintenance;
+ }
+
@Override
public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
final ObjectNode jsonObject = registry.getClassIdentifier(getClass(), serialVersionUID);
@@ -131,8 +139,10 @@
final JsonNode compressorDecompressorNode = json.get("compressorDecompressorFactory");
final ICompressorDecompressorFactory compDecompFactory = (ICompressorDecompressorFactory) registry
.deserializeOrDefault(compressorDecompressorNode, NoOpCompressorDecompressorFactory.class);
+ boolean isSecondaryNoIncrementalMaintenance =
+ getOrDefaultBoolean(json, "isSecondaryNoIncrementalMaintenance", false);
return new LSMBTreeLocalResource(registry, json, bloomFilterKeyFields, bloomFilterFalsePositiveRate, isPrimary,
- btreeFields, compDecompFactory, hasBloomFilter);
+ btreeFields, compDecompFactory, hasBloomFilter, isSecondaryNoIncrementalMaintenance);
}
@Override
@@ -145,13 +155,16 @@
json.put("isPrimary", isPrimary);
json.putPOJO("btreeFields", btreeFields);
json.putPOJO("compressorDecompressorFactory", compressorDecompressorFactory.toJson(registry));
+ json.put("isSecondaryNoIncrementalMaintenance", isSecondaryNoIncrementalMaintenance);
}
private static boolean getOrDefaultHasBloomFilter(JsonNode json, boolean isPrimary) {
- if (json.has(HAS_BLOOM_FILTER_FIELD)) {
- return json.get(HAS_BLOOM_FILTER_FIELD).asBoolean();
- }
- // for backward compatibiliy, only primary indexes have bloom filters
- return isPrimary;
+ // for backward compatibility, only primary indexes have bloom filters
+ return getOrDefaultBoolean(json, HAS_BLOOM_FILTER_FIELD, isPrimary);
}
+
+ private static boolean getOrDefaultBoolean(JsonNode jsonNode, String fieldName, boolean defaultValue) {
+ return jsonNode.has(fieldName) ? jsonNode.get(fieldName).asBoolean() : defaultValue;
+ }
+
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResourceFactory.java
index e926a48..6695e90 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResourceFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResourceFactory.java
@@ -48,6 +48,7 @@
protected final boolean isPrimary;
protected final int[] btreeFields;
protected final ICompressorDecompressorFactory compressorDecompressorFactory;
+ protected final boolean isSecondaryNoIncrementalMaintenance;
public LSMBTreeLocalResourceFactory(IStorageManager storageManager, ITypeTraits[] typeTraits,
IBinaryComparatorFactory[] cmpFactories, ITypeTraits[] filterTypeTraits,
@@ -59,7 +60,8 @@
Map<String, String> mergePolicyProperties, boolean durable, int[] bloomFilterKeyFields,
double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields,
ICompressorDecompressorFactory compressorDecompressorFactory, boolean hasBloomFilter,
- ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
+ ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector,
+ boolean isSecondaryNoIncrementalMaintenance) {
super(storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
opTrackerFactory, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, nullTypeTraits,
@@ -70,6 +72,7 @@
this.isPrimary = isPrimary;
this.btreeFields = btreeFields;
this.compressorDecompressorFactory = compressorDecompressorFactory;
+ this.isSecondaryNoIncrementalMaintenance = isSecondaryNoIncrementalMaintenance;
}
@Override
@@ -78,7 +81,8 @@
isPrimary, fileRef.getRelativePath(), storageManager, mergePolicyFactory, mergePolicyProperties,
filterTypeTraits, filterCmpFactories, btreeFields, filterFields, opTrackerProvider, ioOpCallbackFactory,
pageWriteCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider, durable,
- compressorDecompressorFactory, hasBloomFilter, nullTypeTraits, nullIntrospector);
+ compressorDecompressorFactory, hasBloomFilter, nullTypeTraits, nullIntrospector,
+ isSecondaryNoIncrementalMaintenance);
}
private void readObject(java.io.ObjectInputStream in) throws IOException, ClassNotFoundException {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/pom.xml b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/pom.xml
index 7226f3b..83789e7 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/pom.xml
@@ -23,7 +23,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<licenses>
<license>
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IBatchController.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IBatchController.java
new file mode 100644
index 0000000..e061589
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IBatchController.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.lsm.common.api;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public interface IBatchController {
+ String KEY_BATCH_CONTROLLER = "BATCH_CONTROLLER";
+
+ void batchEnter(ILSMIndexOperationContext ctx, ILSMHarness lsmHarness, IFrameOperationCallback callback)
+ throws HyracksDataException;
+
+ void batchExit(ILSMIndexOperationContext ctx, ILSMHarness lsmHarness, IFrameOperationCallback callback,
+ boolean batchSuccessful) throws HyracksDataException;
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IFrameOperationCallback.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IFrameOperationCallback.java
index 1f89af2..2fbc0c5 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IFrameOperationCallback.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IFrameOperationCallback.java
@@ -28,14 +28,24 @@
public interface IFrameOperationCallback extends Closeable {
/**
* Called once processing the frame is done before calling nextFrame on the next IFrameWriter in
- * the pipeline
+ * the pipeline. In the event this frame completion will also exit the component, this will be
+ * called prior to {@link #beforeExit(boolean)}.
*
* @throws HyracksDataException
*/
void frameCompleted() throws HyracksDataException;
/**
- * Called when the task has failed.
+ * Called just prior to exiting the component on batch completion: not all batches may result
+ * in a component exit, depending on the decision of the {@link IBatchController}.
+ *
+ * @throws HyracksDataException
+ */
+ void beforeExit(boolean success) throws HyracksDataException;
+
+ /**
+ * Called when the batch processing, {@link #frameCompleted()} or {@link #beforeExit(boolean)}
+ * invocation has failed.
*
* @param th
*/
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMHarness.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMHarness.java
index 9e8c568..68de45a 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMHarness.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMHarness.java
@@ -39,7 +39,6 @@
* @param tuple
* the operation tuple
* @throws HyracksDataException
- * @throws IndexException
*/
void forceModify(ILSMIndexOperationContext ctx, ITupleReference tuple) throws HyracksDataException;
@@ -54,7 +53,6 @@
* the operation tuple
* @return
* @throws HyracksDataException
- * @throws IndexException
*/
boolean modify(ILSMIndexOperationContext ctx, boolean tryOperation, ITupleReference tuple)
throws HyracksDataException;
@@ -69,7 +67,6 @@
* @param pred
* the search predicate
* @throws HyracksDataException
- * @throws IndexException
*/
void search(ILSMIndexOperationContext ctx, IIndexCursor cursor, ISearchPredicate pred) throws HyracksDataException;
@@ -104,9 +101,7 @@
* Schedule a merge
*
* @param ctx
- * @param callback
* @throws HyracksDataException
- * @throws IndexException
*/
ILSMIOOperation scheduleMerge(ILSMIndexOperationContext ctx) throws HyracksDataException;
@@ -114,9 +109,7 @@
* Schedule full merge
*
* @param ctx
- * @param callback
* @throws HyracksDataException
- * @throws IndexException
*/
ILSMIOOperation scheduleFullMerge(ILSMIndexOperationContext ctx) throws HyracksDataException;
@@ -125,7 +118,6 @@
*
* @param operation
* @throws HyracksDataException
- * @throws IndexException
*/
void merge(ILSMIOOperation operation) throws HyracksDataException;
@@ -133,7 +125,6 @@
* Schedule a flush
*
* @param ctx
- * @param callback
* @throws HyracksDataException
*/
ILSMIOOperation scheduleFlush(ILSMIndexOperationContext ctx) throws HyracksDataException;
@@ -143,7 +134,6 @@
*
* @param operation
* @throws HyracksDataException
- * @throws IndexException
*/
void flush(ILSMIOOperation operation) throws HyracksDataException;
@@ -153,7 +143,6 @@
* @param ioOperation
* the io operation that added the new component
* @throws HyracksDataException
- * @throws IndexException
*/
void addBulkLoadedComponent(ILSMIOOperation ioOperation) throws HyracksDataException;
@@ -225,20 +214,22 @@
/**
* Perform batch operation on all tuples in the passed frame tuple accessor
*
- * @param ctx
- * the operation ctx
- * @param accessor
- * the frame tuple accessor
- * @param tuple
- * the mutable tuple used to pass the tuple to the processor
- * @param processor
- * the tuple processor
- * @param frameOpCallback
- * the callback at the end of the frame
+ * @param ctx the operation ctx
+ * @param accessor the frame tuple accessor
+ * @param tuple the mutable tuple used to pass the tuple to the processor
+ * @param processor the tuple processor
+ * @param frameOpCallback the callback at the end of the frame
+ * @param batchController
* @throws HyracksDataException
*/
void batchOperate(ILSMIndexOperationContext ctx, FrameTupleAccessor accessor, FrameTupleReference tuple,
- IFrameTupleProcessor processor, IFrameOperationCallback frameOpCallback) throws HyracksDataException;
+ IFrameTupleProcessor processor, IFrameOperationCallback frameOpCallback, IBatchController batchController)
+ throws HyracksDataException;
+
+ void enter(ILSMIndexOperationContext ctx, LSMOperationType opType) throws HyracksDataException;
+
+ void exit(ILSMIndexOperationContext ctx, IFrameOperationCallback callback, boolean success, LSMOperationType op)
+ throws HyracksDataException;
/**
* Rollback components that match the passed predicate
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/freepage/VirtualFreePageManager.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/freepage/VirtualFreePageManager.java
index 9d62c0d..0e4f835 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/freepage/VirtualFreePageManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/freepage/VirtualFreePageManager.java
@@ -104,10 +104,13 @@
page = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, currentPageId.get()), true);
if (leafFrameFactory != null) {
page.acquireWriteLatch();
- ITreeIndexFrame leafFrame = leafFrameFactory.createFrame();
- leafFrame.setPage(page);
- leafFrame.initBuffer((byte) 0);
- page.releaseWriteLatch(true);
+ try {
+ ITreeIndexFrame leafFrame = leafFrameFactory.createFrame();
+ leafFrame.setPage(page);
+ leafFrame.initBuffer((byte) 0);
+ } finally {
+ page.releaseWriteLatch(true);
+ }
}
bufferCache.unpin(page);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractAsynchronousScheduler.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractAsynchronousScheduler.java
index e266a6f..049da38 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractAsynchronousScheduler.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractAsynchronousScheduler.java
@@ -38,9 +38,10 @@
private final int maxNumFlushes;
protected final Map<String, ILSMIOOperation> runningFlushOperations = new HashMap<>();
+ protected final Map<String, ILSMIOOperation> runningReplicateOperations = new HashMap<>();
protected final Deque<ILSMIOOperation> waitingFlushOperations = new ArrayDeque<>();
protected final Deque<ILSMIOOperation> waitingMergeOperations = new ArrayDeque<>();
-
+ protected final Deque<ILSMIOOperation> waitingReplicateOperations = new ArrayDeque<>();
protected final Map<String, Throwable> failedGroups = new HashMap<>();
public AbstractAsynchronousScheduler(ThreadFactory threadFactory, final IIoOperationFailedCallback callback,
@@ -58,8 +59,11 @@
case MERGE:
scheduleMerge(operation);
break;
+ case REPLICATE:
+ scheduleReplicate(operation);
+ break;
case NOOP:
- return;
+ break;
default:
// this should never happen
// just guard here to avoid silent failures in case of future extensions
@@ -75,6 +79,10 @@
break;
case MERGE:
completeMerge(operation);
+ break;
+ case REPLICATE:
+ completeReplicate(operation);
+ break;
case NOOP:
return;
default:
@@ -149,6 +157,46 @@
}
}
+ private void scheduleReplicate(ILSMIOOperation operation) {
+ String id = operation.getIndexIdentifier();
+ synchronized (executor) {
+ if (runningReplicateOperations.size() >= maxNumFlushes || runningReplicateOperations.containsKey(id)) {
+ waitingReplicateOperations.add(operation);
+ } else {
+ runningReplicateOperations.put(id, operation);
+ executor.submit(operation);
+ }
+ }
+ }
+
+ private void completeReplicate(ILSMIOOperation operation) {
+ String id = operation.getIndexIdentifier();
+ synchronized (executor) {
+ runningReplicateOperations.remove(id);
+ // Schedule replicate in FIFO order. Must make sure that there is at most one scheduled replicate for each index.
+ for (ILSMIOOperation replicateOp : waitingReplicateOperations) {
+ String replicateOpId = replicateOp.getIndexIdentifier();
+ if (runningReplicateOperations.size() < maxNumFlushes) {
+ if (!runningReplicateOperations.containsKey(replicateOpId) && !replicateOp.isCompleted()) {
+ runningReplicateOperations.put(replicateOpId, replicateOp);
+ executor.submit(replicateOp);
+ }
+ } else {
+ break;
+ }
+ }
+ // cleanup scheduled replicate
+ while (!waitingReplicateOperations.isEmpty()) {
+ ILSMIOOperation top = waitingReplicateOperations.peek();
+ if (top.isCompleted() || runningReplicateOperations.get(top.getIndexIdentifier()) == top) {
+ waitingReplicateOperations.poll();
+ } else {
+ break;
+ }
+ }
+ }
+ }
+
@Override
public void close() throws IOException {
executor.shutdown();
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMHarness.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMHarness.java
index 950a8e5..017e767 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMHarness.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMHarness.java
@@ -33,6 +33,7 @@
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.IBatchController;
import org.apache.hyracks.storage.am.lsm.common.api.IFrameOperationCallback;
import org.apache.hyracks.storage.am.lsm.common.api.IFrameTupleProcessor;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
@@ -688,15 +689,27 @@
lsmIndex.updateFilter(ctx, tuple);
}
- private void enter(ILSMIndexOperationContext ctx) throws HyracksDataException {
+ @Override
+ public void enter(ILSMIndexOperationContext ctx, LSMOperationType op) throws HyracksDataException {
if (!lsmIndex.isMemoryComponentsAllocated()) {
lsmIndex.allocateMemoryComponents();
}
- getAndEnterComponents(ctx, LSMOperationType.MODIFICATION, false);
+ getAndEnterComponents(ctx, op, false);
}
- private void exit(ILSMIndexOperationContext ctx) throws HyracksDataException {
- getAndExitComponentsAndComplete(ctx, LSMOperationType.MODIFICATION);
+ @Override
+ public void exit(ILSMIndexOperationContext ctx, IFrameOperationCallback callback, boolean success,
+ LSMOperationType op) throws HyracksDataException {
+ try {
+ callback.beforeExit(success);
+ } catch (Throwable th) {
+ // TODO(mblow): we don't distinguish between the three distinct phases we can encounter
+ // failures in the callback API- we might need this eventually
+ callback.fail(th);
+ throw th;
+ } finally {
+ getAndExitComponentsAndComplete(ctx, op);
+ }
}
private void getAndExitComponentsAndComplete(ILSMIndexOperationContext ctx, LSMOperationType op)
@@ -711,12 +724,15 @@
@Override
public void batchOperate(ILSMIndexOperationContext ctx, FrameTupleAccessor accessor, FrameTupleReference tuple,
- IFrameTupleProcessor processor, IFrameOperationCallback frameOpCallback) throws HyracksDataException {
+ IFrameTupleProcessor processor, IFrameOperationCallback frameOpCallback, IBatchController batchController)
+ throws HyracksDataException {
processor.start();
- enter(ctx);
+ batchController.batchEnter(ctx, this, frameOpCallback);
+ boolean success = false;
try {
try {
processFrame(accessor, tuple, processor);
+ success = true;
frameOpCallback.frameCompleted();
} catch (Throwable th) {
processor.fail(th);
@@ -728,7 +744,7 @@
LOGGER.warn("Failed to process frame", e);
throw e;
} finally {
- exit(ctx);
+ batchController.batchExit(ctx, this, frameOpCallback, success);
ctx.logPerformanceCounters(accessor.getTupleCount());
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMTreeIndexAccessor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMTreeIndexAccessor.java
index 8412b8c..e688727 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMTreeIndexAccessor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMTreeIndexAccessor.java
@@ -28,6 +28,7 @@
import org.apache.hyracks.dataflow.common.data.accessors.FrameTupleReference;
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.IBatchController;
import org.apache.hyracks.storage.am.lsm.common.api.IFrameOperationCallback;
import org.apache.hyracks.storage.am.lsm.common.api.IFrameTupleProcessor;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
@@ -210,8 +211,8 @@
}
public void batchOperate(FrameTupleAccessor accessor, FrameTupleReference tuple, IFrameTupleProcessor processor,
- IFrameOperationCallback frameOpCallback) throws HyracksDataException {
- lsmHarness.batchOperate(ctx, accessor, tuple, processor, frameOpCallback);
+ IFrameOperationCallback frameOpCallback, IBatchController batchController) throws HyracksDataException {
+ lsmHarness.batchOperate(ctx, accessor, tuple, processor, frameOpCallback, batchController);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/util/ComponentUtils.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/util/ComponentUtils.java
index 1ff9fa8..842ec61 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/util/ComponentUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/util/ComponentUtils.java
@@ -34,7 +34,6 @@
import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
import org.apache.hyracks.storage.common.buffercache.IBufferCache;
import org.apache.hyracks.storage.common.buffercache.IPageWriteFailureCallback;
-import org.apache.logging.log4j.Level;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
@@ -80,11 +79,7 @@
*/
public static void get(ILSMIndex index, IValueReference key, ArrayBackedValueStorage value)
throws HyracksDataException {
- boolean loggable = LOGGER.isDebugEnabled();
value.reset();
- if (loggable) {
- LOGGER.log(Level.DEBUG, "Getting " + key + " from index " + index);
- }
// Lock the opTracker to ensure index components don't change
synchronized (index.getOperationTracker()) {
ILSMMemoryComponent cmc = index.getCurrentMemoryComponent();
@@ -92,33 +87,17 @@
index.getCurrentMemoryComponent().getMetadata().get(key, value);
}
if (value.getLength() == 0) {
- if (loggable) {
- LOGGER.log(Level.DEBUG, key + " was not found in mutable memory component of " + index);
- }
- // was not found in the in current mutable component, search in the other in memory components
+ // was not found in the in current mutable component, search in the other in-memory components
fromImmutableMemoryComponents(index, key, value);
if (value.getLength() == 0) {
- if (loggable) {
- LOGGER.log(Level.DEBUG, key + " was not found in all immmutable memory components of " + index);
- }
- // was not found in the in all in memory components, search in the disk components
+ // was not found in all in-memory components, search in the disk components
fromDiskComponents(index, key, value);
- if (loggable) {
- if (value.getLength() == 0) {
- LOGGER.log(Level.DEBUG, key + " was not found in all disk components of " + index);
- } else {
- LOGGER.log(Level.DEBUG, key + " was found in disk components of " + index);
- }
- }
- } else {
- if (loggable) {
- LOGGER.log(Level.DEBUG, key + " was found in the immutable memory components of " + index);
+ if (value.getLength() == 0) {
+ LOGGER.debug("{} was NOT found", key);
}
}
} else {
- if (loggable) {
- LOGGER.log(Level.DEBUG, key + " was found in mutable memory component of " + index);
- }
+ LOGGER.debug("{} was found in mutable memory component {}", key, cmc);
}
}
}
@@ -143,17 +122,11 @@
private static void fromDiskComponents(ILSMIndex index, IValueReference key, ArrayBackedValueStorage value)
throws HyracksDataException {
- boolean loggable = LOGGER.isDebugEnabled();
- if (loggable) {
- LOGGER.log(Level.DEBUG, "Getting " + key + " from disk components of " + index);
- }
for (ILSMDiskComponent c : index.getDiskComponents()) {
- if (loggable) {
- LOGGER.log(Level.DEBUG, "Getting " + key + " from disk components " + c);
- }
c.getMetadata().get(key, value);
if (value.getLength() != 0) {
// Found
+ LOGGER.debug("{} was found in disk component {}", key, c);
return;
}
}
@@ -161,21 +134,10 @@
private static void fromImmutableMemoryComponents(ILSMIndex index, IValueReference key,
ArrayBackedValueStorage value) throws HyracksDataException {
- boolean loggable = LOGGER.isDebugEnabled();
- if (loggable) {
- LOGGER.log(Level.DEBUG, "Getting " + key + " from immutable memory components of " + index);
- }
List<ILSMMemoryComponent> memComponents = index.getMemoryComponents();
int numOtherMemComponents = memComponents.size() - 1;
int next = index.getCurrentMemoryComponentIndex();
- if (loggable) {
- LOGGER.log(Level.DEBUG, index + " has " + numOtherMemComponents + " immutable memory components");
- }
for (int i = 0; i < numOtherMemComponents; i++) {
- if (loggable) {
- LOGGER.log(Level.DEBUG,
- "trying to get " + key + " from immutable memory components number: " + (i + 1));
- }
next = next - 1;
if (next < 0) {
next = memComponents.size() - 1;
@@ -185,6 +147,7 @@
c.getMetadata().get(key, value);
if (value.getLength() != 0) {
// Found
+ LOGGER.debug("{} was found in immutable memory component {}", key, c);
return;
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/pom.xml b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/pom.xml
index 3a58d74..db0997d 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/pom.xml
@@ -23,7 +23,7 @@
<parent>
<artifactId>hyracks</artifactId>
<groupId>org.apache.hyracks</groupId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
<relativePath>..</relativePath>
</parent>
<licenses>
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/pom.xml b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/pom.xml
index 7d3eff1..e89f384 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/pom.xml
@@ -23,7 +23,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<licenses>
<license>
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/pom.xml b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/pom.xml
index 0c92622..2c889e2 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/pom.xml
@@ -25,7 +25,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/pom.xml b/hyracks-fullstack/hyracks/hyracks-storage-common/pom.xml
index a1c4b02..d6cebb3 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/pom.xml
@@ -24,7 +24,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<licenses>
<license>
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/pom.xml b/hyracks-fullstack/hyracks/hyracks-test-support/pom.xml
index 8e4b23f..8b05c39 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/pom.xml
@@ -25,7 +25,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestJobletContext.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestJobletContext.java
index 46e3eec..bf4cbd0 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestJobletContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestJobletContext.java
@@ -20,6 +20,7 @@
import java.nio.ByteBuffer;
import java.time.ZoneId;
+import java.util.concurrent.atomic.AtomicLong;
import org.apache.hyracks.api.application.INCServiceContext;
import org.apache.hyracks.api.context.IHyracksJobletContext;
@@ -42,6 +43,7 @@
private final WorkspaceFileFactory fileFactory;
private final long jobStartTime;
private final String jobStartTimeZoneId;
+ private final AtomicLong ids;
TestJobletContext(int frameSize, INCServiceContext serviceContext, JobId jobId) throws HyracksException {
this.serviceContext = serviceContext;
@@ -50,6 +52,7 @@
this.frameManger = new FrameManager(frameSize);
this.jobStartTime = System.currentTimeMillis();
this.jobStartTimeZoneId = ZoneId.systemDefault().getId();
+ this.ids = new AtomicLong();
}
@Override
@@ -151,4 +154,8 @@
return this.getClass().getClassLoader();
}
+ @Override
+ public long nextUniqueId() {
+ return ids.getAndIncrement();
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestTaskContext.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestTaskContext.java
index dcb85f6..c73ebee 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestTaskContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestTaskContext.java
@@ -49,15 +49,17 @@
public class TestTaskContext implements IHyracksTaskContext {
private final TestJobletContext jobletContext;
private final TaskAttemptId taskId;
+ private int partitionCount;
private WorkspaceFileFactory fileFactory;
private Map<Object, IStateObject> stateObjectMap = new HashMap<>();
private Object sharedObject;
private final IStatsCollector statsCollector = new StatsCollector();
private final ThreadStats threadStats = new ThreadStats();
- public TestTaskContext(TestJobletContext jobletContext, TaskAttemptId taskId) {
+ public TestTaskContext(TestJobletContext jobletContext, TaskAttemptId taskId, int partitionCount) {
this.jobletContext = jobletContext;
this.taskId = taskId;
+ this.partitionCount = partitionCount;
fileFactory = new WorkspaceFileFactory(this, getIoManager());
}
@@ -129,6 +131,11 @@
}
@Override
+ public int getPartitionCount() {
+ return partitionCount;
+ }
+
+ @Override
public synchronized void setStateObject(IStateObject taskState) {
stateObjectMap.put(taskState.getId(), taskState);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestUtils.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestUtils.java
index 3f78234..2348a1a 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestUtils.java
@@ -38,7 +38,7 @@
import org.apache.hyracks.api.dataflow.TaskId;
import org.apache.hyracks.api.exceptions.HyracksException;
import org.apache.hyracks.api.exceptions.IWarningCollector;
-import org.apache.hyracks.api.exceptions.Warning;
+import org.apache.hyracks.api.exceptions.NoOpWarningCollector;
import org.apache.hyracks.api.io.IODeviceHandle;
import org.apache.hyracks.api.job.JobId;
import org.apache.hyracks.api.util.CleanupUtils;
@@ -52,22 +52,7 @@
public class TestUtils {
private static final int DEFAULT_FRAME_SIZE = 32768;
- public static final IWarningCollector NOOP_WARNING_COLLECTOR = new IWarningCollector() {
- @Override
- public void warn(Warning warning) {
- // no-op
- }
-
- @Override
- public boolean shouldWarn() {
- return false;
- }
-
- @Override
- public long getTotalWarningsCount() {
- return 0;
- }
- };
+ public static final IWarningCollector NOOP_WARNING_COLLECTOR = NoOpWarningCollector.INSTANCE;
public static IHyracksTaskContext createHyracksTask() {
return create(DEFAULT_FRAME_SIZE);
@@ -91,7 +76,7 @@
INCServiceContext serviceCtx = new TestNCServiceContext(ioManager, null);
TestJobletContext jobletCtx = new TestJobletContext(frameSize, serviceCtx, new JobId(0));
TaskAttemptId tid = new TaskAttemptId(new TaskId(new ActivityId(new OperatorDescriptorId(0), 0), 0), 0);
- IHyracksTaskContext taskCtx = new TestTaskContext(jobletCtx, tid);
+ IHyracksTaskContext taskCtx = new TestTaskContext(jobletCtx, tid, 1);
return taskCtx;
} catch (HyracksException e) {
throw new RuntimeException(e);
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-dataflow-common-test/pom.xml b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-dataflow-common-test/pom.xml
index 06381a9..1fd0aed 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-dataflow-common-test/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-dataflow-common-test/pom.xml
@@ -25,7 +25,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks-tests</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<build>
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-bloomfilter-test/pom.xml b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-bloomfilter-test/pom.xml
index f061079..800fcdd 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-bloomfilter-test/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-bloomfilter-test/pom.xml
@@ -24,7 +24,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks-tests</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<build>
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/pom.xml b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/pom.xml
index dc93d8b..038e346 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/pom.xml
@@ -25,7 +25,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks-tests</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<properties>
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/pom.xml b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/pom.xml
index 2e882ac..f191a2b 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/pom.xml
@@ -24,7 +24,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks-tests</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<build>
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResource.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResource.java
index 8e2d6dd..c7d3dff 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResource.java
@@ -56,19 +56,21 @@
ILSMOperationTrackerFactory opTrackerProvider, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
IMetadataPageManagerFactory metadataPageManagerFactory, IVirtualBufferCacheProvider vbcProvider,
- ILSMIOOperationSchedulerProvider ioSchedulerProvider, boolean durable, boolean hasBloomFilter) {
+ ILSMIOOperationSchedulerProvider ioSchedulerProvider, boolean durable, boolean hasBloomFilter,
+ boolean isSecondaryNoIncrementalMaintenance) {
super(typeTraits, cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate, isPrimary, path,
storageManager, mergePolicyFactory, mergePolicyProperties, filterTypeTraits, filterCmpFactories,
btreeFields, filterFields, opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory,
metadataPageManagerFactory, vbcProvider, ioSchedulerProvider, durable,
- NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter, null, null);
+ NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter, null, null,
+ isSecondaryNoIncrementalMaintenance);
}
protected TestLsmBtreeLocalResource(IPersistedResourceRegistry registry, JsonNode json, int[] bloomFilterKeyFields,
- double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields, boolean hasBloomFilter)
- throws HyracksDataException {
+ double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields, boolean hasBloomFilter,
+ boolean isSecondaryNoIncrementalMaintenance) throws HyracksDataException {
super(registry, json, bloomFilterKeyFields, bloomFilterFalsePositiveRate, isPrimary, btreeFields,
- NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter);
+ NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter, isSecondaryNoIncrementalMaintenance);
}
@Override
@@ -107,7 +109,8 @@
final boolean isPrimary = json.get("isPrimary").asBoolean();
final boolean hasBloomFilter = json.get("hasBloomFilter").asBoolean();
final int[] btreeFields = OBJECT_MAPPER.convertValue(json.get("btreeFields"), int[].class);
+ boolean isSecondaryNoIncrementalMaintenance = json.get("isSecondaryNoIncrementalMaintenance").asBoolean();
return new TestLsmBtreeLocalResource(registry, json, bloomFilterKeyFields, bloomFilterFalsePositiveRate,
- isPrimary, btreeFields, hasBloomFilter);
+ isPrimary, btreeFields, hasBloomFilter, isSecondaryNoIncrementalMaintenance);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResourceFactory.java
index 7d9b24a..48adf91 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResourceFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResourceFactory.java
@@ -46,12 +46,14 @@
IMetadataPageManagerFactory metadataPageManagerFactory, IVirtualBufferCacheProvider vbcProvider,
ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMMergePolicyFactory mergePolicyFactory,
Map<String, String> mergePolicyProperties, boolean durable, int[] bloomFilterKeyFields,
- double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields, boolean hasBloomFilter) {
+ double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields, boolean hasBloomFilter,
+ boolean isSecondaryNoIncrementalMaintenance) {
super(storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
opTrackerFactory, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable,
bloomFilterKeyFields, bloomFilterFalsePositiveRate, isPrimary, btreeFields,
- NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter, null, null);
+ NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter, null, null,
+ isSecondaryNoIncrementalMaintenance);
}
@Override
@@ -60,6 +62,6 @@
bloomFilterFalsePositiveRate, isPrimary, fileRef.getRelativePath(), storageManager, mergePolicyFactory,
mergePolicyProperties, filterTypeTraits, filterCmpFactories, btreeFields, filterFields,
opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
- vbcProvider, ioSchedulerProvider, durable, hasBloomFilter);
+ vbcProvider, ioSchedulerProvider, durable, hasBloomFilter, isSecondaryNoIncrementalMaintenance);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/pom.xml b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/pom.xml
index b0ea9ab..cbae843 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
@@ -24,7 +24,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks-tests</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<build>
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/pom.xml b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/pom.xml
index 10f1705..4be352f 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/pom.xml
@@ -24,7 +24,7 @@
<parent>
<artifactId>hyracks-tests</artifactId>
<groupId>org.apache.hyracks</groupId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
<relativePath>..</relativePath>
</parent>
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java
index aa9cd5a..144f1ad 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java
@@ -740,6 +740,11 @@
}
@Override
+ public int getPartitionCount() {
+ return 1;
+ }
+
+ @Override
public ICounterContext getCounterContext() {
return null;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/pom.xml b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/pom.xml
index 4629123..4db6a64 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/pom.xml
@@ -24,7 +24,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks-tests</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<build>
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/pom.xml b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/pom.xml
index 3462ec2..718cb1b 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/pom.xml
@@ -25,7 +25,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks-tests</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<build>
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-common-test/pom.xml b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-common-test/pom.xml
index 53808b9..86561c6 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-common-test/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-common-test/pom.xml
@@ -23,7 +23,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks-tests</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<build>
<plugins>
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/pom.xml b/hyracks-fullstack/hyracks/hyracks-tests/pom.xml
index 00c7949..a26adea 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-tests/pom.xml
@@ -26,7 +26,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/hyracks-fullstack/hyracks/hyracks-util/pom.xml b/hyracks-fullstack/hyracks/hyracks-util/pom.xml
index 7653ad9..ebe8760 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-util/pom.xml
@@ -22,7 +22,7 @@
<parent>
<artifactId>hyracks</artifactId>
<groupId>org.apache.hyracks</groupId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<modelVersion>4.0.0</modelVersion>
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/IOThrowingAction.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/IOThrowingAction.java
new file mode 100644
index 0000000..b9430d8
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/IOThrowingAction.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.util;
+
+import java.io.IOException;
+
+@FunctionalInterface
+public interface IOThrowingAction {
+ void run() throws IOException; // NOSONAR
+
+ static ComputingAction<Void> asComputingAction(IOThrowingAction action) {
+ return () -> {
+ action.run();
+ return null;
+ };
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/InterruptibleSupplier.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/InterruptibleSupplier.java
new file mode 100644
index 0000000..5de8c84
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/InterruptibleSupplier.java
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.util;
+
+@FunctionalInterface
+public interface InterruptibleSupplier<T> {
+ T get() throws InterruptedException;
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ReflectionUtils.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ReflectionUtils.java
index a5c83f0..7cc5cf8 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ReflectionUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ReflectionUtils.java
@@ -20,6 +20,7 @@
import java.io.IOException;
import java.lang.reflect.Field;
+import java.lang.reflect.Method;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
@@ -103,4 +104,10 @@
throw new IOException(e);
}
}
+
+ public static Method getAccessibleMethod(Class<?> clazz, String methodName) throws NoSuchMethodException {
+ Method m = clazz.getDeclaredMethod(methodName);
+ m.setAccessible(true);
+ return m;
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/string/UTF8StringUtil.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/string/UTF8StringUtil.java
deleted file mode 100644
index 3eb8687..0000000
--- a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/string/UTF8StringUtil.java
+++ /dev/null
@@ -1,707 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * http://www.apache.org/licenses/LICENSE-2.0
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.hyracks.util.string;
-
-import java.io.ByteArrayOutputStream;
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.DataOutputStream;
-import java.io.EOFException;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.io.UTFDataFormatException;
-import java.lang.ref.SoftReference;
-
-import org.apache.hyracks.util.encoding.VarLenIntEncoderDecoder;
-
-/**
- * A helper package to operate the UTF8String in Hyracks.
- * Most of the codes were migrated from asterix-fuzzyjoin and hyracks-storage-am-invertedindex
- */
-public class UTF8StringUtil {
-
- public static final String LOW_SURROGATE_WITHOUT_HIGH_SURROGATE =
- "Decoding error: got a low surrogate without a leading high surrogate";
- public static final String HIGH_SURROGATE_WITHOUT_LOW_SURROGATE =
- "Decoding error: got a high surrogate without a following low surrogate";
-
- private UTF8StringUtil() {
- }
-
- public static char charAt(byte[] b, int s) {
- if (s >= b.length) {
- throw new ArrayIndexOutOfBoundsException(s);
- }
- int c = b[s] & 0xff;
- switch (c >> 4) {
- case 0:
- case 1:
- case 2:
- case 3:
- case 4:
- case 5:
- case 6:
- case 7:
- return (char) c;
-
- case 12:
- case 13:
- return (char) (((c & 0x1F) << 6) | ((b[s + 1]) & 0x3F));
-
- case 14:
- return (char) (((c & 0x0F) << 12) | (((b[s + 1]) & 0x3F) << 6) | (b[s + 2] & 0x3F));
-
- default:
- throw new IllegalArgumentException();
- }
- }
-
- public static int charSize(byte[] b, int s) {
- int c = b[s] & 0xff;
- switch (c >> 4) {
- case 0:
- case 1:
- case 2:
- case 3:
- case 4:
- case 5:
- case 6:
- case 7:
- return 1;
-
- case 12:
- case 13:
- return 2;
-
- case 14:
- return 3;
-
- default:
- throw new IllegalStateException();
- }
- }
-
- public static int codePointAt(byte[] b, int s) {
- char c1 = charAt(b, s);
-
- if (Character.isLowSurrogate(c1)) {
- // In this case, the index s doesn't point to a correct position
- throw new IllegalArgumentException(LOW_SURROGATE_WITHOUT_HIGH_SURROGATE);
- }
-
- if (Character.isHighSurrogate(c1)) {
- // If c1 is the a high surrogate and also the last char in the byte array (that means the byte array is somehow illegal),
- // then an exception will be thrown because there is no low surrogate (c2) available in the byte array
- s += charSize(b, s);
- char c2 = charAt(b, s);
- if (Character.isLowSurrogate(c2)) {
- return Character.toCodePoint(c1, c2);
- } else {
- throw new IllegalArgumentException(HIGH_SURROGATE_WITHOUT_LOW_SURROGATE);
- }
- }
-
- return c1;
- }
-
- public static int codePointSize(byte[] b, int s) {
- char c1 = charAt(b, s);
- int size1 = charSize(b, s);
-
- if (Character.isLowSurrogate(c1)) {
- throw new IllegalArgumentException(LOW_SURROGATE_WITHOUT_HIGH_SURROGATE);
- }
-
- if (Character.isHighSurrogate(c1)) {
- // Similar to the above codePointAt(),
- // if c1 is the a high surrogate and also the last char in the byte array (that means the byte array is somehow illegal),
- // then an exception will be thrown because there is no low surrogate available in the byte array
- s += size1;
- int size2 = charSize(b, s);
- return size1 + size2;
- }
-
- return size1;
- }
-
- public static boolean isCharStart(byte[] b, int s) {
- int c = b[s] & 0xff;
- return (c >> 6) != 2;
- }
-
- public static int getModifiedUTF8Len(char c) {
- if (c >= 0x0001 && c <= 0x007F) {
- return 1;
- } else if (c <= 0x07FF) {
- return 2;
- } else {
- return 3;
- }
- }
-
- public static int writeCharAsModifiedUTF8(char c, DataOutput dos) throws IOException {
- if (c >= 0x0001 && c <= 0x007F) {
- dos.writeByte(c);
- return 1;
- } else if (c <= 0x07FF) {
- dos.writeByte((byte) (0xC0 | ((c >> 6) & 0x3F)));
- dos.writeByte((byte) (0x80 | (c & 0x3F)));
- return 2;
- } else {
- dos.writeByte((byte) (0xE0 | ((c >> 12) & 0x0F)));
- dos.writeByte((byte) (0x80 | ((c >> 6) & 0x3F)));
- dos.writeByte((byte) (0x80 | (c & 0x3F)));
- return 3;
- }
- }
-
- public static int writeCharAsModifiedUTF8(char c, OutputStream dos) throws IOException {
- if (c >= 0x0001 && c <= 0x007F) {
- dos.write(c);
- return 1;
- } else if (c <= 0x07FF) {
- dos.write((byte) (0xC0 | ((c >> 6) & 0x3F)));
- dos.write((byte) (0x80 | (c & 0x3F)));
- return 2;
- } else {
- dos.write((byte) (0xE0 | ((c >> 12) & 0x0F)));
- dos.write((byte) (0x80 | ((c >> 6) & 0x3F)));
- dos.write((byte) (0x80 | (c & 0x3F)));
- return 3;
- }
- }
-
- // The result is the number of Java Chars (8 bytes) in the string
- public static int getStringLength(byte[] b, int s) {
- int len = getUTFLength(b, s);
- int pos = s + getNumBytesToStoreLength(len);
- return getStringLength(b, pos, len);
- }
-
- public static int getStringLength(byte[] b, int offs, int len) {
- int pos = offs;
- int end = pos + len;
- int charCount = 0;
- while (pos < end) {
- charCount++;
- pos += charSize(b, pos);
- }
- return charCount;
- }
-
- public static int getNumCodePoint(byte[] b, int s) {
- int len = getUTFLength(b, s);
- int pos = s + getNumBytesToStoreLength(len);
- int end = pos + len;
- int codePointCount = 0;
- while (pos < end) {
- codePointCount++;
- pos += codePointSize(b, pos);
- }
-
- return codePointCount;
- }
-
- public static int getUTFLength(byte[] b, int s) {
- return VarLenIntEncoderDecoder.decode(b, s);
- }
-
- public static int getNumBytesToStoreLength(int strlen) {
- return VarLenIntEncoderDecoder.getBytesRequired(strlen);
- }
-
- public static int codePointToUTF8(int codePoint, char[] tempChars, byte[] outputUTF8) {
- int len = 0;
- int numChars = Character.toChars(codePoint, tempChars, 0);
- for (int i = 0; i < numChars; i++) {
- len += writeToBytes(outputUTF8, len, tempChars[i]);
- }
-
- return len;
- }
-
- /**
- * Compute the normalized key of the UTF8 string.
- * The normalized key in Hyracks is mainly used to speedup the comparison between pointable data.
- * In the UTF8StringPTR case, we compute the integer value by using the first 2 chars.
- * The comparator will first use this integer to get the result ( <,>, or =), it will check
- * the actual bytes only if the normalized key is equal. Thus this normalized key must be
- * consistent with the comparison result.
- */
- public static int normalize(byte[] bytes, int start) {
- int len = getUTFLength(bytes, start);
- long nk = 0;
- int offset = start + getNumBytesToStoreLength(len);
- for (int i = 0; i < 2; ++i) {
- nk <<= 16;
- if (i < len) {
- nk += (charAt(bytes, offset)) & 0xffff;
- offset += charSize(bytes, offset);
- }
- }
- return (int) (nk >> 1); // make it always positive.
- }
-
- public static int compareTo(byte[] thisBytes, int thisStart, byte[] thatBytes, int thatStart) {
- return compareTo(thisBytes, thisStart, thatBytes, thatStart, false, false);
- }
-
- // the start and length of each are the ones calculated by UTF8StringPointable. caller should provide proper values
- public static int compareTo(byte[] thisBytes, int thisStart, int thisLength, byte[] thatBytes, int thatStart,
- int thatLength) {
- return compareTo(thisBytes, thisStart, thisLength, thatBytes, thatStart, thatLength, false, false);
- }
-
- /**
- * This function provides the raw bytes-based comparison for UTF8 strings.
- * Note that the comparison may not deliver the correct ordering for certain languages that include 2 or 3 bytes characters.
- * But it works for single-byte character languages.
- */
- public static int rawByteCompareTo(byte[] thisBytes, int thisStart, byte[] thatBytes, int thatStart) {
- return compareTo(thisBytes, thisStart, thatBytes, thatStart, false, true);
- }
-
- public static int lowerCaseCompareTo(byte[] thisBytes, int thisStart, byte[] thatBytes, int thatStart) {
- return compareTo(thisBytes, thisStart, thatBytes, thatStart, true, false);
- }
-
- // Certain type of string does not include lengthByte in the beginning and
- // the length of the given string is given explicitly as a parameter. (e.g., token in a string)
- public static int lowerCaseCompareTo(byte[] thisBytes, int thisStart, int thisLength, byte[] thatBytes,
- int thatStart, int thatLength) {
- return compareTo(thisBytes, thisStart, thisLength, thatBytes, thatStart, thatLength, true, false);
- }
-
- public static int hash(byte[] bytes, int start, int coefficient, int r) {
- return hash(bytes, start, false, false, coefficient, r);
- }
-
- public static int hash(byte[] bytes, int start) {
- return hash(bytes, start, false, false, 31, Integer.MAX_VALUE);
- }
-
- private static int hash(byte[] bytes, int start, boolean useLowerCase, boolean useRawByte, int coefficient, int r) {
- int utflen = getUTFLength(bytes, start);
- int sStart = start + getNumBytesToStoreLength(utflen);
- return hash(bytes, sStart, utflen, useLowerCase, useRawByte, coefficient, r);
- }
-
- /**
- * This function provides the raw bytes-based hash function for UTF8 strings.
- * Note that the hash values may not deliver the correct ordering for certain languages that include 2 or 3 bytes characters.
- * But it works for single-byte character languages.
- */
- public static int rawBytehash(byte[] bytes, int start) {
- return hash(bytes, start, false, true, 31, Integer.MAX_VALUE);
- }
-
- public static int lowerCaseHash(byte[] bytes, int start) {
- return hash(bytes, start, true, false, 31, Integer.MAX_VALUE);
- }
-
- // Certain type of string does not include lengthByte in the beginning and
- // the length of the given string is given explicitly as a parameter.
- public static int lowerCaseHash(byte[] bytes, int start, int length) {
- return hash(bytes, start, length, true, false, 31, Integer.MAX_VALUE);
- }
-
- public static String toString(byte[] bytes, int start) {
- StringBuilder builder = new StringBuilder();
- return toString(builder, bytes, start).toString();
- }
-
- public static StringBuilder toString(StringBuilder builder, byte[] bytes, int start) {
- int utfLen = getUTFLength(bytes, start);
- int offset = getNumBytesToStoreLength(utfLen);
- while (utfLen > 0) {
- char c = charAt(bytes, start + offset);
- builder.append(c);
- int cLen = getModifiedUTF8Len(c);
- offset += cLen;
- utfLen -= cLen;
- }
- return builder;
- }
-
- // Different from the above toString() methods, here we assume the byte[] doesn't contain NumBytesToStoreLength
- // In fact, this is used for string tokenizer: get "hello" and "world" from the bytes of "hello world"
- public static String getUTF8StringInArray(byte[] b, int start, int len) {
- StringBuilder builder = new StringBuilder();
-
- for (int i = start; i < start + len;) {
- char c = UTF8StringUtil.charAt(b, i);
- builder.append(c);
- i += UTF8StringUtil.charSize(b, i);
- }
-
- return builder.toString();
- }
-
- public static void printUTF8StringWithQuotes(byte[] b, int s, int l, OutputStream os) throws IOException {
- printUTF8String(b, s, l, os, true);
- }
-
- public static void printUTF8StringNoQuotes(byte[] b, int s, int l, OutputStream os) throws IOException {
- printUTF8String(b, s, l, os, false);
- }
-
- public static void printUTF8StringWithQuotes(String str, OutputStream os) throws IOException {
- printUTF8String(str, os, true);
- }
-
- public static void printUTF8StringNoQuotes(String str, OutputStream os) throws IOException {
- printUTF8String(str, os, false);
- }
-
- public static int encodeUTF8Length(int length, byte[] bytes, int start) {
- return VarLenIntEncoderDecoder.encode(length, bytes, start);
- }
-
- public static int writeUTF8Length(int length, byte[] bytes, DataOutput out) throws IOException {
- int nbytes = encodeUTF8Length(length, bytes, 0);
- out.write(bytes, 0, nbytes);
- return nbytes;
- }
-
- private static void printUTF8String(byte[] b, int s, int l, OutputStream os, boolean useQuotes) throws IOException {
- int stringLength = getUTFLength(b, s);
- int position = s + getNumBytesToStoreLength(stringLength);
- int maxPosition = position + stringLength;
- if (useQuotes) {
- os.write('\"');
- }
- while (position < maxPosition) {
- char c = charAt(b, position);
- if (c == '\\' || c == '"') {
- // escape
- os.write('\\');
- }
- int sz = charSize(b, position);
- while (sz > 0) {
- os.write(b[position]);
- position++;
- sz--;
- }
- }
- if (useQuotes) {
- os.write('\"');
- }
- }
-
- private static void printUTF8String(String string, OutputStream os, boolean useQuotes) throws IOException {
- if (useQuotes) {
- os.write('\"');
- }
- for (int i = 0; i < string.length(); i++) {
- char ch = string.charAt(i);
- writeCharAsModifiedUTF8(ch, os);
- }
- if (useQuotes) {
- os.write('\"');
- }
- }
-
- private static int compareTo(byte[] thisBytes, int thisStart, byte[] thatBytes, int thatStart, boolean useLowerCase,
- boolean useRawByte) {
- int thisLength = getUTFLength(thisBytes, thisStart);
- int thatLength = getUTFLength(thatBytes, thatStart);
- int thisActualStart = thisStart + getNumBytesToStoreLength(thisLength);
- int thatActualStart = thatStart + getNumBytesToStoreLength(thatLength);
- return compareTo(thisBytes, thisActualStart, thisLength, thatBytes, thatActualStart, thatLength, useLowerCase,
- useRawByte);
- }
-
- private static int compareTo(byte[] thisBytes, int thisActualStart, int thisLength, byte[] thatBytes,
- int thatActualStart, int thatLength, boolean useLowerCase, boolean useRawByte) {
- int c1 = 0;
- int c2 = 0;
-
- while (c1 < thisLength && c2 < thatLength) {
- char ch1, ch2;
- if (useRawByte) {
- ch1 = (char) thisBytes[thisActualStart + c1];
- ch2 = (char) thatBytes[thatActualStart + c2];
- } else {
- ch1 = charAt(thisBytes, thisActualStart + c1);
- ch2 = charAt(thatBytes, thatActualStart + c2);
-
- if (useLowerCase) {
- ch1 = Character.toLowerCase(ch1);
- ch2 = Character.toLowerCase(ch2);
- }
- }
-
- if (ch1 != ch2) {
- return ch1 - ch2;
- }
- c1 += charSize(thisBytes, thisActualStart + c1);
- c2 += charSize(thatBytes, thatActualStart + c2);
- }
- return thisLength - thatLength;
- }
-
- private static int hash(byte[] bytes, int start, int length, boolean useLowerCase, boolean useRawByte,
- int coefficient, int r) {
- int h = 0;
- int c = 0;
-
- while (c < length) {
- char ch;
- if (useRawByte) {
- ch = (char) bytes[start + c];
- } else {
- ch = charAt(bytes, start + c);
- if (useLowerCase) {
- ch = Character.toLowerCase(ch);
- }
- }
- h = (coefficient * h + ch) % r;
- c += charSize(bytes, start + c);
- }
- return h;
- }
-
- public static byte[] writeStringToBytes(String string) {
- UTF8StringWriter writer = new UTF8StringWriter();
- ByteArrayOutputStream bos = new ByteArrayOutputStream();
- DataOutputStream dos = new DataOutputStream(bos);
- try {
- writer.writeUTF8(string, dos);
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
- return bos.toByteArray();
- }
-
- /**
- * Reads from the
- * stream <code>in</code> a representation
- * of a Unicode character string encoded in
- * <a href="DataInput.html#modified-utf-8">modified UTF-8</a> format;
- * this string of characters is then returned as a <code>String</code>.
- * The details of the modified UTF-8 representation
- * are exactly the same as for the <code>readUTF</code>
- * method of <code>DataInput</code>.
- *
- * @param in
- * a data input stream.
- * @return a Unicode string.
- * @throws EOFException
- * if the input stream reaches the end
- * before all the bytes.
- * @throws IOException
- * the stream has been closed and the contained
- * input stream does not support reading after close, or
- * another I/O error occurs.
- * @throws UTFDataFormatException
- * if the bytes do not represent a
- * valid modified UTF-8 encoding of a Unicode string.
- * @see java.io.DataInputStream#readUnsignedShort()
- */
- public static String readUTF8(DataInput in) throws IOException {
- return readUTF8(in, null);
- }
-
- public static String readUTF8(DataInput in, UTF8StringReader reader) throws IOException {
- int utflen = VarLenIntEncoderDecoder.decode(in);
- byte[] bytearr;
- char[] chararr;
-
- if (reader == null) {
- bytearr = new byte[utflen * 2];
- chararr = new char[utflen * 2];
- } else {
- if (reader.bytearr == null || reader.bytearr.length < utflen) {
- reader.bytearr = new byte[utflen * 2];
- reader.chararr = new char[utflen * 2];
- }
- bytearr = reader.bytearr;
- chararr = reader.chararr;
- }
-
- int c, char2, char3;
- int count = 0;
- int chararr_count = 0;
-
- in.readFully(bytearr, 0, utflen);
-
- while (count < utflen) {
- c = bytearr[count] & 0xff;
- if (c > 127) {
- break;
- }
- count++;
- chararr[chararr_count++] = (char) c;
- }
-
- while (count < utflen) {
- c = bytearr[count] & 0xff;
- switch (c >> 4) {
- case 0:
- case 1:
- case 2:
- case 3:
- case 4:
- case 5:
- case 6:
- case 7:
- /* 0xxxxxxx*/
- count++;
- chararr[chararr_count++] = (char) c;
- break;
- case 12:
- case 13:
- /* 110x xxxx 10xx xxxx*/
- count += 2;
- if (count > utflen) {
- throw new UTFDataFormatException("malformed input: partial character at end");
- }
- char2 = bytearr[count - 1];
- if ((char2 & 0xC0) != 0x80) {
- throw new UTFDataFormatException("malformed input around byte " + count);
- }
- chararr[chararr_count++] = (char) (((c & 0x1F) << 6) | (char2 & 0x3F));
- break;
- case 14:
- /* 1110 xxxx 10xx xxxx 10xx xxxx */
- count += 3;
- if (count > utflen) {
- throw new UTFDataFormatException("malformed input: partial character at end");
- }
- char2 = bytearr[count - 2];
- char3 = bytearr[count - 1];
- if (((char2 & 0xC0) != 0x80) || ((char3 & 0xC0) != 0x80)) {
- throw new UTFDataFormatException("malformed input around byte " + (count - 1));
- }
- chararr[chararr_count++] = (char) (((c & 0x0F) << 12) | ((char2 & 0x3F) << 6) | (char3 & 0x3F));
- break;
- default:
- /* 10xx xxxx, 1111 xxxx */
- throw new UTFDataFormatException("malformed input around byte " + count);
- }
- }
- // The number of chars produced may be less than utflen
- return new String(chararr, 0, chararr_count);
- }
-
- /**
- * Write a UTF8 String <code>str</code> into the DataOutput <code>out</code>
- *
- * @param str,
- * a Unicode string;
- * @param out,
- * a Data output stream.
- * @throws IOException
- */
- public static void writeUTF8(CharSequence str, DataOutput out) throws IOException {
- writeUTF8(str, out, null);
- }
-
- public static void writeUTF8(CharSequence str, DataOutput out, UTF8StringWriter writer) throws IOException {
- int strlen = str.length();
- int utflen = 0;
- char c;
- int count = 0;
-
- for (int i = 0; i < strlen; i++) {
- // ToDo: we shouldn't use str.charAt(i) to convert raw byte array to UTF-8 chars
- // one UTF-8 char has at most four bytes, and one Java char we get via str.charAt(i) has 2 bytes
- // In this case, a UTF-8 char may be consistent of 2 Java chars, and 1 Java char can be converted into 3 UTF-8 bytes
- // For the emoji, it can be 6 bytes after encoded to UTF-8
- c = str.charAt(i);
- utflen += UTF8StringUtil.getModifiedUTF8Len(c);
- }
-
- byte[] tempBytes = getTempBytes(writer, utflen);
- count += VarLenIntEncoderDecoder.encode(utflen, tempBytes, count);
- int i = 0;
- for (; i < strlen; i++) {
- c = str.charAt(i);
- if (!((c >= 0x0001) && (c <= 0x007F))) {
- break;
- }
- tempBytes[count++] = (byte) c;
- }
-
- for (; i < strlen; i++) {
- c = str.charAt(i);
- count += writeToBytes(tempBytes, count, c);
- }
- out.write(tempBytes, 0, count);
- }
-
- public static void writeUTF8(char[] buffer, int start, int length, DataOutput out, UTF8StringWriter writer)
- throws IOException {
- int utflen = 0;
- int count = 0;
- char c;
-
- for (int i = 0; i < length; i++) {
- c = buffer[i + start];
- utflen += UTF8StringUtil.getModifiedUTF8Len(c);
- }
-
- byte[] tempBytes = getTempBytes(writer, utflen);
- count += VarLenIntEncoderDecoder.encode(utflen, tempBytes, count);
-
- int i = 0;
- for (; i < length; i++) {
- c = buffer[i + start];
- if (!((c >= 0x0001) && (c <= 0x007F))) {
- break;
- }
- tempBytes[count++] = (byte) c;
- }
-
- for (; i < length; i++) {
- c = buffer[i + start];
- count += writeToBytes(tempBytes, count, c);
- }
- out.write(tempBytes, 0, count);
- }
-
- private static int writeToBytes(byte[] tempBytes, int count, char c) {
- int orig = count;
- if ((c >= 0x0001) && (c <= 0x007F)) {
- tempBytes[count++] = (byte) c;
- } else if (c > 0x07FF) {
- tempBytes[count++] = (byte) (0xE0 | ((c >> 12) & 0x0F));
- tempBytes[count++] = (byte) (0x80 | ((c >> 6) & 0x3F));
- tempBytes[count++] = (byte) (0x80 | (c & 0x3F));
- } else {
- tempBytes[count++] = (byte) (0xC0 | ((c >> 6) & 0x1F));
- tempBytes[count++] = (byte) (0x80 | (c & 0x3F));
- }
- return count - orig;
- }
-
- private static byte[] getTempBytes(UTF8StringWriter writer, int utflen) {
- byte[] tempBytes;
- if (writer == null) {
- tempBytes = new byte[utflen + 5];
- } else {
- byte[] writerTempBytes = writer.tempBytesRef != null ? writer.tempBytesRef.get() : null;
- if (writerTempBytes == null || writerTempBytes.length < utflen + 5) {
- writerTempBytes = new byte[utflen + 5];
- writer.tempBytesRef = new SoftReference<>(writerTempBytes);
- }
- tempBytes = writerTempBytes;
- }
- return tempBytes;
- }
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/test/java/org/apache/hyracks/util/string/UTF8StringReaderWriterTest.java b/hyracks-fullstack/hyracks/hyracks-util/src/test/java/org/apache/hyracks/util/string/UTF8StringReaderWriterTest.java
deleted file mode 100644
index 9010c9c..0000000
--- a/hyracks-fullstack/hyracks/hyracks-util/src/test/java/org/apache/hyracks/util/string/UTF8StringReaderWriterTest.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.hyracks.util.string;
-
-import static org.apache.hyracks.util.string.UTF8StringSample.EMPTY_STRING;
-import static org.apache.hyracks.util.string.UTF8StringSample.STRING_LEN_127;
-import static org.apache.hyracks.util.string.UTF8StringSample.STRING_LEN_128;
-import static org.apache.hyracks.util.string.UTF8StringSample.STRING_LEN_3;
-import static org.apache.hyracks.util.string.UTF8StringSample.STRING_LEN_LARGE;
-import static org.apache.hyracks.util.string.UTF8StringSample.STRING_LEN_LARGE_SUB_1;
-import static org.apache.hyracks.util.string.UTF8StringSample.STRING_LEN_MEDIUM;
-import static org.apache.hyracks.util.string.UTF8StringSample.STRING_LEN_MEDIUM_SUB_1;
-import static org.apache.hyracks.util.string.UTF8StringSample.STRING_UTF8_3;
-import static org.apache.hyracks.util.string.UTF8StringSample.STRING_UTF8_MIX;
-import static org.junit.Assert.assertEquals;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-
-import org.junit.Test;
-
-public class UTF8StringReaderWriterTest {
-
- UTF8StringWriter writer = new UTF8StringWriter();
- UTF8StringReader reader = new UTF8StringReader();
-
- @Test
- public void testWriterReader() throws IOException {
- writeAndReadOneString(EMPTY_STRING);
- writeAndReadOneString(STRING_LEN_3);
-
- writeAndReadOneString(STRING_LEN_127);
- writeAndReadOneString(STRING_LEN_128);
- writeAndReadOneString(STRING_LEN_MEDIUM_SUB_1);
- }
-
- @Test
- public void testMedium() throws IOException {
- writeAndReadOneString(STRING_LEN_MEDIUM);
- writeAndReadOneString(STRING_LEN_LARGE_SUB_1);
- }
-
- @Test
- public void testLarge() throws IOException {
- writeAndReadOneString(STRING_LEN_LARGE);
- }
-
- @Test
- public void testUTF8() throws IOException {
- writeAndReadOneString(STRING_UTF8_3);
- writeAndReadOneString(STRING_UTF8_MIX);
- }
-
- private void writeAndReadOneString(String testString) throws IOException {
- ByteArrayOutputStream bos = new ByteArrayOutputStream();
- DataOutputStream dos = new DataOutputStream(bos);
- writer.writeUTF8(testString, dos);
-
- ByteArrayInputStream bis = new ByteArrayInputStream(bos.toByteArray(), 0, bos.size());
- assertEquals(testString, reader.readUTF(new DataInputStream(bis)));
-
- int lastOffset = bos.size();
- char[] charArray = testString.toCharArray();
- writer.writeUTF8(charArray, 0, charArray.length, dos);
-
- bis = new ByteArrayInputStream(bos.toByteArray(), lastOffset, bos.size());
- assertEquals(testString, reader.readUTF(new DataInputStream(bis)));
- }
-
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/test/java/org/apache/hyracks/util/string/UTF8StringUtilTest.java b/hyracks-fullstack/hyracks/hyracks-util/src/test/java/org/apache/hyracks/util/string/UTF8StringUtilTest.java
deleted file mode 100644
index c7468d2..0000000
--- a/hyracks-fullstack/hyracks/hyracks-util/src/test/java/org/apache/hyracks/util/string/UTF8StringUtilTest.java
+++ /dev/null
@@ -1,189 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.hyracks.util.string;
-
-import static org.apache.hyracks.util.string.UTF8StringSample.STRING_LEN_127;
-import static org.apache.hyracks.util.string.UTF8StringSample.STRING_LEN_128;
-import static org.apache.hyracks.util.string.UTF8StringSample.STRING_LEN_MEDIUM;
-import static org.apache.hyracks.util.string.UTF8StringSample.STRING_UTF8_3;
-import static org.apache.hyracks.util.string.UTF8StringSample.STRING_UTF8_MIX;
-import static org.apache.hyracks.util.string.UTF8StringSample.STRING_UTF8_MIX_LOWERCASE;
-import static org.apache.hyracks.util.string.UTF8StringUtil.charAt;
-import static org.apache.hyracks.util.string.UTF8StringUtil.charSize;
-import static org.apache.hyracks.util.string.UTF8StringUtil.compareTo;
-import static org.apache.hyracks.util.string.UTF8StringUtil.getModifiedUTF8Len;
-import static org.apache.hyracks.util.string.UTF8StringUtil.getNumBytesToStoreLength;
-import static org.apache.hyracks.util.string.UTF8StringUtil.getNumCodePoint;
-import static org.apache.hyracks.util.string.UTF8StringUtil.getStringLength;
-import static org.apache.hyracks.util.string.UTF8StringUtil.getUTF8StringInArray;
-import static org.apache.hyracks.util.string.UTF8StringUtil.getUTFLength;
-import static org.apache.hyracks.util.string.UTF8StringUtil.hash;
-import static org.apache.hyracks.util.string.UTF8StringUtil.lowerCaseCompareTo;
-import static org.apache.hyracks.util.string.UTF8StringUtil.lowerCaseHash;
-import static org.apache.hyracks.util.string.UTF8StringUtil.normalize;
-import static org.apache.hyracks.util.string.UTF8StringUtil.rawByteCompareTo;
-import static org.apache.hyracks.util.string.UTF8StringUtil.writeStringToBytes;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.junit.Test;
-
-public class UTF8StringUtilTest {
-
- @Test
- public void testCharAtCharSizeGetLen() throws Exception {
- char[] utf8Mix = STRING_UTF8_MIX.toCharArray();
- byte[] buffer = writeStringToBytes(STRING_UTF8_MIX);
- int pos = getNumBytesToStoreLength(getUTFLength(buffer, 0));
- for (char c : utf8Mix) {
- assertEquals(c, charAt(buffer, pos));
- assertEquals(getModifiedUTF8Len(c), charSize(buffer, pos));
- pos += charSize(buffer, pos);
- }
- }
-
- @Test
- public void testGetStringLength() throws Exception {
- byte[] buffer = writeStringToBytes(STRING_UTF8_MIX);
- assertEquals(STRING_UTF8_MIX.length(), getStringLength(buffer, 0));
- }
-
- @Test
- public void testChinese() {
- byte[] bufferDe = writeStringToBytes("的");
- byte[] bufferLi = writeStringToBytes("离");
- int ret = compareTo(bufferDe, 0, bufferLi, 0);
- assertTrue(ret != 0);
- }
-
- @Test
- public void testCompareToAndNormolize() throws Exception {
- testCompare(STRING_UTF8_MIX, STRING_UTF8_MIX, OPTION.STANDARD);
- testCompare(STRING_UTF8_3, STRING_UTF8_MIX, OPTION.STANDARD);
- testCompare(STRING_LEN_MEDIUM, STRING_UTF8_MIX, OPTION.STANDARD);
- }
-
- public boolean isSameSign(int r1, int r2) {
- if (r1 > 0) {
- return r2 > 0;
- }
- if (r1 < 0) {
- return r2 < 0;
- }
- return r2 == 0;
- }
-
- enum OPTION {
- STANDARD,
- RAW_BYTE,
- LOWERCASE
- }
-
- public void testCompare(String str1, String str2, OPTION option) throws IOException {
- byte[] buffer1 = writeStringToBytes(str1);
- byte[] buffer2 = writeStringToBytes(str2);
-
- switch (option) {
- case STANDARD:
- assertEquals(str1.compareTo(str2), compareTo(buffer1, 0, buffer2, 0));
- int n1 = normalize(buffer1, 0);
- int n2 = normalize(buffer2, 0);
- assertTrue(isSameSign(str1.compareTo(str2), n1 - n2));
- break;
- case RAW_BYTE:
- assertEquals(str1.compareTo(str2), rawByteCompareTo(buffer1, 0, buffer2, 0));
- break;
- case LOWERCASE:
- assertEquals(str1.compareToIgnoreCase(str2), lowerCaseCompareTo(buffer1, 0, buffer2, 0));
- break;
- }
-
- }
-
- @Test
- public void testRawByteCompareTo() throws Exception {
- testCompare(STRING_LEN_MEDIUM, STRING_LEN_MEDIUM, OPTION.RAW_BYTE);
- testCompare(STRING_LEN_127, STRING_LEN_128, OPTION.RAW_BYTE);
- }
-
- @Test
- public void testLowerCaseCompareTo() throws Exception {
- testCompare(STRING_LEN_127, STRING_LEN_128, OPTION.LOWERCASE);
- testCompare(STRING_LEN_127, STRING_UTF8_MIX, OPTION.LOWERCASE);
- testCompare(STRING_UTF8_MIX, STRING_UTF8_MIX_LOWERCASE, OPTION.LOWERCASE);
- testCompare(STRING_UTF8_MIX_LOWERCASE, STRING_UTF8_MIX, OPTION.LOWERCASE);
- }
-
- @Test
- public void testToString() throws Exception {
-
- StringBuilder sb = new StringBuilder();
- byte[] buffer = writeStringToBytes(STRING_UTF8_MIX);
- assertEquals(STRING_UTF8_MIX, UTF8StringUtil.toString(sb, buffer, 0).toString());
- }
-
- @Test
- public void testHash() throws IOException {
- byte[] buffer = writeStringToBytes(STRING_UTF8_MIX_LOWERCASE);
- int lowerHash = hash(buffer, 0);
-
- buffer = writeStringToBytes(STRING_UTF8_MIX_LOWERCASE);
- int upperHash = lowerCaseHash(buffer, 0);
- assertEquals(lowerHash, upperHash);
-
- int familyOne = hash(buffer, 0, 7, 297);
- int familyTwo = hash(buffer, 0, 8, 297);
- assertTrue(familyOne != familyTwo);
- }
-
- @Test
- public void testGetUTF8StringInArray() {
- String str = null;
- byte[] bytes = null;
- List<String> answer = null;
-
- str = "database group at university of California, Irvine 23333";
- bytes = writeStringToBytes(str);
- // First byte in bytes is for the number of bytes of the entire string,
- // and it should be skipped in getUTF8StringInArray
- assertEquals("database", getUTF8StringInArray(bytes, 1, 8));
- assertEquals("at", getUTF8StringInArray(bytes, 16, 2));
- // test upper case
- assertEquals("California", getUTF8StringInArray(bytes, 33, 10));
- // test non-english char
- assertEquals(",", getUTF8StringInArray(bytes, 43, 1));
- assertEquals("Irvine", getUTF8StringInArray(bytes, 45, 6));
- // test number
- assertEquals("23333", getUTF8StringInArray(bytes, 52, 5));
- }
-
- @Test
- public void testGetNumCodePoint() {
- String str = "\uD83D\uDC69\u200D\uD83D\uDC69\u200D\uD83D\uDC67\u200D\uD83D\uDC66";
- assertEquals(getNumCodePoint(writeStringToBytes(str), 0), 7);
-
- str = "\uD83D\uDC69\u200D\uD83D\uDC69\u200D\uD83D\uDC67\u200D\uD83D\uDC66\uD83C\uDDE8\uD83C\uDDF3";
- assertEquals(getNumCodePoint(writeStringToBytes(str), 0), 9);
- }
-
-}
diff --git a/hyracks-fullstack/hyracks/pom.xml b/hyracks-fullstack/hyracks/pom.xml
index 443e8c0..a50e544 100644
--- a/hyracks-fullstack/hyracks/pom.xml
+++ b/hyracks-fullstack/hyracks/pom.xml
@@ -26,7 +26,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>apache-hyracks</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
</parent>
<build>
diff --git a/hyracks-fullstack/pom.xml b/hyracks-fullstack/pom.xml
index 432947e..487cadd 100644
--- a/hyracks-fullstack/pom.xml
+++ b/hyracks-fullstack/pom.xml
@@ -21,7 +21,7 @@
<modelVersion>4.0.0</modelVersion>
<groupId>org.apache.hyracks</groupId>
<artifactId>apache-hyracks</artifactId>
- <version>0.3.8-SNAPSHOT</version>
+ <version>0.3.8.2-SNAPSHOT</version>
<packaging>pom</packaging>
<name>hyracks-ecosystem-full-stack</name>
<url>${implementation.url}</url>
@@ -74,9 +74,9 @@
<jacoco.version>0.7.6.201602180812</jacoco.version>
<log4j.version>2.19.0</log4j.version>
<snappy.version>1.1.10.5</snappy.version>
- <jackson.version>2.14.1</jackson.version>
+ <jackson.version>2.14.3</jackson.version>
<jackson-databind.version>${jackson.version}</jackson-databind.version>
- <netty.version>4.1.87.Final</netty.version>
+ <netty.version>4.1.101.Final</netty.version>
<implementation.title>Apache Hyracks and Algebricks - ${project.name}</implementation.title>
<implementation.url>https://asterixdb.apache.org/</implementation.url>
@@ -112,6 +112,16 @@
</dependency>
<dependency>
<groupId>io.netty</groupId>
+ <artifactId>netty-codec-dns</artifactId>
+ <version>${netty.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-codec-socks</artifactId>
+ <version>${netty.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>io.netty</groupId>
<artifactId>netty-handler</artifactId>
<version>${netty.version}</version>
</dependency>
@@ -127,6 +137,11 @@
</dependency>
<dependency>
<groupId>io.netty</groupId>
+ <artifactId>netty-resolver</artifactId>
+ <version>${netty.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>io.netty</groupId>
<artifactId>netty-resolver-dns</artifactId>
<version>${netty.version}</version>
</dependency>
@@ -181,12 +196,8 @@
<artifactId>jdk.tools</artifactId>
</exclusion>
<exclusion>
- <groupId>log4j</groupId>
- <artifactId>log4j</artifactId>
- </exclusion>
- <exclusion>
<groupId>org.slf4j</groupId>
- <artifactId>slf4j-api</artifactId>
+ <artifactId>slf4j-reload4j</artifactId>
</exclusion>
<exclusion>
<groupId>org.slf4j</groupId>
@@ -204,8 +215,12 @@
<artifactId>netty-all</artifactId>
</exclusion>
<exclusion>
- <groupId>log4j</groupId>
- <artifactId>log4j</artifactId>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-reload4j</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>ch.qos.reload4j</groupId>
+ <artifactId>reload4j</artifactId>
</exclusion>
</exclusions>
</dependency>
@@ -215,6 +230,14 @@
<version>${hadoop.version}</version>
<exclusions>
<exclusion>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-reload4j</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>ch.qos.reload4j</groupId>
+ <artifactId>reload4j</artifactId>
+ </exclusion>
+ <exclusion>
<groupId>log4j</groupId>
<artifactId>log4j</artifactId>
</exclusion>
@@ -233,8 +256,12 @@
<scope>test</scope>
<exclusions>
<exclusion>
- <groupId>log4j</groupId>
- <artifactId>log4j</artifactId>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-reload4j</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>ch.qos.reload4j</groupId>
+ <artifactId>reload4j</artifactId>
</exclusion>
</exclusions>
</dependency>
@@ -735,7 +762,14 @@
<plugin>
<groupId>net.revelc.code</groupId>
<artifactId>impsort-maven-plugin</artifactId>
- <version>1.2.0</version>
+ <version>1.9.0</version>
+ <dependencies>
+ <dependency>
+ <groupId>com.github.javaparser</groupId>
+ <artifactId>javaparser-core</artifactId>
+ <version>3.25.5</version>
+ </dependency>
+ </dependencies>
</plugin>
<!--This plugin's configuration is used to store Eclipse m2e settings only. It has no influence on the Maven build itself.-->
<plugin>
diff --git a/pom.xml b/pom.xml
index eac0b88..ff76d8c 100644
--- a/pom.xml
+++ b/pom.xml
@@ -19,7 +19,7 @@
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
<modelVersion>4.0.0</modelVersion>
<artifactId>apache-asterixdb-fullstack</artifactId>
- <version>0.9.8-SNAPSHOT</version>
+ <version>0.9.8.2-SNAPSHOT</version>
<packaging>pom</packaging>
<name>hyracks-asterix</name>