Merge branch 'release-0.9.8'
Change-Id: I511044b28d6a0f2f2d2639ea1432281f4b2a5c21
diff --git a/asterixdb/NOTICE b/asterixdb/NOTICE
index 17c97bb..a46bfe7 100644
--- a/asterixdb/NOTICE
+++ b/asterixdb/NOTICE
@@ -1,5 +1,5 @@
Apache AsterixDB
-Copyright 2015-2022 The Apache Software Foundation
+Copyright 2015-2023 The Apache Software Foundation
This product includes software developed at
The Apache Software Foundation (http://www.apache.org/).
diff --git a/asterixdb/asterix-algebra/pom.xml b/asterixdb/asterix-algebra/pom.xml
index d8ac2b4..7a24cbe 100644
--- a/asterixdb/asterix-algebra/pom.xml
+++ b/asterixdb/asterix-algebra/pom.xml
@@ -202,6 +202,10 @@
<artifactId>hyracks-api</artifactId>
</dependency>
<dependency>
+ <groupId>org.apache.hyracks</groupId>
+ <artifactId>hyracks-client</artifactId>
+ </dependency>
+ <dependency>
<groupId>org.apache.logging.log4j</groupId>
<artifactId>log4j-api</artifactId>
</dependency>
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/BTreeSearchPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/BTreeSearchPOperator.java
index 06a6687..fed180b 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/BTreeSearchPOperator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/BTreeSearchPOperator.java
@@ -62,9 +62,12 @@
import org.apache.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
import org.apache.hyracks.algebricks.core.algebra.properties.UnorderedPartitionedProperty;
import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
+import org.apache.hyracks.storage.am.common.impls.DefaultTupleProjectorFactory;
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeBatchPointSearchCursor;
/**
* Contributes the runtime operator for an unnest-map representing a BTree search.
@@ -157,7 +160,9 @@
nonMatchWriterFactory, dataset, jobGenParams.getIndexName(), lowKeyIndexes, highKeyIndexes,
jobGenParams.isLowKeyInclusive(), jobGenParams.isHighKeyInclusive(), propagateFilter,
nonFilterWriterFactory, minFilterFieldIndexes, maxFilterFieldIndexes, tupleFilterFactory, outputLimit,
- unnestMap.getGenerateCallBackProceedResultVar(), isPrimaryIndexPointSearch(op));
+ unnestMap.getGenerateCallBackProceedResultVar(),
+ isPrimaryIndexPointSearch(op, context.getPhysicalOptimizationConfig()),
+ DefaultTupleProjectorFactory.INSTANCE);
IOperatorDescriptor opDesc = btreeSearch.first;
opDesc.setSourceLocation(unnestMap.getSourceLocation());
@@ -168,8 +173,12 @@
builder.contributeGraphEdge(srcExchange, 0, unnestMap, 0);
}
- private boolean isPrimaryIndexPointSearch(ILogicalOperator op) {
- if (!isEqCondition || !isPrimaryIndex || !lowKeyVarList.equals(highKeyVarList)) {
+ /**
+ * Check whether we can use {@link LSMBTreeBatchPointSearchCursor} to perform point-lookups on the primary index
+ */
+ private boolean isPrimaryIndexPointSearch(ILogicalOperator op, PhysicalOptimizationConfig config) {
+ if (!config.isBatchLookupEnabled() || !isEqCondition || !isPrimaryIndex
+ || !lowKeyVarList.equals(highKeyVarList)) {
return false;
}
Index searchIndex = ((DataSourceIndex) idx).getIndex();
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/DefaultRuleSetFactory.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/DefaultRuleSetFactory.java
index ba42036..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..4665333 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,25 @@
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_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/jobgen/QueryLogicalExpressionJobGen.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
index b1d0b47..b8685d5 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
@@ -26,7 +26,6 @@
import org.apache.asterix.external.library.ExternalFunctionDescriptorProvider;
import org.apache.asterix.metadata.declared.MetadataProvider;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IExternalFunctionInfo;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionManager;
import org.apache.asterix.om.functions.IFunctionTypeInferer;
@@ -138,7 +137,7 @@
throws AlgebricksException {
IScalarEvaluatorFactory[] args = codegenArguments(expr, env, inputSchemas, context);
IFunctionDescriptor fd;
- if (expr.getFunctionInfo() instanceof IExternalFunctionInfo) {
+ if (expr.getFunctionInfo().isExternal()) {
// Expr is an external function
fd = ExternalFunctionDescriptorProvider.resolveExternalFunction(expr, env, context);
} else {
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/AnalysisUtil.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/AnalysisUtil.java
index 8b5c8bc..b6f79d9 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/AnalysisUtil.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/AnalysisUtil.java
@@ -18,32 +18,70 @@
*/
package org.apache.asterix.optimizer.base;
+import java.io.DataInputStream;
+import java.nio.ByteBuffer;
import java.util.ArrayList;
+import java.util.Arrays;
import java.util.List;
+import org.apache.asterix.common.dataflow.ICcApplicationContext;
import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.common.transactions.TxnId;
+import org.apache.asterix.common.utils.JobUtils;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
import org.apache.asterix.metadata.declared.DataSourceId;
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.metadata.declared.ResultSetDataSink;
+import org.apache.asterix.metadata.declared.ResultSetSinkId;
+import org.apache.asterix.om.base.IAObject;
import org.apache.asterix.om.functions.BuiltinFunctions;
import org.apache.asterix.optimizer.rules.am.AccessMethodUtils;
+import org.apache.asterix.runtime.job.listener.JobEventListenerFactory;
+import org.apache.asterix.translator.ResultMetadata;
+import org.apache.asterix.translator.SessionConfig;
import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.compiler.api.ICompiler;
+import org.apache.hyracks.algebricks.compiler.api.ICompilerFactory;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractDataSourceOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.WindowOperator;
+import org.apache.hyracks.algebricks.core.algebra.plan.ALogicalPlanImpl;
+import org.apache.hyracks.algebricks.core.rewriter.base.IRuleSetKind;
+import org.apache.hyracks.api.comm.IFrame;
+import org.apache.hyracks.api.comm.VSizeFrame;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.api.result.IResultSetReader;
+import org.apache.hyracks.api.result.ResultSetId;
+import org.apache.hyracks.control.nc.resources.memory.FrameManager;
+import org.apache.hyracks.data.std.util.ByteArrayAccessibleInputStream;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
public class AnalysisUtil {
+
+ private static final List<FunctionIdentifier> fieldAccessFunctions =
+ Arrays.asList(BuiltinFunctions.GET_DATA, BuiltinFunctions.GET_HANDLE, BuiltinFunctions.TYPE_OF);
+
/*
* If the first child of op is of type opType, then it returns that child,
* o/w returns null.
@@ -204,11 +242,86 @@
return true;
}
- private static List<FunctionIdentifier> fieldAccessFunctions = new ArrayList<>();
+ public static List<List<IAObject>> runQuery(Mutable<ILogicalOperator> topOp, List<LogicalVariable> resultVars,
+ IOptimizationContext queryOptCtx, IRuleSetKind ruleSetKind) throws AlgebricksException {
- static {
- fieldAccessFunctions.add(BuiltinFunctions.GET_DATA);
- fieldAccessFunctions.add(BuiltinFunctions.GET_HANDLE);
- fieldAccessFunctions.add(BuiltinFunctions.TYPE_OF);
+ MetadataProvider metadataProvider = (MetadataProvider) queryOptCtx.getMetadataProvider();
+ ICcApplicationContext appCtx = metadataProvider.getApplicationContext();
+ TxnId mainTxnId = metadataProvider.getTxnId();
+ try {
+ TxnId newTxnId = metadataProvider.getTxnIdFactory().create();
+ metadataProvider.setTxnId(newTxnId);
+
+ IVariableTypeEnvironment topOpTypeEnv = queryOptCtx.getOutputTypeEnvironment(topOp.getValue());
+ SerializerDeserializerProvider serdeProvider = SerializerDeserializerProvider.INSTANCE;
+
+ int nFields = resultVars.size();
+ List<Mutable<ILogicalExpression>> resultExprList = new ArrayList<>(nFields);
+ List<ISerializerDeserializer<?>> resultSerdeList = new ArrayList<>(nFields);
+
+ for (LogicalVariable var : resultVars) {
+ Object varType = topOpTypeEnv.getVarType(var);
+ if (varType == null) {
+ throw new IllegalArgumentException("Cannot determine type of " + var);
+ }
+ resultSerdeList.add(serdeProvider.getSerializerDeserializer(varType));
+ resultExprList.add(new MutableObject<>(new VariableReferenceExpression(var)));
+ }
+
+ ResultMetadata resultMetadata = new ResultMetadata(SessionConfig.OutputFormat.ADM);
+ ResultSetId resultSetId = new ResultSetId(metadataProvider.getResultSetIdCounter().getAndInc());
+ ResultSetSinkId rssId = new ResultSetSinkId(resultSetId);
+ ResultSetDataSink sink = new ResultSetDataSink(rssId, null);
+
+ DistributeResultOperator resultOp = new DistributeResultOperator(resultExprList, sink, resultMetadata);
+ resultOp.getInputs().add(topOp);
+ queryOptCtx.computeAndSetTypeEnvironmentForOperator(resultOp);
+
+ MutableObject<ILogicalOperator> newResultOpRef = new MutableObject<>(resultOp);
+
+ ICompilerFactory compilerFactory = (ICompilerFactory) queryOptCtx.getCompilerFactory();
+ ICompiler compiler =
+ compilerFactory.createCompiler(new ALogicalPlanImpl(newResultOpRef), queryOptCtx, ruleSetKind);
+ compiler.optimize();
+
+ JobSpecification jobSpec = compiler.createJob(appCtx, new JobEventListenerFactory(newTxnId, false));
+
+ JobId jobId = JobUtils.runJob(appCtx.getHcc(), jobSpec, true);
+
+ IResultSetReader resultSetReader = appCtx.getResultSet().createReader(jobId, resultSetId);
+ FrameManager frameManager = new FrameManager(queryOptCtx.getPhysicalOptimizationConfig().getFrameSize());
+ IFrame frame = new VSizeFrame(frameManager);
+
+ FrameTupleAccessor fta = new FrameTupleAccessor(null);
+ ByteArrayAccessibleInputStream bais = new ByteArrayAccessibleInputStream(frame.getBuffer().array(), 0, 0);
+ DataInputStream dis = new DataInputStream(bais);
+ List<List<IAObject>> result = new ArrayList<>();
+
+ while (resultSetReader.read(frame) > 0) {
+ ByteBuffer buffer = frame.getBuffer();
+ fta.reset(buffer);
+ int nTuples = fta.getTupleCount();
+ for (int tupleIdx = 0; tupleIdx < nTuples; tupleIdx++) {
+ int tupleStart = fta.getTupleStartOffset(tupleIdx);
+ int tupleEnd = fta.getTupleEndOffset(tupleIdx);
+ bais.setContent(buffer.array(), tupleStart, tupleEnd - tupleStart);
+
+ List<IAObject> values = new ArrayList<>(nFields);
+ for (int fieldIdx = 0; fieldIdx < nFields; fieldIdx++) {
+ IAObject value = (IAObject) resultSerdeList.get(fieldIdx).deserialize(dis);
+ values.add(value);
+ }
+ result.add(values);
+ }
+ }
+
+ return result;
+ } catch (AlgebricksException e) {
+ throw e;
+ } catch (Exception e) {
+ throw new AlgebricksException(e);
+ } finally {
+ metadataProvider.setTxnId(mainTxnId);
+ }
}
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/AsterixOptimizationContext.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/AsterixOptimizationContext.java
index 51c03eb..7b2dd7a 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/AsterixOptimizationContext.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/AsterixOptimizationContext.java
@@ -31,6 +31,7 @@
import org.apache.hyracks.algebricks.core.algebra.expressions.IMissableTypeComputer;
import org.apache.hyracks.algebricks.core.algebra.prettyprint.IPlanPrettyPrinter;
import org.apache.hyracks.algebricks.core.rewriter.base.AlgebricksOptimizationContext;
+import org.apache.hyracks.algebricks.core.rewriter.base.IOptimizationContextFactory;
import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
import org.apache.hyracks.api.exceptions.IWarningCollector;
@@ -39,17 +40,24 @@
public final class AsterixOptimizationContext extends AlgebricksOptimizationContext {
- private final Int2ObjectMap<Set<DataSource>> dataSourceMap = new Int2ObjectOpenHashMap<>();
+ private final Int2ObjectOpenHashMap<Set<DataSource>> dataSourceMap;
- public AsterixOptimizationContext(int varCounter, IExpressionEvalSizeComputer expressionEvalSizeComputer,
+ public AsterixOptimizationContext(IOptimizationContextFactory optContextFactory, int varCounter,
+ IExpressionEvalSizeComputer expressionEvalSizeComputer,
IMergeAggregationExpressionFactory mergeAggregationExpressionFactory,
IExpressionTypeComputer expressionTypeComputer, IMissableTypeComputer nullableTypeComputer,
IConflictingTypeResolver conflictingTypeResovler, PhysicalOptimizationConfig physicalOptimizationConfig,
AlgebricksPartitionConstraint clusterLocations, IPlanPrettyPrinter prettyPrinter,
IWarningCollector warningCollector) {
- super(varCounter, expressionEvalSizeComputer, mergeAggregationExpressionFactory, expressionTypeComputer,
- nullableTypeComputer, conflictingTypeResovler, physicalOptimizationConfig, clusterLocations,
- prettyPrinter, warningCollector);
+ super(optContextFactory, varCounter, expressionEvalSizeComputer, mergeAggregationExpressionFactory,
+ expressionTypeComputer, nullableTypeComputer, conflictingTypeResovler, physicalOptimizationConfig,
+ clusterLocations, prettyPrinter, warningCollector);
+ dataSourceMap = new Int2ObjectOpenHashMap<>();
+ }
+
+ public AsterixOptimizationContext(AsterixOptimizationContext from) {
+ super(from);
+ dataSourceMap = from.dataSourceMap.clone();
}
public void addDataSource(DataSource dataSource) {
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/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/ConstantFoldingRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
index 10aedc7..9fe1ba4 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
@@ -29,7 +29,6 @@
import org.apache.asterix.common.dataflow.ICcApplicationContext;
import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.common.exceptions.ErrorCode;
-import org.apache.asterix.common.exceptions.NoOpWarningCollector;
import org.apache.asterix.common.exceptions.WarningCollector;
import org.apache.asterix.dataflow.data.common.ExpressionTypeComputer;
import org.apache.asterix.dataflow.data.nontagged.MissingWriterFactory;
@@ -48,7 +47,6 @@
import org.apache.asterix.om.base.IAObject;
import org.apache.asterix.om.constants.AsterixConstantValue;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IExternalFunctionInfo;
import org.apache.asterix.om.typecomputer.impl.TypeComputeUtils;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.ATypeTag;
@@ -90,11 +88,14 @@
import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.serializer.ResultSerializerFactoryProvider;
+import org.apache.hyracks.algebricks.runtime.writers.PrinterBasedWriterFactory;
import org.apache.hyracks.api.application.IServiceContext;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.NoOpWarningCollector;
import org.apache.hyracks.api.exceptions.Warning;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -151,8 +152,9 @@
jobGenCtx = new JobGenContext(null, metadataProvider, appCtx, SerializerDeserializerProvider.INSTANCE,
BinaryHashFunctionFactoryProvider.INSTANCE, BinaryHashFunctionFamilyProvider.INSTANCE,
BinaryComparatorFactoryProvider.INSTANCE, TypeTraitProvider.INSTANCE, BinaryBooleanInspector.FACTORY,
- BinaryIntegerInspector.FACTORY, ADMPrinterFactoryProvider.INSTANCE, MissingWriterFactory.INSTANCE,
- NullWriterFactory.INSTANCE, UnnestingPositionWriterFactory.INSTANCE, null,
+ BinaryIntegerInspector.FACTORY, ADMPrinterFactoryProvider.INSTANCE, PrinterBasedWriterFactory.INSTANCE,
+ ResultSerializerFactoryProvider.INSTANCE, MissingWriterFactory.INSTANCE, NullWriterFactory.INSTANCE,
+ UnnestingPositionWriterFactory.INSTANCE, null,
new ExpressionRuntimeProvider(new QueryLogicalExpressionJobGen(metadataProvider.getFunctionManager())),
ExpressionTypeComputer.INSTANCE, null, null, null, null, GlobalConfig.DEFAULT_FRAME_SIZE, null,
NoOpWarningCollector.INSTANCE, 0, new PhysicalOptimizationConfig());
@@ -218,10 +220,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 +237,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 +249,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());
@@ -390,7 +392,7 @@
private boolean canConstantFold(ScalarFunctionCallExpression function) throws AlgebricksException {
// skip external functions because they're not available at compile time (on CC)
IFunctionInfo fi = function.getFunctionInfo();
- if (fi instanceof IExternalFunctionInfo) {
+ if (fi.isExternal()) {
return false;
}
// skip all functions that would produce records/arrays/multisets (derived types) in their open format
@@ -427,7 +429,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 +441,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 +467,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..09ad4d1 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
@@ -27,6 +27,7 @@
import java.util.Objects;
import java.util.Set;
import java.util.stream.Collectors;
+import java.util.stream.Stream;
import org.apache.asterix.algebra.operators.CommitOperator;
import org.apache.asterix.common.config.DatasetConfig.DatasetType;
@@ -190,18 +191,21 @@
metaType = (ARecordType) mp.findType(dataset.getMetaItemTypeDataverseName(), dataset.getMetaItemTypeName());
}
List<Index> indexes = mp.getDatasetIndexes(dataset.getDataverseName(), dataset.getDatasetName());
+ Stream<Index> indexStream = indexes.stream();
+ indexStream = indexStream.filter(index -> index.getIndexType() != IndexType.SAMPLE);
if (primaryIndexModificationOp.getOperation() == Kind.INSERT && !primaryIndexModificationOp.isBulkload()) {
// for insert, primary key index is handled together when primary index
- indexes = indexes.stream().filter(index -> !index.isPrimaryKeyIndex()).collect(Collectors.toList());
+ indexStream = indexStream.filter(index -> !index.isPrimaryKeyIndex());
}
-
- // Set the top operator pointer to the primary IndexInsertDeleteOperator
- ILogicalOperator currentTop = primaryIndexModificationOp;
+ indexes = indexStream.collect(Collectors.toList());
// Put an n-gram or a keyword index in the later stage of index-update,
// since TokenizeOperator needs to be involved.
Collections.sort(indexes, (o1, o2) -> o1.getIndexType().ordinal() - o2.getIndexType().ordinal());
+ // Set the top operator pointer to the primary IndexInsertDeleteOperator
+ ILogicalOperator currentTop = primaryIndexModificationOp;
+
// At this point, we have the data type info, and the indexes info as well
int secondaryIndexTotalCnt = indexes.size() - 1;
if (secondaryIndexTotalCnt > 0) {
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/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/PushFieldAccessRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushFieldAccessRule.java
index c82aa33..9070de1 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushFieldAccessRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushFieldAccessRule.java
@@ -22,6 +22,7 @@
import java.util.HashSet;
import java.util.LinkedList;
import java.util.List;
+import java.util.Set;
import org.apache.asterix.algebra.base.OperatorAnnotation;
import org.apache.asterix.common.config.DatasetConfig.DatasetType;
@@ -176,11 +177,11 @@
return e1.equals(e2);
}
- private boolean pushDownFieldAccessRec(Mutable<ILogicalOperator> opRef, IOptimizationContext context,
+ private boolean pushDownFieldAccessRec(Mutable<ILogicalOperator> assignOpRef, IOptimizationContext context,
String finalAnnot) throws AlgebricksException {
- AssignOperator assignOp = (AssignOperator) opRef.getValue();
- Mutable<ILogicalOperator> opRef2 = assignOp.getInputs().get(0);
- AbstractLogicalOperator inputOp = (AbstractLogicalOperator) opRef2.getValue();
+ AssignOperator assignOp = (AssignOperator) assignOpRef.getValue();
+ Mutable<ILogicalOperator> inputOpRef = assignOp.getInputs().get(0);
+ AbstractLogicalOperator inputOp = (AbstractLogicalOperator) inputOpRef.getValue();
// If it's not an indexed field, it is pushed so that scan can be rewritten into index search.
if (inputOp.getOperatorTag() == LogicalOperatorTag.PROJECT
|| context.checkAndAddToAlreadyCompared(assignOp, inputOp)
@@ -196,24 +197,32 @@
return false;
}
if (testAndModifyRedundantOp(assignOp, inputOp)) {
- pushDownFieldAccessRec(opRef2, context, finalAnnot);
+ pushDownFieldAccessRec(inputOpRef, context, finalAnnot);
return true;
}
- HashSet<LogicalVariable> usedInAccess = new HashSet<>();
+ Set<LogicalVariable> usedInAccess = new HashSet<>();
VariableUtilities.getUsedVariables(assignOp, usedInAccess);
-
- HashSet<LogicalVariable> produced2 = new HashSet<>();
+ if (usedInAccess.isEmpty()) {
+ return false;
+ }
+ Set<LogicalVariable> produced = new HashSet<>();
+ ILogicalOperator dataScanOp =
+ getDataScanOp(assignOpRef, assignOp, inputOpRef, inputOp, usedInAccess, produced, context);
+ if (dataScanOp != null) {
+ // this means the assign op is next to the data-scan op (either was moved or already next to data-scan)
+ // we just need to try replacing field access by the primary key if it refers to one
+ boolean assignMoved = inputOp != dataScanOp;
+ return rewriteFieldAccessToPK(context, finalAnnot, assignOp, dataScanOp) || assignMoved;
+ }
+ produced.clear();
if (inputOp.getOperatorTag() == LogicalOperatorTag.GROUP) {
- VariableUtilities.getLiveVariables(inputOp, produced2);
+ VariableUtilities.getLiveVariables(inputOp, produced);
} else {
- VariableUtilities.getProducedVariables(inputOp, produced2);
+ VariableUtilities.getProducedVariables(inputOp, produced);
}
boolean pushItDown = false;
HashSet<LogicalVariable> inter = new HashSet<>(usedInAccess);
- if (inter.isEmpty()) { // ground value
- return false;
- }
- inter.retainAll(produced2);
+ inter.retainAll(produced);
if (inter.isEmpty()) {
pushItDown = true;
} else if (inputOp.getOperatorTag() == LogicalOperatorTag.GROUP) {
@@ -254,18 +263,18 @@
if (inputOp.getOperatorTag() == LogicalOperatorTag.NESTEDTUPLESOURCE) {
Mutable<ILogicalOperator> childOfSubplan =
((NestedTupleSourceOperator) inputOp).getDataSourceReference().getValue().getInputs().get(0);
- pushAccessDown(opRef, inputOp, childOfSubplan, context, finalAnnot);
+ pushAccessDown(assignOpRef, inputOp, childOfSubplan, context, finalAnnot);
return true;
}
if (inputOp.getInputs().size() == 1 && !inputOp.hasNestedPlans()) {
- pushAccessDown(opRef, inputOp, inputOp.getInputs().get(0), context, finalAnnot);
+ pushAccessDown(assignOpRef, inputOp, inputOp.getInputs().get(0), context, finalAnnot);
return true;
} else {
for (Mutable<ILogicalOperator> inp : inputOp.getInputs()) {
HashSet<LogicalVariable> v2 = new HashSet<>();
VariableUtilities.getLiveVariables(inp.getValue(), v2);
if (v2.containsAll(usedInAccess)) {
- pushAccessDown(opRef, inputOp, inp, context, finalAnnot);
+ pushAccessDown(assignOpRef, inputOp, inp, context, finalAnnot);
return true;
}
}
@@ -277,7 +286,7 @@
HashSet<LogicalVariable> v2 = new HashSet<>();
VariableUtilities.getLiveVariables(root.getValue(), v2);
if (v2.containsAll(usedInAccess)) {
- pushAccessDown(opRef, inputOp, root, context, finalAnnot);
+ pushAccessDown(assignOpRef, inputOp, root, context, finalAnnot);
return true;
}
}
@@ -286,73 +295,133 @@
return false;
} else {
// check if the accessed field is one of the partitioning key fields. If yes, we can equate the 2 variables
- if (inputOp.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
- DataSourceScanOperator scan = (DataSourceScanOperator) inputOp;
- IDataSource<DataSourceId> dataSource = (IDataSource<DataSourceId>) scan.getDataSource();
- byte dsType = ((DataSource) dataSource).getDatasourceType();
- if (dsType != DataSource.Type.INTERNAL_DATASET && dsType != DataSource.Type.EXTERNAL_DATASET) {
- return false;
+ return rewriteFieldAccessToPK(context, finalAnnot, assignOp, inputOp);
+ }
+ }
+
+ /**
+ * Tries to rewrite field access to its equivalent PK. For example, a data scan operator of dataset "ds" produces
+ * the following variables: $PK1, $PK2,.., $ds, ($meta_var?). Given field access: $$ds.getField("id") and given that
+ * the field "id" is one of the primary keys of ds, the field access $$ds.getField("id") is replaced by the primary
+ * key variable (one of the $PKs).
+ * @return true if the field access in the assign operator was replaced by the primary key variable.
+ */
+ private boolean rewriteFieldAccessToPK(IOptimizationContext context, String finalAnnot, AssignOperator assignOp,
+ ILogicalOperator inputOp) throws AlgebricksException {
+ if (inputOp.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
+ DataSourceScanOperator scan = (DataSourceScanOperator) inputOp;
+ IDataSource<DataSourceId> dataSource = (IDataSource<DataSourceId>) scan.getDataSource();
+ byte dsType = ((DataSource) dataSource).getDatasourceType();
+ if (dsType != DataSource.Type.INTERNAL_DATASET && dsType != DataSource.Type.EXTERNAL_DATASET) {
+ return false;
+ }
+ DataSourceId asid = dataSource.getId();
+ MetadataProvider mp = (MetadataProvider) context.getMetadataProvider();
+ Dataset dataset = mp.findDataset(asid.getDataverseName(), asid.getDatasourceName());
+ if (dataset == null) {
+ throw new CompilationException(ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE, scan.getSourceLocation(),
+ asid.getDatasourceName(), asid.getDataverseName());
+ }
+ if (dataset.getDatasetType() != DatasetType.INTERNAL) {
+ setAsFinal(assignOp, context, finalAnnot);
+ return false;
+ }
+
+ List<LogicalVariable> allVars = scan.getVariables();
+ LogicalVariable dataRecVarInScan = ((DataSource) dataSource).getDataRecordVariable(allVars);
+ LogicalVariable metaRecVarInScan = ((DataSource) dataSource).getMetaVariable(allVars);
+
+ // data part
+ String dataTypeName = dataset.getItemTypeName();
+ IAType dataType = mp.findType(dataset.getItemTypeDataverseName(), dataTypeName);
+ if (dataType.getTypeTag() != ATypeTag.OBJECT) {
+ return false;
+ }
+ ARecordType dataRecType = (ARecordType) dataType;
+ Pair<ILogicalExpression, List<String>> fieldPathAndVar = getFieldExpression(assignOp, dataRecType);
+ ILogicalExpression targetRecVar = fieldPathAndVar.first;
+ List<String> targetFieldPath = fieldPathAndVar.second;
+ boolean rewrite = false;
+ boolean fieldFromMeta = false;
+ if (sameRecords(targetRecVar, dataRecVarInScan)) {
+ rewrite = true;
+ } else {
+ // check meta part
+ IAType metaType = mp.findMetaType(dataset); // could be null
+ if (metaType != null && metaType.getTypeTag() == ATypeTag.OBJECT) {
+ fieldPathAndVar = getFieldExpression(assignOp, (ARecordType) metaType);
+ targetRecVar = fieldPathAndVar.first;
+ targetFieldPath = fieldPathAndVar.second;
+ if (sameRecords(targetRecVar, metaRecVarInScan)) {
+ rewrite = true;
+ fieldFromMeta = true;
+ }
}
- DataSourceId asid = dataSource.getId();
- MetadataProvider mp = (MetadataProvider) context.getMetadataProvider();
- Dataset dataset = mp.findDataset(asid.getDataverseName(), asid.getDatasourceName());
- if (dataset == null) {
- throw new CompilationException(ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE, scan.getSourceLocation(),
- asid.getDatasourceName(), asid.getDataverseName());
- }
- if (dataset.getDatasetType() != DatasetType.INTERNAL) {
+ }
+
+ if (rewrite) {
+ int p = DatasetUtil.getPositionOfPartitioningKeyField(dataset, targetFieldPath, fieldFromMeta);
+ if (p < 0) { // not one of the partitioning fields
setAsFinal(assignOp, context, finalAnnot);
return false;
}
-
- List<LogicalVariable> allVars = scan.getVariables();
- LogicalVariable dataRecVarInScan = ((DataSource) dataSource).getDataRecordVariable(allVars);
- LogicalVariable metaRecVarInScan = ((DataSource) dataSource).getMetaVariable(allVars);
-
- // data part
- String dataTypeName = dataset.getItemTypeName();
- IAType dataType = mp.findType(dataset.getItemTypeDataverseName(), dataTypeName);
- if (dataType.getTypeTag() != ATypeTag.OBJECT) {
- return false;
- }
- ARecordType dataRecType = (ARecordType) dataType;
- Pair<ILogicalExpression, List<String>> fieldPathAndVar = getFieldExpression(assignOp, dataRecType);
- ILogicalExpression targetRecVar = fieldPathAndVar.first;
- List<String> targetFieldPath = fieldPathAndVar.second;
- boolean rewrite = false;
- boolean fieldFromMeta = false;
- if (sameRecords(targetRecVar, dataRecVarInScan)) {
- rewrite = true;
- } else {
- // check meta part
- IAType metaType = mp.findMetaType(dataset); // could be null
- if (metaType != null && metaType.getTypeTag() == ATypeTag.OBJECT) {
- fieldPathAndVar = getFieldExpression(assignOp, (ARecordType) metaType);
- targetRecVar = fieldPathAndVar.first;
- targetFieldPath = fieldPathAndVar.second;
- if (sameRecords(targetRecVar, metaRecVarInScan)) {
- rewrite = true;
- fieldFromMeta = true;
- }
- }
- }
-
- if (rewrite) {
- int p = DatasetUtil.getPositionOfPartitioningKeyField(dataset, targetFieldPath, fieldFromMeta);
- if (p < 0) { // not one of the partitioning fields
- setAsFinal(assignOp, context, finalAnnot);
- return false;
- }
- LogicalVariable keyVar = scan.getVariables().get(p);
- VariableReferenceExpression keyVarRef = new VariableReferenceExpression(keyVar);
- keyVarRef.setSourceLocation(targetRecVar.getSourceLocation());
- assignOp.getExpressions().get(0).setValue(keyVarRef);
- return true;
- }
+ LogicalVariable keyVar = scan.getVariables().get(p);
+ VariableReferenceExpression keyVarRef = new VariableReferenceExpression(keyVar);
+ keyVarRef.setSourceLocation(targetRecVar.getSourceLocation());
+ assignOp.getExpressions().get(0).setValue(keyVarRef);
+ return true;
}
- setAsFinal(assignOp, context, finalAnnot);
- return false;
}
+ setAsFinal(assignOp, context, finalAnnot);
+ return false;
+ }
+
+ /**
+ * Looks for a data scan operator where the data scan operator is below only assign operators. Then, if
+ * applicable, the assign operator is moved down and placed above the data-scan.
+ *
+ * @return the data scan operator if it exists below multiple assign operators only and the assign operator is now
+ * above the data-scan.
+ */
+ private ILogicalOperator getDataScanOp(Mutable<ILogicalOperator> assignOpRef, AssignOperator assignOp,
+ Mutable<ILogicalOperator> assignInputRef, ILogicalOperator assignInput, Set<LogicalVariable> usedInAssign,
+ Set<LogicalVariable> producedByInput, IOptimizationContext context) throws AlgebricksException {
+ ILogicalOperator firstInput = assignInput;
+ while (assignInput.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+ if (isRedundantAssign(assignOp, assignInput)) {
+ return null;
+ }
+ assignInputRef = assignInput.getInputs().get(0);
+ assignInput = assignInputRef.getValue();
+ }
+ if (assignInput.getOperatorTag() != LogicalOperatorTag.DATASOURCESCAN) {
+ return null;
+ }
+ VariableUtilities.getProducedVariables(assignInput, producedByInput);
+ if (!producedByInput.containsAll(usedInAssign)) {
+ return null;
+ }
+ if (firstInput == assignInput) {
+ // the input to the assign operator is already a data-scan
+ return assignInput;
+ }
+ ILogicalOperator op = firstInput;
+ // to make the behaviour the same as the recursive call, make sure to add the intermediate assigns to the
+ // already compared set
+ while (op.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+ context.checkAndAddToAlreadyCompared(assignOp, op);
+ op = op.getInputs().get(0).getValue();
+ }
+ // add the data-scan to the already compared set
+ context.checkAndAddToAlreadyCompared(assignOp, assignInput);
+ // move the assign op down, place it above the data-scan
+ assignOpRef.setValue(firstInput);
+ List<Mutable<ILogicalOperator>> assignInputs = assignOp.getInputs();
+ assignInputs.get(0).setValue(assignInput);
+ assignInputRef.setValue(assignOp);
+ context.computeAndSetTypeEnvironmentForOperator(assignOp);
+ context.computeAndSetTypeEnvironmentForOperator(firstInput);
+ return assignInput;
}
/**
@@ -398,12 +467,9 @@
}
private boolean testAndModifyRedundantOp(AssignOperator access, AbstractLogicalOperator op2) {
- if (op2.getOperatorTag() != LogicalOperatorTag.ASSIGN) {
- return false;
- }
- AssignOperator a2 = (AssignOperator) op2;
- ILogicalExpression accessExpr0 = getFirstExpr(access);
- if (accessExpr0.equals(getFirstExpr(a2))) {
+ if (isRedundantAssign(access, op2)) {
+ AssignOperator a2 = (AssignOperator) op2;
+ ILogicalExpression accessExpr0 = getFirstExpr(access);
VariableReferenceExpression varRef = new VariableReferenceExpression(a2.getVariables().get(0));
varRef.setSourceLocation(accessExpr0.getSourceLocation());
access.getExpressions().get(0).setValue(varRef);
@@ -413,6 +479,14 @@
}
}
+ private static boolean isRedundantAssign(AssignOperator assignOp, ILogicalOperator inputOp) {
+ if (inputOp.getOperatorTag() != LogicalOperatorTag.ASSIGN) {
+ return false;
+ }
+ ILogicalExpression assignOpExpr = getFirstExpr(assignOp);
+ return assignOpExpr.equals(getFirstExpr((AssignOperator) inputOp));
+ }
+
// indirect recursivity with pushDownFieldAccessRec
private void pushAccessDown(Mutable<ILogicalOperator> fldAccessOpRef, ILogicalOperator op2,
Mutable<ILogicalOperator> inputOfOp2, IOptimizationContext context, String finalAnnot)
@@ -429,8 +503,7 @@
pushDownFieldAccessRec(inputOfOp2, context, finalAnnot);
}
- private ILogicalExpression getFirstExpr(AssignOperator assign) {
+ 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/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/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/SweepIllegalNonfunctionalFunctions.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SweepIllegalNonfunctionalFunctions.java
index 2509010..2a6604e 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SweepIllegalNonfunctionalFunctions.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SweepIllegalNonfunctionalFunctions.java
@@ -61,6 +61,7 @@
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.SwitchOperator;
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;
@@ -205,6 +206,11 @@
}
@Override
+ public Void visitSwitchOperator(SwitchOperator op, Void arg) throws AlgebricksException {
+ return null;
+ }
+
+ @Override
public Void visitMaterializeOperator(MaterializeOperator op, Void arg) throws AlgebricksException {
return null;
}
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..9c69902 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,14 +24,17 @@
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.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.LinkedHashMap;
import java.util.List;
+import java.util.Map;
import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
import org.apache.asterix.algebra.operators.physical.ExternalDataLookupPOperator;
import org.apache.asterix.common.annotations.AbstractExpressionAnnotationWithIndexNames;
@@ -61,6 +64,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;
@@ -85,6 +89,7 @@
import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractLogicalExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionTypeComputer;
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.UnnestingFunctionCallExpression;
@@ -93,6 +98,7 @@
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.functions.IFunctionInfo;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
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.AbstractLogicalOperator.ExecutionMode;
@@ -147,6 +153,21 @@
BuiltinFunctions.YEAR_MONTH_DURATION_DEFAULT_NULL_CONSTRUCTOR,
BuiltinFunctions.UUID_DEFAULT_NULL_CONSTRUCTOR, BuiltinFunctions.BINARY_BASE64_DEFAULT_NULL_CONSTRUCTOR);
+ // TODO (GLENN): We can definitely expand the whitelist here...
+ private final static Map<FunctionIdentifier, Set<Integer>> INDEX_USE_ON_FUNCTION_CALL_WHITELIST;
+ private final static Set<Integer> ALL_INDEX_FUNCTION_ARGUMENTS = Collections.emptySet();
+ static {
+ INDEX_USE_ON_FUNCTION_CALL_WHITELIST = new HashMap<>();
+ INDEX_USE_ON_FUNCTION_CALL_WHITELIST.put(BuiltinFunctions.RECORD_ADD, Set.of(0));
+ INDEX_USE_ON_FUNCTION_CALL_WHITELIST.put(BuiltinFunctions.ADD_FIELDS, Set.of(0));
+ INDEX_USE_ON_FUNCTION_CALL_WHITELIST.put(BuiltinFunctions.RECORD_REMOVE, Set.of(0));
+ INDEX_USE_ON_FUNCTION_CALL_WHITELIST.put(BuiltinFunctions.RECORD_RENAME, Set.of(0));
+ INDEX_USE_ON_FUNCTION_CALL_WHITELIST.put(BuiltinFunctions.REMOVE_FIELDS, Set.of(0));
+ INDEX_USE_ON_FUNCTION_CALL_WHITELIST.put(BuiltinFunctions.RECORD_CONCAT, ALL_INDEX_FUNCTION_ARGUMENTS);
+ }
+
+ 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 +453,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 +985,7 @@
leftOuterMissingValue, context, sourceLoc);
}
+ OperatorManipulationUtil.copyCardCostAnnotations(joinRef.getValue(), finalOp);
joinRef.setValue(finalOp);
return true;
}
@@ -1735,10 +1760,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 +1962,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 +2024,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 +2384,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 +2873,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 +2920,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 +2939,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 +2951,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 +2980,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 =
@@ -2987,12 +3007,25 @@
isByName = true;
}
if (isFieldAccess) {
- LogicalVariable sourceVar =
- ((VariableReferenceExpression) funcExpr.getArguments().get(0).getValue()).getVariableReference();
- if (sourceVar.equals(metaVar)) {
- fieldSource.setValue(1);
+ ILogicalExpression funcExprArg0 = funcExpr.getArguments().get(0).getValue();
+ MutableInt sourceIndicator = new MutableInt(0);
+ LogicalVariable sourceVar;
+ if (funcExprArg0.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+ // This might be a field-access on an indexable-function-call (or nested indexable-function-calls).
+ List<LogicalVariable> foundDatasourceVariables = new ArrayList<>();
+ if (canUseIndexOnFunction((AbstractFunctionCallExpression) funcExprArg0, sourceIndicator,
+ foundDatasourceVariables, optFuncExpr, op.computeInputTypeEnvironment(context), context)) {
+ // TODO (GLENN): In the case of OBJECT_CONCAT w/ potentially multiple datasource variables, we
+ // will not explore each variable. This method definitely needs refactoring in the
+ // future to handle such a case.
+ sourceVar = foundDatasourceVariables.get(0);
+ } else {
+ return NO_FIELD_NAME;
+ }
+ } else if (funcExprArg0.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+ return NO_FIELD_NAME;
} else {
- fieldSource.setValue(0);
+ sourceVar = ((VariableReferenceExpression) funcExprArg0).getVariableReference();
}
if (optFuncExpr != null) {
optFuncExpr.setLogicalExpr(funcVarIndex, parentFuncExpr);
@@ -3024,43 +3057,48 @@
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 };
}
}
}
if (assignAndExpressionIndexes != null && assignAndExpressionIndexes[0] > -1) {
//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);
+ // Is the next operator composed of functions that are not a field access? If so, do not recurse.
+ ILogicalOperator nextOp = subTree.getAssignsAndUnnests().get(assignAndExpressionIndexes[0]);
+ boolean isIndexOnFunction = false;
+ if (nextOp.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+ AssignOperator nextAssignOp = (AssignOperator) nextOp;
+ ILogicalExpression leadingArgumentExpr = nextAssignOp.getExpressions().get(0).getValue();
+ if (leadingArgumentExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+ IVariableTypeEnvironment typeEnv = nextAssignOp.computeInputTypeEnvironment(context);
+ isIndexOnFunction = canUseIndexOnFunction((AbstractFunctionCallExpression) leadingArgumentExpr,
+ sourceIndicator, new HashSet<>(), optFuncExpr, typeEnv, context);
+ }
+ }
- boolean isPreviousOperatorLegalUnnest = isUnnestOverVarAllowed && subTree.getAssignsAndUnnests()
- .get(assignAndExpressionIndexes[0]).getOperatorTag().equals(LogicalOperatorTag.UNNEST);
- if (parentFieldNames.isEmpty() && !isPreviousOperatorLegalUnnest) {
+ // Otherwise... recurse.
+ Pair<List<String>, Integer> parentFieldNames =
+ !isIndexOnFunction
+ ? getFieldNameAndStepsFromSubTree(optFuncExpr, subTree, assignAndExpressionIndexes[0],
+ assignAndExpressionIndexes[1], funcVarIndex, parentFuncExpr, context)
+ : NO_FIELD_NAME;
+
+ if (parentFieldNames.first.isEmpty() && !isIndexOnFunction) {
//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,30 +3106,38 @@
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));
+
+ if (!isIndexOnFunction) {
+ //add fieldName to the nested fieldName, return
+ if (nestedAccessFieldName != null) {
+ parentFieldNames.first.addAll(nestedAccessFieldName);
+ } else {
+ parentFieldNames.first.add(fieldName);
}
+ return (parentFieldNames);
+
} else {
- parentFieldNames.add(fieldName);
+ if (nestedAccessFieldName != null) {
+ return new Pair<>(nestedAccessFieldName, sourceIndicator.getValue());
+ } else {
+ return new Pair<>(new ArrayList<>(List.of(fieldName)), sourceIndicator.getValue());
+ }
}
- return (parentFieldNames);
}
if (optFuncExpr != null) {
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 +3147,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 +3177,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,
@@ -3214,4 +3259,73 @@
return funId.equals(FIELD_ACCESS_BY_NAME) || funId.equals(FIELD_ACCESS_BY_INDEX)
|| funId.equals(FIELD_ACCESS_NESTED);
}
+
+ /**
+ * If we are accessing some field through a function application (or series of function applications) of the
+ * following:
+ * <p><pre>
+ * | OBJECT_ADD | OBJECT_REMOVE | OBJECT_ADD_FIELDS |
+ * | OBJECT_CONCAT | OBJECT_RENAME | OBJECT_REMOVE_FIELDS |
+ * </pre>
+ * ...then we still might be able to use an index. Check the output type of applying our function(s) and verify
+ * that the input is a data source variable.
+ */
+ public static boolean canUseIndexOnFunction(AbstractFunctionCallExpression funcExpr, MutableInt sourceIndicator,
+ Collection<LogicalVariable> foundDatasourceVariables, IOptimizableFuncExpr optFuncExpr,
+ IVariableTypeEnvironment typeEnv, IOptimizationContext context) throws AlgebricksException {
+ FunctionIdentifier functionID = funcExpr.getFunctionIdentifier();
+ if (!INDEX_USE_ON_FUNCTION_CALL_WHITELIST.containsKey(functionID)) {
+ return false;
+ }
+
+ // Our output should be an object (this is more of a sanity check given that we have a whitelist).
+ IExpressionTypeComputer expressionTypeComputer = context.getExpressionTypeComputer();
+ IMetadataProvider<?, ?> metadataProvider = context.getMetadataProvider();
+ IAType originalOutputType = (IAType) expressionTypeComputer.getType(funcExpr, metadataProvider, typeEnv);
+ IAType outputType = TypeComputeUtils.getActualType(originalOutputType);
+ ARecordType outputRecordType = TypeComputeUtils.extractRecordType(outputType);
+ if (outputRecordType == null) {
+ return false;
+ }
+
+ // Check the type of our input, according to record variables in each function's argument.
+ boolean isDataSourceVariableFound = false;
+ Set<Integer> indicesToCheck = INDEX_USE_ON_FUNCTION_CALL_WHITELIST.get(functionID);
+ if (indicesToCheck.equals(ALL_INDEX_FUNCTION_ARGUMENTS)) {
+ indicesToCheck = IntStream.range(0, funcExpr.getArguments().size()).boxed().collect(Collectors.toSet());
+ }
+ for (Integer functionCallArgumentIndex : indicesToCheck) {
+ ILogicalExpression inputRecordExpr = funcExpr.getArguments().get(functionCallArgumentIndex).getValue();
+ switch (inputRecordExpr.getExpressionTag()) {
+ case FUNCTION_CALL:
+ AbstractFunctionCallExpression arg0FuncExpr = (AbstractFunctionCallExpression) inputRecordExpr;
+ isDataSourceVariableFound |= canUseIndexOnFunction(arg0FuncExpr, sourceIndicator,
+ foundDatasourceVariables, optFuncExpr, typeEnv, context);
+ break;
+
+ case VARIABLE:
+ // Base case. We should be using a data source variable here.
+ VariableReferenceExpression inputRecordVarExpr = (VariableReferenceExpression) inputRecordExpr;
+ LogicalVariable inputRecordVar = inputRecordVarExpr.getVariableReference();
+ if (optFuncExpr != null) {
+ for (int i = 0; i < optFuncExpr.getNumLogicalVars(); i++) {
+ OptimizableOperatorSubTree operatorSubTree = optFuncExpr.getOperatorSubTree(i);
+ if (operatorSubTree == null) {
+ continue;
+ }
+ if (operatorSubTree.getDataSourceVariables().stream().anyMatch(inputRecordVar::equals)) {
+ OptimizableOperatorSubTree.RecordTypeSource recordTypeSource =
+ operatorSubTree.getRecordTypeFor(inputRecordVar);
+ foundDatasourceVariables.add(inputRecordVar);
+ sourceIndicator.setValue(recordTypeSource.sourceIndicator);
+ isDataSourceVariableFound = true;
+ break;
+ }
+ }
+ }
+ break;
+ }
+ }
+ return isDataSourceVariableFound;
+ }
}
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 5dd0135..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
@@ -753,7 +753,7 @@
} else {
leftOuterUnnestMapRequired = false;
}
-
+ AbstractUnnestMapOperator unnestMapOp;
if (conditionRef.getValue() != null) {
// The job gen parameters are transferred to the actual job gen
// via the UnnestMapOperator's function arguments.
@@ -765,7 +765,6 @@
new UnnestingFunctionCallExpression(primaryIndexSearch, primaryIndexFuncArgs);
primaryIndexSearchFunc.setSourceLocation(dataSourceOp.getSourceLocation());
primaryIndexSearchFunc.setReturnsUniqueValues(true);
- AbstractUnnestMapOperator unnestMapOp;
if (!leftOuterUnnestMapRequired) {
unnestMapOp = new UnnestMapOperator(scanVariables,
new MutableObject<ILogicalExpression>(primaryIndexSearchFunc), primaryIndexOutputTypes,
@@ -775,10 +774,7 @@
new MutableObject<ILogicalExpression>(primaryIndexSearchFunc), primaryIndexOutputTypes,
leftOuterMissingValue);
}
- unnestMapOp.setSourceLocation(dataSourceOp.getSourceLocation());
- indexSearchOp = unnestMapOp;
} else {
- AbstractUnnestMapOperator unnestMapOp;
if (!leftOuterUnnestMapRequired) {
unnestMapOp = new UnnestMapOperator(scanVariables,
((UnnestMapOperator) secondaryIndexUnnestOp).getExpressionRef(), primaryIndexOutputTypes,
@@ -788,11 +784,11 @@
((LeftOuterUnnestMapOperator) secondaryIndexUnnestOp).getExpressionRef(),
primaryIndexOutputTypes, leftOuterMissingValue);
}
- unnestMapOp.setSourceLocation(dataSourceOp.getSourceLocation());
- indexSearchOp = unnestMapOp;
}
- // TODO: shouldn't indexSearchOp execution mode be set to that of the input? the default is UNPARTITIONED
- indexSearchOp.getInputs().add(new MutableObject<>(inputOp));
+ unnestMapOp.setExecutionMode(ExecutionMode.PARTITIONED);
+ unnestMapOp.setSourceLocation(dataSourceOp.getSourceLocation());
+ unnestMapOp.getInputs().add(new MutableObject<>(inputOp));
+ indexSearchOp = unnestMapOp;
// Adds equivalence classes --- one equivalent class between a primary key
// variable and a record field-access expression.
@@ -800,6 +796,7 @@
metaRecordType, dataset, context);
}
+ OperatorManipulationUtil.copyCardCostAnnotations(dataSourceOp, indexSearchOp);
return indexSearchOp;
}
@@ -1024,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)) {
@@ -1037,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..3328127 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);
@@ -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,7 @@
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/JoinFromSubplanRewrite.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/JoinFromSubplanRewrite.java
index aa80462..f4fa0fc 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);
@@ -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..183eb46
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/EnumerateJoinsRule.java
@@ -0,0 +1,623 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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));
+ 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) { // there must be a select operator for CBO to do any optimization.
+ 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);
+ if (leftInput.getOperatorTag() == LogicalOperatorTag.SELECT) {
+ addCardCostAnnotations(leftInput, 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) {
+ // 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);
+ if (leftInput.getOperatorTag() == LogicalOperatorTag.SELECT) {
+ addCardCostAnnotations(leftInput, leftPlan);
+ }
+ joinOp.getInputs().get(0).setValue(leftInput);
+ addCardCostAnnotations(findDataSourceScanOperator(leftInput), leftPlan);
+ } else {
+ // join
+ totalNumberOfJoins.increment();
+ ILogicalOperator leftInput = joinOps.get(totalNumberOfJoins.intValue());
+ joinOp.getInputs().get(0).setValue(leftInput);
+ buildNewTree(allPlans.get(leftIndex), joinLeafInputsHashMap, joinOps, totalNumberOfJoins);
+ }
+
+ if (rightPlan.IsScanNode()) {
+ // leaf
+ ILogicalOperator rightInput = joinLeafInputsHashMap.get(rightPlan.getEmptyTupleSourceOp());
+ skipAllIndexes(rightPlan, rightInput);
+ if (rightInput.getOperatorTag() == LogicalOperatorTag.SELECT) {
+ addCardCostAnnotations(rightInput, rightPlan);
+ }
+ joinOp.getInputs().get(1).setValue(rightInput);
+ addCardCostAnnotations(findDataSourceScanOperator(rightInput), rightPlan);
+ } else {
+ // join
+ totalNumberOfJoins.increment();
+ ILogicalOperator rightInput = joinOps.get(totalNumberOfJoins.intValue());
+ joinOp.getInputs().get(1).setValue(rightInput);
+ buildNewTree(allPlans.get(rightIndex), joinLeafInputsHashMap, joinOps, totalNumberOfJoins);
+ }
+ }
+
+ // 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..822d824
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/EstimatedCostComputationVisitor.java
@@ -0,0 +1,368 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.SwitchOperator;
+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> visitSwitchOperator(SwitchOperator 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..d0c7ea1
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/JoinEnum.java
@@ -0,0 +1,783 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.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 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();
+ } 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);
+ } 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..43247d4
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/JoinNode.java
@@ -0,0 +1,982 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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() {
+ 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)) {
+ // 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 (selectivityAnnotation != null) {
+ sel = selectivityAnnotation.getSelectivity();
+ if (sel >= joinEnum.stats.SELECTIVITY_FOR_SECONDARY_INDEX_SELECTION) {
+ afce.putAnnotation(SkipSecondaryIndexSearchExpressionAnnotation
+ .newInstance(Collections.singleton(chosenIndex.getIndexName())));
+ } else {
+ buildIndexPlan();
+ }
+ }
+ }
+ }
+ }
+
+ 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..96bc412
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/Stats.java
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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;
+
+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
+ return 1.0;
+ }
+ if (leftIndex == idx1) {
+ return anno.getJoinProductivity() / card2;
+ } else {
+ return anno.getJoinProductivity() / 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 = getSelectivityFromAnnotation(
+ (AbstractFunctionCallExpression) afcExpr.getArguments().get(0).getValue(), join);
+ for (int i = 1; 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();
+ 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/optimizer/rules/pushdown/ExpectedSchemaNodeToIATypeTranslatorVisitor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/ExpectedSchemaNodeToIATypeTranslatorVisitor.java
index c746994..0da2c05 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/ExpectedSchemaNodeToIATypeTranslatorVisitor.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/ExpectedSchemaNodeToIATypeTranslatorVisitor.java
@@ -39,6 +39,7 @@
import org.apache.asterix.optimizer.rules.pushdown.schema.UnionExpectedSchemaNode;
import org.apache.asterix.runtime.projection.DataProjectionInfo;
import org.apache.asterix.runtime.projection.FunctionCallInformation;
+import org.apache.asterix.runtime.projection.ProjectionFiltrationWarningFactoryProvider;
/**
* This visitor translates the {@link IExpectedSchemaNode} to {@link IAType} record.
@@ -109,6 +110,7 @@
}
private FunctionCallInformation createFunctionCallInformation(IExpectedSchemaNode node) {
- return new FunctionCallInformation(node.getFunctionName(), node.getSourceLocation());
+ return new FunctionCallInformation(node.getFunctionName(), node.getSourceLocation(),
+ ProjectionFiltrationWarningFactoryProvider.TYPE_MISMATCH_FACTORY);
}
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/OperatorValueAccessPushdownVisitor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/OperatorValueAccessPushdownVisitor.java
index 6739384..05bc161 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/OperatorValueAccessPushdownVisitor.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/OperatorValueAccessPushdownVisitor.java
@@ -74,6 +74,7 @@
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.SwitchOperator;
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;
@@ -362,6 +363,12 @@
}
@Override
+ public Void visitSwitchOperator(SwitchOperator op, Void arg) throws AlgebricksException {
+ visitInputs(op);
+ return null;
+ }
+
+ @Override
public Void visitMaterializeOperator(MaterializeOperator op, Void arg) throws AlgebricksException {
visitInputs(op);
return null;
@@ -474,4 +481,4 @@
private void visitInputs(ILogicalOperator op) throws AlgebricksException {
visitInputs(op, null);
}
-}
\ No newline at end of file
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java
index 4ac44b4..55165ed 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java
@@ -80,6 +80,7 @@
import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
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.SwitchOperator;
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;
@@ -570,6 +571,11 @@
}
@Override
+ public ILogicalOperator visitSwitchOperator(SwitchOperator op, Void arg) throws AlgebricksException {
+ return visitSingleInputOperator(op);
+ }
+
+ @Override
public ILogicalOperator visitMaterializeOperator(MaterializeOperator op, Void arg) throws AlgebricksException {
return visitSingleInputOperator(op);
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineLeftNtsInSubplanJoinFlatteningVisitor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineLeftNtsInSubplanJoinFlatteningVisitor.java
index d4b7853..cd476f9 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineLeftNtsInSubplanJoinFlatteningVisitor.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineLeftNtsInSubplanJoinFlatteningVisitor.java
@@ -63,6 +63,7 @@
import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
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.SwitchOperator;
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;
@@ -297,6 +298,11 @@
}
@Override
+ public ILogicalOperator visitSwitchOperator(SwitchOperator op, Void arg) throws AlgebricksException {
+ return visitSingleInputOperator(op);
+ }
+
+ @Override
public ILogicalOperator visitMaterializeOperator(MaterializeOperator op, Void arg) throws AlgebricksException {
return visitSingleInputOperator(op);
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/SubplanSpecialFlatteningCheckVisitor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/SubplanSpecialFlatteningCheckVisitor.java
index a0bb4b6..8bb7502 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/SubplanSpecialFlatteningCheckVisitor.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/SubplanSpecialFlatteningCheckVisitor.java
@@ -48,6 +48,7 @@
import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
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.SwitchOperator;
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;
@@ -167,6 +168,11 @@
}
@Override
+ public Boolean visitSwitchOperator(SwitchOperator op, Void arg) throws AlgebricksException {
+ return visitInputs(op);
+ }
+
+ @Override
public Boolean visitMaterializeOperator(MaterializeOperator op, Void arg) throws AlgebricksException {
return visitInputs(op);
}
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/IStatementExecutor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutor.java
index 8a91059..b47d45d 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutor.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutor.java
@@ -51,7 +51,6 @@
import org.apache.hyracks.api.job.JobSpecification;
import org.apache.hyracks.api.result.ResultSetId;
-import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.node.ObjectNode;
@@ -137,6 +136,7 @@
private Profile profile;
private ProfileType profileType;
private long totalWarningsCount;
+ private long compileTime;
public long getCount() {
return count;
@@ -187,10 +187,18 @@
public void setProfileType(ProfileType profileType) {
this.profileType = profileType;
}
+
+ public void setCompileTime(long compileTime) {
+ this.compileTime = compileTime;
+ }
+
+ public long getCompileTime() {
+ return compileTime;
+ }
}
class Profile implements Serializable {
- private static final long serialVersionUID = 4813321148252768375L;
+ private static final long serialVersionUID = 4813321148252768376L;
private transient ObjectNode profile;
@@ -200,16 +208,15 @@
private void writeObject(ObjectOutputStream out) throws IOException {
ObjectMapper om = new ObjectMapper();
- out.writeUTF(om.writeValueAsString(profile));
+ byte[] bytes = om.writeValueAsBytes(profile);
+ out.writeInt(bytes.length);
+ out.write(bytes);
}
private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
ObjectMapper om = new ObjectMapper();
- JsonNode inNode = om.readTree(in.readUTF());
- if (!inNode.isObject()) {
- throw new IOException("Deserialization error");
- }
- profile = (ObjectNode) inNode;
+ int length = in.readInt();
+ profile = (ObjectNode) om.readTree(in.readNBytes(length));
}
public ObjectNode getProfile() {
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..daa1d2f 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
@@ -1716,8 +1716,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,6 +1748,32 @@
}
}
+ /**
+ * 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();
@@ -1902,8 +1927,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 +1942,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 +2027,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..0290acf 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;
@@ -37,6 +40,7 @@
import org.apache.asterix.lang.common.base.Expression;
import org.apache.asterix.lang.common.base.Expression.Kind;
import org.apache.asterix.lang.common.base.ILangExpression;
+import org.apache.asterix.lang.common.base.IVisitorExtension;
import org.apache.asterix.lang.common.base.Literal;
import org.apache.asterix.lang.common.clause.GroupbyClause;
import org.apache.asterix.lang.common.clause.LetClause;
@@ -325,6 +329,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 +584,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);
@@ -1029,6 +1041,13 @@
: super.visit(qe, tupSource);
}
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visit(IVisitorExtension ve, Mutable<ILogicalOperator> arg)
+ throws CompilationException {
+ // Language extensions should create a child of this class.
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, "Extension dispatch not implemented!");
+ }
+
// At this point "$x in list_expr" is a quantified expression:
// "some $y in list_expr satisfies $x = $y"
// Look for such quantified expression with a constant list_expr ([e1, e2, ... eN])
diff --git a/asterixdb/asterix-app/pom.xml b/asterixdb/asterix-app/pom.xml
index aac5bcf..9680667 100644
--- a/asterixdb/asterix-app/pom.xml
+++ b/asterixdb/asterix-app/pom.xml
@@ -267,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>
@@ -388,7 +389,7 @@
<profile>
<id>asterix-gerrit-asterix-app</id>
<properties>
- <test.excludes>**/SqlppExecutionWithCancellationTest.java,**/DmlTest.java,**/RepeatedTest.java,**/SqlppExecutionTest.java,**/AqlExecutionTest.java,**/*Compression*Test.java,**/*Ssl*Test.java</test.excludes>
+ <test.excludes>**/SqlppExecutionWithCancellationTest.java,**/DmlTest.java,**/RepeatedTest.java,**/SqlppExecutionTest.java,**/AqlExecutionTest.java,**/*Compression*Test.java,**/*Ssl*Test.java,**/Podman*.java</test.excludes>
<itest.excludes>**/*.java</itest.excludes>
</properties>
<build>
@@ -570,7 +571,6 @@
<dependency>
<groupId>com.e-movimento.tinytools</groupId>
<artifactId>privilegedaccessor</artifactId>
- <version>1.2.2</version>
<scope>test</scope>
</dependency>
<dependency>
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..41be44b 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
@@ -40,6 +40,7 @@
import org.apache.asterix.common.api.IResponsePrinter;
import org.apache.asterix.common.config.CompilerProperties;
import org.apache.asterix.common.config.OptimizationConfUtil;
+import org.apache.asterix.common.dataflow.ICcApplicationContext;
import org.apache.asterix.common.exceptions.ACIDException;
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.common.exceptions.CompilationException;
@@ -54,7 +55,6 @@
import org.apache.asterix.dataflow.data.common.MergeAggregationExpressionFactory;
import org.apache.asterix.dataflow.data.common.MissableTypeComputer;
import org.apache.asterix.dataflow.data.common.PartialAggregationTypeComputer;
-import org.apache.asterix.external.feed.watch.FeedActivityDetails;
import org.apache.asterix.formats.base.IDataFormat;
import org.apache.asterix.jobgen.QueryLogicalExpressionJobGen;
import org.apache.asterix.lang.common.base.IAstPrintVisitorFactory;
@@ -63,27 +63,18 @@
import org.apache.asterix.lang.common.base.IRewriterFactory;
import org.apache.asterix.lang.common.base.Statement;
import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
-import org.apache.asterix.lang.common.statement.FunctionDecl;
import org.apache.asterix.lang.common.statement.Query;
-import org.apache.asterix.lang.common.statement.StartFeedStatement;
-import org.apache.asterix.lang.common.statement.ViewDecl;
import org.apache.asterix.lang.common.struct.VarIdentifier;
-import org.apache.asterix.lang.common.util.FunctionUtil;
-import org.apache.asterix.lang.sqlpp.rewrites.SqlppQueryRewriter;
import org.apache.asterix.metadata.declared.MetadataProvider;
import org.apache.asterix.om.base.IAObject;
import org.apache.asterix.optimizer.base.AsterixOptimizationContext;
-import org.apache.asterix.optimizer.base.FuzzyUtils;
-import org.apache.asterix.optimizer.rules.DisjunctivePredicateToJoinRule;
-import org.apache.asterix.optimizer.rules.SetAsterixPhysicalOperatorsRule;
-import org.apache.asterix.optimizer.rules.util.EquivalenceClassUtils;
import org.apache.asterix.runtime.job.listener.JobEventListenerFactory;
import org.apache.asterix.translator.CompiledStatements.ICompiledDmlStatement;
import org.apache.asterix.translator.ExecutionPlans;
import org.apache.asterix.translator.IRequestParameters;
+import org.apache.asterix.translator.ResultMetadata;
import org.apache.asterix.translator.SessionConfig;
import org.apache.asterix.translator.SessionOutput;
-import org.apache.asterix.translator.SqlppExpressionToPlanTranslator;
import org.apache.asterix.utils.ResourceUtils;
import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
@@ -106,6 +97,8 @@
import org.apache.hyracks.algebricks.core.rewriter.base.IOptimizationContextFactory;
import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
import org.apache.hyracks.algebricks.data.IPrinterFactoryProvider;
+import org.apache.hyracks.algebricks.runtime.serializer.ResultSerializerFactoryProvider;
+import org.apache.hyracks.algebricks.runtime.writers.PrinterBasedWriterFactory;
import org.apache.hyracks.api.client.IClusterInfoCollector;
import org.apache.hyracks.api.client.IHyracksClientConnection;
import org.apache.hyracks.api.client.NodeControllerInfo;
@@ -121,7 +114,6 @@
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.ObjectWriter;
-import com.google.common.collect.ImmutableSet;
/**
* Provides helper methods for compilation of a query into a JobSpec and submission
@@ -133,29 +125,11 @@
public static final String PREFIX_INTERNAL_PARAMETERS = "_internal";
- // A white list of supported configurable parameters.
- private static final Set<String> CONFIGURABLE_PARAMETER_NAMES = ImmutableSet.of(
- CompilerProperties.COMPILER_JOINMEMORY_KEY, CompilerProperties.COMPILER_GROUPMEMORY_KEY,
- CompilerProperties.COMPILER_SORTMEMORY_KEY, CompilerProperties.COMPILER_WINDOWMEMORY_KEY,
- CompilerProperties.COMPILER_TEXTSEARCHMEMORY_KEY, CompilerProperties.COMPILER_PARALLELISM_KEY,
- CompilerProperties.COMPILER_SORT_PARALLEL_KEY, CompilerProperties.COMPILER_SORT_SAMPLES_KEY,
- CompilerProperties.COMPILER_EXTERNALSCANMEMORY_KEY, CompilerProperties.COMPILER_INDEXONLY_KEY,
- CompilerProperties.COMPILER_INTERNAL_SANITYCHECK_KEY,
- CompilerProperties.COMPILER_EXTERNAL_FIELD_PUSHDOWN_KEY, CompilerProperties.COMPILER_SUBPLAN_MERGE_KEY,
- CompilerProperties.COMPILER_SUBPLAN_NESTEDPUSHDOWN_KEY, CompilerProperties.COMPILER_ARRAYINDEX_KEY,
- CompilerProperties.COMPILER_MIN_MEMORY_ALLOCATION_KEY, FunctionUtil.IMPORT_PRIVATE_FUNCTIONS,
- FuzzyUtils.SIM_FUNCTION_PROP_NAME, FuzzyUtils.SIM_THRESHOLD_PROP_NAME,
- StartFeedStatement.WAIT_FOR_COMPLETION, FeedActivityDetails.FEED_POLICY_NAME,
- FeedActivityDetails.COLLECT_LOCATIONS, SqlppQueryRewriter.INLINE_WITH_OPTION,
- SqlppExpressionToPlanTranslator.REWRITE_IN_AS_OR_OPTION, "hash_merge", "output-record-type",
- DisjunctivePredicateToJoinRule.REWRITE_OR_AS_JOIN_OPTION,
- SetAsterixPhysicalOperatorsRule.REWRITE_ATTEMPT_BATCH_ASSIGN,
- EquivalenceClassUtils.REWRITE_INTERNAL_QUERYUID_PK, SqlppQueryRewriter.SQL_COMPAT_OPTION);
-
private final IRewriterFactory rewriterFactory;
private final IAstPrintVisitorFactory astPrintVisitorFactory;
private final ILangExpressionToPlanTranslatorFactory translatorFactory;
private final IRuleSetFactory ruleSetFactory;
+ private final Set<String> configurableParameterNames;
private final ExecutionPlans executionPlans;
public APIFramework(ILangCompilationProvider compilationProvider) {
@@ -163,6 +137,7 @@
this.astPrintVisitorFactory = compilationProvider.getAstPrintVisitorFactory();
this.translatorFactory = compilationProvider.getExpressionToPlanTranslatorFactory();
this.ruleSetFactory = compilationProvider.getRuleSetFactory();
+ this.configurableParameterNames = compilationProvider.getCompilerOptions();
executionPlans = new ExecutionPlans();
}
@@ -181,17 +156,21 @@
IConflictingTypeResolver conflictingTypeResolver, PhysicalOptimizationConfig physicalOptimizationConfig,
AlgebricksPartitionConstraint clusterLocations, IWarningCollector warningCollector) {
IPlanPrettyPrinter prettyPrinter = PlanPrettyPrinter.createStringPlanPrettyPrinter();
- return new AsterixOptimizationContext(varCounter, expressionEvalSizeComputer,
+ return new AsterixOptimizationContext(this, varCounter, expressionEvalSizeComputer,
mergeAggregationExpressionFactory, expressionTypeComputer, missableTypeComputer,
conflictingTypeResolver, physicalOptimizationConfig, clusterLocations, prettyPrinter,
warningCollector);
}
+
+ @Override
+ public IOptimizationContext cloneOptimizationContext(IOptimizationContext oc) {
+ return new AsterixOptimizationContext((AsterixOptimizationContext) oc);
+ }
}
- public Pair<IReturningStatement, Integer> reWriteQuery(List<FunctionDecl> declaredFunctions,
- List<ViewDecl> declaredViews, MetadataProvider metadataProvider, IReturningStatement q,
- SessionOutput output, boolean allowNonStoredUdfCalls, boolean inlineUdfsAndViews,
- Collection<VarIdentifier> externalVars, IWarningCollector warningCollector) throws CompilationException {
+ public Pair<IReturningStatement, Integer> reWriteQuery(LangRewritingContext langRewritingContext,
+ IReturningStatement q, SessionOutput output, boolean allowNonStoredUdfCalls, boolean inlineUdfsAndViews,
+ Collection<VarIdentifier> externalVars) throws CompilationException {
if (q == null) {
return null;
}
@@ -200,9 +179,7 @@
generateExpressionTree(q);
}
IQueryRewriter rw = rewriterFactory.createQueryRewriter();
- LangRewritingContext rwCtx = new LangRewritingContext(metadataProvider, declaredFunctions, declaredViews,
- warningCollector, q.getVarCounter());
- rw.rewrite(rwCtx, q, allowNonStoredUdfCalls, inlineUdfsAndViews, externalVars);
+ rw.rewrite(langRewritingContext, q, allowNonStoredUdfCalls, inlineUdfsAndViews, externalVars);
return new Pair<>(q, q.getVarCounter());
}
@@ -233,20 +210,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 +233,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 +273,7 @@
output.out().write(buf.toString());
} else {
if (isQuery || isLoad) {
- generateOptimizedLogicalPlan(plan, output.config().getPlanFormat());
+ generateOptimizedLogicalPlan(plan, output.config().getPlanFormat(), cboMode);
}
}
}
@@ -296,6 +294,42 @@
boolean printSignature = isQuery && requestParameters != null && requestParameters.isPrintSignature();
+ if (printSignature && !isExplainOnly) { //explainOnly adds the signature later
+ printer.addResultPrinter(SignaturePrinter.newInstance(executionPlans));
+ }
+
+ if (!conf.isGenerateJobSpec()) {
+ if (isQuery || isLoad) {
+ generateOptimizedLogicalPlan(plan, output.config().getPlanFormat(), cboMode);
+ }
+ return null;
+ }
+
+ JobEventListenerFactory jobEventListenerFactory =
+ new JobEventListenerFactory(txnId, metadataProvider.isWriteTransaction());
+ 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 = ccAppContext.getNodeJobTracker();
+ final AlgebricksAbsolutePartitionConstraint jobLocations =
+ getJobLocations(spec, nodeJobTracker, computationLocations);
+ final IClusterCapacity jobRequiredCapacity =
+ ResourceUtils.getRequiredCapacity(plan, jobLocations, physOptConf);
+ spec.setRequiredClusterCapacity(jobRequiredCapacity);
+ }
+ }
+
+ if (conf.is(SessionConfig.OOB_OPTIMIZED_LOGICAL_PLAN) || isExplainOnly) {
+ if (isQuery || isLoad) {
+ generateOptimizedLogicalPlan(plan, spec.getLogical2PhysicalMap(), output.config().getPlanFormat(),
+ cboMode);
+ }
+ }
+
if (isExplainOnly) {
printPlanAsResult(metadataProvider, output, printer, printSignature);
if (!conf.is(SessionConfig.OOB_OPTIMIZED_LOGICAL_PLAN)) {
@@ -304,47 +338,6 @@
return null;
}
- if (printSignature) {
- printer.addResultPrinter(SignaturePrinter.newInstance(executionPlans));
- }
-
- if (!conf.isGenerateJobSpec()) {
- 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);
-
- 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 (isQuery && conf.is(SessionConfig.OOB_HYRACKS_JOB)) {
generateJob(spec);
}
@@ -502,10 +495,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 +523,22 @@
}
}
- 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, Map<Object, String> log2phys,
+ SessionConfig.PlanFormat format, boolean printOptimizerEstimates) throws AlgebricksException {
+ executionPlans.setOptimizedLogicalPlan(
+ getPrettyPrintVisitor(format).printPlan(plan, log2phys, printOptimizerEstimates).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/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/NCQueryServiceServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryServiceServlet.java
index 1ec7405..d2d7a9b 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryServiceServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryServiceServlet.java
@@ -204,6 +204,7 @@
stats.setJobProfile(responseStats.getJobProfile());
stats.setProcessedObjects(responseStats.getProcessedObjects());
stats.updateTotalWarningsCount(responseStats.getTotalWarningsCount());
+ stats.setCompileTime(responseStats.getCompileTime());
}
private static void updatePropertiesFromCC(IStatementExecutor.StatementProperties statementProperties,
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCUdfApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCUdfApiServlet.java
index 3d0f7fc..47685d8 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCUdfApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCUdfApiServlet.java
@@ -81,7 +81,7 @@
protected final IReceptionist receptionist;
protected Path workingDir;
- protected String sysAuthHeader;
+ private String sysAuthHeader;
private ILibraryManager libraryManager;
private int timeout;
@@ -250,7 +250,7 @@
}
URI downloadURI = createDownloadURI(libraryTempFile);
doCreate(dvAndName.getFirst(), dvAndName.getSecond(), language,
- ExternalLibraryUtils.digestToHexString(digest), downloadURI, true, sysAuthHeader,
+ ExternalLibraryUtils.digestToHexString(digest), downloadURI, true, getSysAuthHeader(),
requestReference, request);
} else if (op == LibraryOperation.DELETE) {
//DELETE semantics imply ifExists
@@ -281,6 +281,10 @@
}
}
+ protected String getSysAuthHeader() {
+ return sysAuthHeader;
+ }
+
private void writeException(Exception e, IServletResponse response) {
response.setStatus(toHttpErrorStatus(e));
PrintWriter responseWriter = response.writer();
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..d9df051 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;
@@ -102,7 +102,7 @@
printer.printResults();
ResponseMetrics metrics = ResponseMetrics.of(System.nanoTime() - elapsedStart,
metadata.getJobDuration(), stats.getCount(), stats.getSize(), metadata.getProcessedObjects(), 0,
- metadata.getTotalWarningsCount());
+ metadata.getTotalWarningsCount(), stats.getCompileTime());
printer.addFooterPrinter(new MetricsPrinter(metrics, HttpUtil.getPreferredCharset(request)));
if (metadata.getJobProfile() != null) {
printer.addFooterPrinter(new ProfilePrinter(metadata.getJobProfile()));
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..e14c185 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
@@ -357,9 +357,9 @@
// in case of ASYNC delivery, the status is printed by query translator
responsePrinter.addFooterPrinter(new StatusPrinter(executionState.getResultStatus()));
}
- final ResponseMetrics metrics =
- ResponseMetrics.of(System.nanoTime() - elapsedStart, executionState.duration(), stats.getCount(),
- stats.getSize(), stats.getProcessedObjects(), errorCount, stats.getTotalWarningsCount());
+ final ResponseMetrics metrics = ResponseMetrics.of(System.nanoTime() - elapsedStart, executionState.duration(),
+ stats.getCount(), stats.getSize(), stats.getProcessedObjects(), errorCount,
+ stats.getTotalWarningsCount(), stats.getCompileTime());
responsePrinter.addFooterPrinter(new MetricsPrinter(metrics, resultCharset));
if (isPrintingProfile(stats)) {
responsePrinter.addFooterPrinter(new ProfilePrinter(stats.getJobProfile()));
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/RecoveryTask.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/RecoveryTask.java
index 3bc1c28..34a54d1 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
@@ -18,6 +18,8 @@
*/
package org.apache.asterix.app.active;
+import static org.apache.hyracks.util.ExitUtil.EC_ACTIVE_RECOVERY_FAILURE;
+
import java.util.concurrent.Callable;
import org.apache.asterix.active.ActivityState;
@@ -33,6 +35,7 @@
import org.apache.asterix.metadata.entities.Dataset;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.util.ExitUtil;
import org.apache.hyracks.util.IRetryPolicy;
import org.apache.logging.log4j.Level;
import org.apache.logging.log4j.LogManager;
@@ -63,7 +66,18 @@
IRetryPolicy policy = retryPolicyFactory.create(listener);
return () -> {
Thread.currentThread().setName("RecoveryTask (" + listener.getEntityId() + ")");
- doRecover(policy);
+ try {
+ doRecover(policy);
+ } catch (InterruptedException e) {
+ LOGGER.warn("recovery task interrupted", e);
+ Thread.currentThread().interrupt();
+ throw e;
+ } catch (Throwable t) {
+ // in case of any unexpected exception during recovery, the recovery attempts will stop forever.
+ // we halt to ensure recovery attempts are resumed after the restart
+ LOGGER.fatal("unexpected exception during recovery; halting...", t);
+ ExitUtil.halt(EC_ACTIVE_RECOVERY_FAILURE);
+ }
return null;
};
}
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/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/DumpIndexReader.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexReader.java
index aed15c6..e60b015 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,11 @@
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.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 +110,64 @@
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 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..6c0382d 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexRewriter.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexRewriter.java
@@ -25,6 +25,7 @@
import org.apache.asterix.metadata.declared.MetadataProvider;
import org.apache.asterix.metadata.entities.Dataset;
import org.apache.asterix.metadata.entities.Index;
+import org.apache.asterix.metadata.utils.ISecondaryIndexOperationsHelper;
import org.apache.asterix.metadata.utils.SecondaryIndexOperationsHelper;
import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -63,7 +64,7 @@
if (index.isPrimaryIndex()) {
throw new CompilationException(ErrorCode.OPERATION_NOT_SUPPORTED_ON_PRIMARY_INDEX, loc, indexName);
}
- SecondaryIndexOperationsHelper secondaryIndexHelper =
+ ISecondaryIndexOperationsHelper secondaryIndexHelper =
SecondaryIndexOperationsHelper.createIndexOperationsHelper(dataset, index, metadataProvider, loc);
IndexDataflowHelperFactory indexDataflowHelperFactory =
new IndexDataflowHelperFactory(metadataProvider.getStorageComponentProvider().getStorageManager(),
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/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/QueryIndexDatasource.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/QueryIndexDatasource.java
new file mode 100644
index 0000000..52cd17c
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/QueryIndexDatasource.java
@@ -0,0 +1,131 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.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;
+import org.apache.hyracks.storage.am.common.impls.DefaultTupleProjectorFactory;
+
+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;
+ }
+
+ @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,
+ DefaultTupleProjectorFactory.INSTANCE);
+ }
+
+ @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() });
+ }
+}
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/nc/NCAppRuntimeContext.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
index 1a89168..a46522e 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
@@ -27,6 +27,7 @@
import java.util.concurrent.ExecutorService;
import org.apache.asterix.active.ActiveManager;
+import org.apache.asterix.app.result.ResultReader;
import org.apache.asterix.common.api.IConfigValidator;
import org.apache.asterix.common.api.IConfigValidatorFactory;
import org.apache.asterix.common.api.ICoordinationService;
@@ -86,6 +87,8 @@
import org.apache.hyracks.api.lifecycle.ILifeCycleComponent;
import org.apache.hyracks.api.lifecycle.ILifeCycleComponentManager;
import org.apache.hyracks.api.network.INetworkSecurityManager;
+import org.apache.hyracks.api.result.IResultSet;
+import org.apache.hyracks.client.result.ResultSet;
import org.apache.hyracks.control.common.controllers.NCConfig;
import org.apache.hyracks.control.nc.NodeControllerService;
import org.apache.hyracks.ipc.impl.HyracksConnection;
@@ -108,6 +111,7 @@
import org.apache.hyracks.storage.common.file.ILocalResourceRepositoryFactory;
import org.apache.hyracks.storage.common.file.IResourceIdFactory;
import org.apache.hyracks.util.MaintainedThreadNameExecutorService;
+import org.apache.hyracks.util.NetworkUtil;
import org.apache.hyracks.util.cache.CacheManager;
import org.apache.hyracks.util.cache.ICacheManager;
import org.apache.logging.log4j.Level;
@@ -147,7 +151,8 @@
private final NCExtensionManager ncExtensionManager;
private final IStorageComponentProvider componentProvider;
private final IPersistedResourceRegistry persistedResourceRegistry;
- private IHyracksClientConnection hcc;
+ private volatile HyracksConnection hcc;
+ private volatile ResultSet resultSet;
private IIndexCheckpointManagerProvider indexCheckpointManagerProvider;
private IReplicaManager replicaManager;
private IReceptionist receptionist;
@@ -513,15 +518,22 @@
@Override
public IHyracksClientConnection getHcc() throws HyracksDataException {
- if (hcc == null || !hcc.isConnected()) {
+ HyracksConnection hc = hcc;
+ if (hc == null || !hc.isConnected()) {
synchronized (this) {
- if (hcc == null || !hcc.isConnected()) {
+ hc = hcc;
+ if (hc == null || !hc.isConnected()) {
try {
+ ResultSet rs = resultSet;
+ resultSet = null;
+ NetworkUtil.closeQuietly(rs);
+
NodeControllerService ncSrv = (NodeControllerService) ncServiceContext.getControllerService();
// TODO(mblow): multicc
CcId primaryCcId = ncSrv.getPrimaryCcId();
ClusterControllerInfo ccInfo = ncSrv.getNodeParameters(primaryCcId).getClusterControllerInfo();
- hcc = new HyracksConnection(ccInfo.getClientNetAddress(), ccInfo.getClientNetPort(),
+ NetworkUtil.closeQuietly(hc);
+ hcc = hc = new HyracksConnection(ccInfo.getClientNetAddress(), ccInfo.getClientNetPort(),
ncSrv.getNetworkSecurityManager().getSocketChannelFactory());
} catch (Exception e) {
throw HyracksDataException.create(e);
@@ -529,7 +541,26 @@
}
}
}
- return hcc;
+ return hc;
+ }
+
+ @Override
+ public IResultSet getResultSet() throws HyracksDataException {
+ ResultSet rs = resultSet;
+ if (rs == null) {
+ synchronized (this) {
+ rs = resultSet;
+ if (rs == null) {
+ try {
+ resultSet = rs = ResultReader.createResultSet(getHcc(), ncServiceContext.getControllerService(),
+ compilerProperties);
+ } catch (Exception e) {
+ throw HyracksDataException.create(e);
+ }
+ }
+ }
+ }
+ return rs;
}
@Override
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/PlanStagesGenerator.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/PlanStagesGenerator.java
index 25e51bb..5d74c13 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/PlanStagesGenerator.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/PlanStagesGenerator.java
@@ -26,6 +26,7 @@
import org.apache.commons.lang3.mutable.Mutable;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
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.PhysicalOperatorTag;
@@ -59,6 +60,7 @@
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.SwitchOperator;
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;
@@ -197,6 +199,12 @@
}
@Override
+ public Void visitSwitchOperator(SwitchOperator op, Void arg) throws AlgebricksException {
+ // TODO (GLENN): Implement this logic
+ throw new NotImplementedException();
+ }
+
+ @Override
public Void visitMaterializeOperator(MaterializeOperator op, Void arg) throws AlgebricksException {
visit(op);
return null;
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/RequiredCapacityVisitor.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/RequiredCapacityVisitor.java
index 024a13e..af383c3 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/RequiredCapacityVisitor.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/RequiredCapacityVisitor.java
@@ -24,6 +24,7 @@
import org.apache.commons.lang3.mutable.Mutable;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
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.PhysicalOperatorTag;
@@ -57,6 +58,7 @@
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.SwitchOperator;
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;
@@ -193,6 +195,12 @@
}
@Override
+ public Void visitSwitchOperator(SwitchOperator op, Void arg) throws AlgebricksException {
+ // TODO (GLENN): Implement this logic
+ throw new NotImplementedException();
+ }
+
+ @Override
public Void visitMaterializeOperator(MaterializeOperator op, Void arg) throws AlgebricksException {
visitInternal(op, true);
return null;
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..66d81d2 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;
@@ -82,7 +82,7 @@
for (JobletProfile jp : jobletProfiles) {
final Collection<TaskProfile> jobletTasksProfile = jp.getTaskProfiles().values();
for (TaskProfile tp : jobletTasksProfile) {
- processedObjects += tp.getStatsCollector().getAggregatedStats().getTupleCounter().get();
+ processedObjects += tp.getStatsCollector().getAggregatedStats().getInputTupleCounter().get();
aggregateTotalWarningsCount += tp.getTotalWarningsCount();
Set<Warning> taskWarnings = tp.getWarnings();
if (AggregateWarnings.size() < maxWarnings && !taskWarnings.isEmpty()) {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResponseMetrics.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResponseMetrics.java
index 8dfe923..d55bb07 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResponseMetrics.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResponseMetrics.java
@@ -28,12 +28,13 @@
private long errorCount;
private long warnCount;
private long diskIoCount;
+ private long compileTime;
private ResponseMetrics() {
}
public static ResponseMetrics of(long elapsedTime, long executionTime, long resultCount, long resultSize,
- long processedObjects, long errorCount, long warnCount) {
+ long processedObjects, long errorCount, long warnCount, long compileTime) {
ResponseMetrics metrics = new ResponseMetrics();
metrics.elapsedTime = elapsedTime;
metrics.executionTime = executionTime;
@@ -42,6 +43,7 @@
metrics.processedObjects = processedObjects;
metrics.errorCount = errorCount;
metrics.warnCount = warnCount;
+ metrics.compileTime = compileTime;
return metrics;
}
@@ -72,4 +74,8 @@
public long getWarnCount() {
return warnCount;
}
+
+ public long getCompileTime() {
+ return compileTime;
+ }
}
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/MetricsPrinter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/MetricsPrinter.java
index 5549683..6b1d670 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/MetricsPrinter.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/MetricsPrinter.java
@@ -32,6 +32,7 @@
public enum Metrics {
ELAPSED_TIME("elapsedTime"),
EXECUTION_TIME("executionTime"),
+ COMPILE_TIME("compileTime"),
RESULT_COUNT("resultCount"),
RESULT_SIZE("resultSize"),
ERROR_COUNT("errorCount"),
@@ -71,6 +72,8 @@
ResultUtil.printField(pw, Metrics.EXECUTION_TIME.str(),
Duration.formatNanos(metrics.getExecutionTime(), useAscii));
pw.print("\n\t");
+ ResultUtil.printField(pw, Metrics.COMPILE_TIME.str(), Duration.formatNanos(metrics.getCompileTime(), useAscii));
+ pw.print("\n\t");
ResultUtil.printField(pw, Metrics.RESULT_COUNT.str(), metrics.getResultCount(), true);
pw.print("\n\t");
ResultUtil.printField(pw, Metrics.RESULT_SIZE.str(), metrics.getResultSize(), true);
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 ebc8097..8e61c50 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
@@ -110,7 +110,10 @@
import org.apache.asterix.lang.common.expression.TypeExpression;
import org.apache.asterix.lang.common.expression.TypeReferenceExpression;
import org.apache.asterix.lang.common.expression.VariableExpr;
+import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
import org.apache.asterix.lang.common.statement.AdapterDropStatement;
+import org.apache.asterix.lang.common.statement.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 +198,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 +212,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 +240,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 +256,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 +334,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 +355,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();
@@ -439,16 +441,22 @@
handleViewDropStatement(metadataProvider, stmt);
break;
case LOAD:
+ if (stats.getProfileType() == Stats.ProfileType.FULL) {
+ this.jobFlags.add(JobFlag.PROFILE_RUNTIME);
+ }
handleLoadStatement(metadataProvider, stmt, hcc);
break;
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);
}
+ if (stats.getProfileType() == Stats.ProfileType.FULL) {
+ this.jobFlags.add(JobFlag.PROFILE_RUNTIME);
+ }
handleInsertUpsertStatement(metadataProvider, stmt, hcc, resultSet, resultDelivery, outMetadata,
stats, requestParameters, stmtParams, stmtRewriter);
break;
@@ -480,7 +488,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 +498,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 +519,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 +539,13 @@
}
protected void configureMetadataProvider(MetadataProvider metadataProvider, Map<String, String> config,
- IResultSerializerFactoryProvider resultSerializerFactoryProvider, IAWriterFactory writerFactory,
- FileSplit outputFile, IRequestParameters requestParameters, Statement statement) {
+ Counter resultSetIdCounter, FileSplit outputFile, IRequestParameters requestParameters,
+ Statement statement) {
if (statement.getKind() == Statement.Kind.QUERY && requestParameters.isSQLCompatMode()) {
metadataProvider.getConfig().put(SqlppQueryRewriter.SQL_COMPAT_OPTION, Boolean.TRUE.toString());
}
metadataProvider.getConfig().putAll(config);
- metadataProvider.setWriterFactory(writerFactory);
- metadataProvider.setResultSerializerFactoryProvider(resultSerializerFactoryProvider);
+ metadataProvider.setResultSetIdCounter(resultSetIdCounter);
metadataProvider.setOutputFile(outputFile);
}
@@ -552,6 +568,13 @@
config.put(pname, pvalue);
}
+ protected LangRewritingContext createLangRewritingContext(MetadataProvider metadataProvider,
+ List<FunctionDecl> declaredFunctions, List<ViewDecl> declaredViews, IWarningCollector warningCollector,
+ int varCounter) {
+ return new LangRewritingContext(metadataProvider, declaredFunctions, declaredViews, warningCollector,
+ varCounter);
+ }
+
protected Pair<IAWriterFactory, FileSplit> handleWriteStatement(Statement stmt)
throws InstantiationException, IllegalAccessException, ClassNotFoundException {
WriteStatement ws = (WriteStatement) stmt;
@@ -610,6 +633,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 +739,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 +1090,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 {
@@ -1273,8 +1305,23 @@
}
if (fieldTypePrime == null) {
- throw new CompilationException(ErrorCode.UNKNOWN_TYPE, indexedElement.getSourceLocation(),
- LogRedactionUtil.userData(String.valueOf(projectPath)));
+ if (projectPath != null) {
+ String fieldName = LogRedactionUtil.userData(RecordUtil.toFullyQualifiedName(projectPath));
+ throw new CompilationException(ErrorCode.COMPILATION_ERROR,
+ indexedElement.getSourceLocation(),
+ "cannot find type of field '" + fieldName + "'");
+ }
+ // projectPath == null should only be the case with array index having UNNESTs only
+ if (indexedElement.hasUnnest()) {
+ List<List<String>> unnestList = indexedElement.getUnnestList();
+ List<String> arrayField = unnestList.get(unnestList.size() - 1);
+ String fieldName = LogRedactionUtil.userData(RecordUtil.toFullyQualifiedName(arrayField));
+ throw new CompilationException(ErrorCode.COMPILATION_ERROR,
+ indexedElement.getSourceLocation(),
+ "cannot find type of elements of field '" + fieldName + "'");
+ }
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE,
+ indexedElement.getSourceLocation(), "cannot find type of field");
}
validateIndexFieldType(indexType, fieldTypePrime, projectPath, indexedElement.getSourceLocation());
@@ -1394,6 +1441,9 @@
stmt.getSourceLocation());
DataverseName dataverseName = getActiveDataverseName(stmtCreateFilter.getDataverseName());
+ if (isCompileOnly()) {
+ return;
+ }
lockUtil.createFullTextFilterBegin(lockManager, metadataProvider.getLocks(), dataverseName, fullTextFilterName);
try {
doCreateFullTextFilter(metadataProvider, stmtCreateFilter, dataverseName);
@@ -1462,6 +1512,9 @@
DataverseName dataverseName = getActiveDataverseName(stmtCreateConfig.getDataverseName());
ImmutableList<String> filterNames = stmtCreateConfig.getFilterNames();
+ if (isCompileOnly()) {
+ return;
+ }
lockUtil.createFullTextConfigBegin(lockManager, metadataProvider.getLocks(), dataverseName, configName,
filterNames);
try {
@@ -1827,6 +1880,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);
@@ -1880,6 +1936,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);
@@ -2020,6 +2079,8 @@
if (activeDataverse.getDataverseName().equals(dataverseName)) {
activeDataverse = MetadataBuiltinEntities.DEFAULT_DATAVERSE;
}
+
+ validateDataverseDatasetsStateAfterDrop(metadataProvider, mdTxnCtx, datasets);
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
return true;
} catch (Exception e) {
@@ -2066,7 +2127,12 @@
protected void validateDataverseStateBeforeDrop(MetadataProvider metadataProvider, Dataverse dataverse,
SourceLocation sourceLoc) throws AlgebricksException {
- // may be overriden by product extensions for additional checks before dropping the dataverse
+ // may be overridden by product extensions for additional checks before dropping the dataverse
+ }
+
+ protected void validateDataverseDatasetsStateAfterDrop(MetadataProvider metadataProvider,
+ MetadataTransactionContext mdTxnCtx, List<Dataset> datasets) throws AlgebricksException {
+ // may be overridden by product extensions for additional checks after dropping the dataverse
}
public void handleDatasetDropStatement(MetadataProvider metadataProvider, Statement stmt,
@@ -2076,6 +2142,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,
@@ -2183,6 +2252,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);
@@ -2223,14 +2295,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);
@@ -2264,8 +2330,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) {
@@ -2287,12 +2354,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;
@@ -2365,6 +2426,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);
@@ -2405,6 +2469,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);
@@ -2454,6 +2521,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);
@@ -2494,6 +2564,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);
@@ -2538,6 +2611,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);
@@ -2692,8 +2768,10 @@
IQueryRewriter queryRewriter = rewriterFactory.createQueryRewriter();
Query wrappedQuery = queryRewriter.createViewAccessorQuery(viewDecl);
metadataProvider.setDefaultDataverse(dv);
- apiFramework.reWriteQuery(declaredFunctions, Collections.singletonList(viewDecl), metadataProvider,
- wrappedQuery, sessionOutput, false, false, Collections.emptyList(), warningCollector);
+ LangRewritingContext langRewritingContext = createLangRewritingContext(metadataProvider, declaredFunctions,
+ Collections.singletonList(viewDecl), warningCollector, wrappedQuery.getVarCounter());
+ apiFramework.reWriteQuery(langRewritingContext, wrappedQuery, sessionOutput, false, false,
+ Collections.emptyList());
List<List<Triple<DataverseName, String, String>>> dependencies =
ViewUtil.getViewDependencies(viewDecl, foreignKeys, queryRewriter);
@@ -2729,6 +2807,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);
@@ -2811,6 +2892,9 @@
}
}
+ if (isCompileOnly()) {
+ return;
+ }
lockUtil.createFunctionBegin(lockManager, metadataProvider.getLocks(), dataverseName, signature.getName(),
libraryDataverseName, libraryName);
try {
@@ -2961,8 +3045,10 @@
fdList.addAll(declaredFunctions);
fdList.add(fd);
metadataProvider.setDefaultDataverse(dv);
- apiFramework.reWriteQuery(fdList, null, metadataProvider, wrappedQuery, sessionOutput, false, false,
- Collections.emptyList(), warningCollector);
+ LangRewritingContext langRewritingContext = createLangRewritingContext(metadataProvider, fdList, null,
+ warningCollector, wrappedQuery.getVarCounter());
+ apiFramework.reWriteQuery(langRewritingContext, wrappedQuery, sessionOutput, false, false,
+ Collections.emptyList());
List<List<Triple<DataverseName, String, String>>> dependencies =
FunctionUtil.getFunctionDependencies(fd, queryRewriter);
@@ -3060,6 +3146,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);
@@ -3118,6 +3207,9 @@
libraryDataverseName = dataverseName;
}
String libraryName = cas.getLibraryName();
+ if (isCompileOnly()) {
+ return;
+ }
lockUtil.createAdapterBegin(lockManager, metadataProvider.getLocks(), dataverseName, adapterName,
libraryDataverseName, libraryName);
try {
@@ -3188,6 +3280,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);
@@ -3238,6 +3333,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);
@@ -3366,6 +3464,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);
@@ -3460,6 +3561,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);
@@ -3507,6 +3611,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);
@@ -3560,7 +3667,7 @@
afterCompile();
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
bActiveTxn = false;
- if (spec != null && sessionConfig.isExecuteQuery()) {
+ if (spec != null && !isCompileOnly()) {
runJob(hcc, spec);
}
} catch (Exception e) {
@@ -3605,7 +3712,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);
@@ -3656,7 +3763,7 @@
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
bActiveTxn = false;
- if (jobSpec != null && sessionConfig.isExecuteQuery()) {
+ if (jobSpec != null && !isCompileOnly()) {
runJob(hcc, jobSpec);
}
return jobSpec;
@@ -3679,8 +3786,10 @@
Map<VarIdentifier, IAObject> externalVars = createExternalVariables(query, stmtParams);
// Query Rewriting (happens under the same ongoing metadata transaction)
- Pair<IReturningStatement, Integer> rewrittenResult = apiFramework.reWriteQuery(declaredFunctions, null,
- metadataProvider, query, sessionOutput, true, true, externalVars.keySet(), warningCollector);
+ LangRewritingContext langRewritingContext = createLangRewritingContext(metadataProvider, declaredFunctions,
+ null, warningCollector, query.getVarCounter());
+ Pair<IReturningStatement, Integer> rewrittenResult = apiFramework.reWriteQuery(langRewritingContext, query,
+ sessionOutput, true, true, externalVars.keySet());
// Query Compilation (happens under the same ongoing metadata transaction)
return apiFramework.compileQuery(clusterInfoCollector, metadataProvider, (Query) rewrittenResult.first,
@@ -3688,7 +3797,7 @@
responsePrinter, warningCollector, requestParameters);
}
- private JobSpecification rewriteCompileInsertUpsert(IClusterInfoCollector clusterInfoCollector,
+ protected JobSpecification rewriteCompileInsertUpsert(IClusterInfoCollector clusterInfoCollector,
MetadataProvider metadataProvider, InsertStatement insertUpsert, Map<String, IAObject> stmtParams)
throws AlgebricksException, ACIDException {
SourceLocation sourceLoc = insertUpsert.getSourceLocation();
@@ -3696,8 +3805,10 @@
Map<VarIdentifier, IAObject> externalVars = createExternalVariables(insertUpsert, stmtParams);
// Insert/upsert statement rewriting (happens under the same ongoing metadata transaction)
- Pair<IReturningStatement, Integer> rewrittenResult = apiFramework.reWriteQuery(declaredFunctions, null,
- metadataProvider, insertUpsert, sessionOutput, true, true, externalVars.keySet(), warningCollector);
+ LangRewritingContext langRewritingContext = createLangRewritingContext(metadataProvider, declaredFunctions,
+ null, warningCollector, insertUpsert.getVarCounter());
+ Pair<IReturningStatement, Integer> rewrittenResult = apiFramework.reWriteQuery(langRewritingContext,
+ insertUpsert, sessionOutput, true, true, externalVars.keySet());
InsertStatement rewrittenInsertUpsert = (InsertStatement) rewrittenResult.first;
DataverseName dataverseName = getActiveDataverseName(rewrittenInsertUpsert.getDataverseName());
@@ -3733,6 +3844,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);
@@ -3772,6 +3886,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();
@@ -3833,6 +3950,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);
@@ -3886,6 +4006,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);
@@ -3915,6 +4038,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);
@@ -3977,6 +4103,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);
@@ -3994,6 +4123,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
@@ -4049,6 +4181,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);
@@ -4088,12 +4223,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);
@@ -4186,6 +4694,7 @@
}
};
final IStatementCompiler compiler = () -> {
+ long compileStart = System.nanoTime();
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
boolean bActiveTxn = true;
metadataProvider.setMetadataTxnContext(mdTxnCtx);
@@ -4196,8 +4705,9 @@
stats.updateTotalWarningsCount(warningCollector.getTotalWarningsCount());
afterCompile();
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ stats.setCompileTime(System.nanoTime() - compileStart);
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) {
@@ -4255,8 +4765,8 @@
private void updateJobStats(JobId jobId, Stats stats, ResultSetId rsId) throws HyracksDataException {
final ClusterControllerService controllerService =
(ClusterControllerService) appCtx.getServiceContext().getControllerService();
- org.apache.asterix.api.common.ResultMetadata resultMetadata =
- (org.apache.asterix.api.common.ResultMetadata) controllerService.getResultDirectoryService()
+ org.apache.asterix.translator.ResultMetadata resultMetadata =
+ (org.apache.asterix.translator.ResultMetadata) controllerService.getResultDirectoryService()
.getResultMetadata(jobId, rsId);
stats.setProcessedObjects(resultMetadata.getProcessedObjects());
if (jobFlags.contains(JobFlag.PROFILE_RUNTIME)) {
@@ -4314,6 +4824,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,
@@ -4367,6 +4883,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);
@@ -4401,9 +4920,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;
@@ -4414,6 +4930,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);
@@ -4748,7 +5270,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();
}
@@ -4773,6 +5295,10 @@
}
}
+ protected boolean isCompileOnly() {
+ return !sessionConfig.isExecuteQuery();
+ }
+
protected void validateDatasetState(MetadataProvider metadataProvider, Dataset dataset, SourceLocation sourceLoc)
throws Exception {
validateIfResourceIsActiveInFeed(metadataProvider.getApplicationContext(), dataset, sourceLoc);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
index 8f3deb1..2a66cfd 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
@@ -53,6 +53,7 @@
import org.apache.asterix.api.http.server.VersionApiServlet;
import org.apache.asterix.app.active.ActiveNotificationHandler;
import org.apache.asterix.app.cc.CCExtensionManager;
+import org.apache.asterix.app.cc.CcApplicationContext;
import org.apache.asterix.app.config.ConfigValidator;
import org.apache.asterix.app.io.PersistedResourceRegistry;
import org.apache.asterix.app.replication.NcLifecycleCoordinator;
@@ -85,7 +86,6 @@
import org.apache.asterix.metadata.lock.MetadataLockManager;
import org.apache.asterix.metadata.utils.MetadataLockUtil;
import org.apache.asterix.runtime.job.resource.JobCapacityController;
-import org.apache.asterix.runtime.utils.CcApplicationContext;
import org.apache.asterix.translator.IStatementExecutorFactory;
import org.apache.asterix.translator.Receptionist;
import org.apache.asterix.util.MetadataBuiltinFunctions;
@@ -126,7 +126,7 @@
protected WebManager webManager;
protected ICcApplicationContext appCtx;
private IJobCapacityController jobCapacityController;
- private IHyracksClientConnection hcc;
+ private HyracksConnection hcc;
@Override
public void init(IServiceContext serviceCtx) throws Exception {
@@ -209,7 +209,7 @@
IReceptionistFactory receptionistFactory, IConfigValidatorFactory configValidatorFactory,
CCExtensionManager ccExtensionManager, IAdapterFactoryService adapterFactoryService)
throws AlgebricksException, IOException {
- return new CcApplicationContext(ccServiceCtx, getHcc(), () -> MetadataManager.INSTANCE, globalRecoveryManager,
+ return new CcApplicationContext(ccServiceCtx, hcc, () -> MetadataManager.INSTANCE, globalRecoveryManager,
lifecycleCoordinator, new ActiveNotificationHandler(), componentProvider, new MetadataLockManager(),
createMetadataLockUtil(), receptionistFactory, configValidatorFactory, ccExtensionManager,
adapterFactoryService);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/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/RedactionUtil.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RedactionUtil.java
index 156b78a..48cf511 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RedactionUtil.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RedactionUtil.java
@@ -20,7 +20,7 @@
import static java.util.regex.Pattern.CASE_INSENSITIVE;
import static java.util.regex.Pattern.DOTALL;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.SECRET_ACCESS_KEY_FIELD_NAME;
+import static org.apache.asterix.external.util.aws.s3.S3Constants.SECRET_ACCESS_KEY_FIELD_NAME;
import java.util.regex.Pattern;
diff --git a/asterixdb/asterix-app/src/main/resources/cc.conf b/asterixdb/asterix-app/src/main/resources/cc.conf
index 98ede89..dc6e5a2 100644
--- a/asterixdb/asterix-app/src/main/resources/cc.conf
+++ b/asterixdb/asterix-app/src/main/resources/cc.conf
@@ -52,6 +52,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
diff --git a/asterixdb/asterix-app/src/main/resources/cc2.conf b/asterixdb/asterix-app/src/main/resources/cc2.conf
index 016c485..c746198 100644
--- a/asterixdb/asterix-app/src/main/resources/cc2.conf
+++ b/asterixdb/asterix-app/src/main/resources/cc2.conf
@@ -48,6 +48,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
diff --git a/asterixdb/asterix-app/src/main/resources/cc3.conf b/asterixdb/asterix-app/src/main/resources/cc3.conf
index 88362aa..8e9b5ab 100644
--- a/asterixdb/asterix-app/src/main/resources/cc3.conf
+++ b/asterixdb/asterix-app/src/main/resources/cc3.conf
@@ -48,6 +48,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
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..deb92e0
--- /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.pagesize=32KB
+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
diff --git a/asterixdb/asterix-app/src/main/resources/entrypoint.py b/asterixdb/asterix-app/src/main/resources/entrypoint.py
index 7bad7ef..918596c 100755
--- a/asterixdb/asterix-app/src/main/resources/entrypoint.py
+++ b/asterixdb/asterix-app/src/main/resources/entrypoint.py
@@ -168,6 +168,7 @@
def quit(self):
self.alive = False
+ self.disconnect_sock()
return True
def handle_call(self):
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..e8c2c1d 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
@@ -43,6 +43,7 @@
import org.apache.asterix.common.library.ILibraryManager;
import org.apache.asterix.hyracks.bootstrap.CCApplication;
import org.apache.asterix.hyracks.bootstrap.NCApplication;
+import org.apache.asterix.lang.common.util.ExpressionUtils;
import org.apache.asterix.test.dataflow.TestLsmIoOpCallbackFactory;
import org.apache.asterix.test.dataflow.TestPrimaryIndexOperationTrackerFactory;
import org.apache.commons.io.FileUtils;
@@ -132,13 +133,13 @@
cc = new ClusterControllerService(ccConfig, ccApplication);
nodeNames = ccConfig.getConfigManager().getNodeNames();
- if (deleteOldInstanceData) {
+ if (deleteOldInstanceData && nodeNames != null) {
deleteTransactionLogs();
removeTestStorageFiles();
deleteCCFiles();
}
final List<NodeControllerService> nodeControllers = new ArrayList<>();
- for (String nodeId : nodeNames) {
+ for (String nodeId : ExpressionUtils.emptyIfNull(nodeNames)) {
// mark this NC as virtual, so that the CC doesn't try to start via NCService...
configManager.set(nodeId, NCConfig.Option.NCSERVICE_PORT, NCConfig.NCSERVICE_PORT_DISABLED);
final INCApplication ncApplication = createNCApplication();
@@ -303,7 +304,7 @@
stopCC(false);
- if (deleteOldInstanceData) {
+ if (deleteOldInstanceData && nodeNames != null) {
deleteTransactionLogs();
removeTestStorageFiles();
deleteCCFiles();
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/VersionApiServletTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/VersionApiServletTest.java
index 8dcfa26..eb63218 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/VersionApiServletTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/VersionApiServletTest.java
@@ -31,8 +31,8 @@
import java.util.concurrent.ConcurrentHashMap;
import org.apache.asterix.api.http.server.VersionApiServlet;
+import org.apache.asterix.app.cc.CcApplicationContext;
import org.apache.asterix.common.config.BuildProperties;
-import org.apache.asterix.runtime.utils.CcApplicationContext;
import org.apache.hyracks.api.client.IHyracksClientConnection;
import org.apache.hyracks.http.api.IServletRequest;
import org.apache.hyracks.http.api.IServletResponse;
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
index 0b80881..34696b1 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
@@ -26,6 +26,7 @@
import java.util.List;
import java.util.Map;
+import org.apache.asterix.app.cc.CcApplicationContext;
import org.apache.asterix.app.nc.NCAppRuntimeContext;
import org.apache.asterix.app.nc.TransactionSubsystem;
import org.apache.asterix.common.config.TransactionProperties;
@@ -61,7 +62,6 @@
import org.apache.asterix.runtime.operators.LSMIndexBulkLoadOperatorNodePushable;
import org.apache.asterix.runtime.operators.LSMPrimaryInsertOperatorNodePushable;
import org.apache.asterix.runtime.operators.LSMPrimaryUpsertOperatorNodePushable;
-import org.apache.asterix.runtime.utils.CcApplicationContext;
import org.apache.asterix.test.runtime.ExecutionTestUtil;
import org.apache.asterix.transaction.management.runtime.CommitRuntime;
import org.apache.asterix.transaction.management.service.logging.LogReader;
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ClusterStateManagerTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ClusterStateManagerTest.java
index b80fa30..a2a3b48 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ClusterStateManagerTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ClusterStateManagerTest.java
@@ -26,6 +26,7 @@
import java.util.TreeMap;
import java.util.concurrent.ConcurrentHashMap;
+import org.apache.asterix.app.cc.CcApplicationContext;
import org.apache.asterix.app.replication.NcLifecycleCoordinator;
import org.apache.asterix.app.replication.message.NCLifecycleTaskReportMessage;
import org.apache.asterix.common.api.IClusterManagementWork.ClusterState;
@@ -37,7 +38,6 @@
import org.apache.asterix.hyracks.bootstrap.CCApplication;
import org.apache.asterix.runtime.transaction.ResourceIdManager;
import org.apache.asterix.runtime.utils.BulkTxnIdFactory;
-import org.apache.asterix.runtime.utils.CcApplicationContext;
import org.apache.asterix.runtime.utils.ClusterStateManager;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.api.application.ICCServiceContext;
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveEventsListenerTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveEventsListenerTest.java
index c71e602..e5e33d0 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveEventsListenerTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveEventsListenerTest.java
@@ -34,6 +34,7 @@
import org.apache.asterix.active.EntityId;
import org.apache.asterix.active.NoRetryPolicyFactory;
import org.apache.asterix.app.active.ActiveNotificationHandler;
+import org.apache.asterix.app.cc.CcApplicationContext;
import org.apache.asterix.common.api.IClusterManagementWork.ClusterState;
import org.apache.asterix.common.api.IMetadataLockManager;
import org.apache.asterix.common.cluster.IClusterStateManager;
@@ -53,7 +54,6 @@
import org.apache.asterix.metadata.utils.MetadataLockUtil;
import org.apache.asterix.runtime.functions.FunctionCollection;
import org.apache.asterix.runtime.functions.FunctionManager;
-import org.apache.asterix.runtime.utils.CcApplicationContext;
import org.apache.asterix.test.active.TestEventsListener.Behavior;
import org.apache.asterix.test.base.TestMethodTracer;
import org.apache.asterix.translator.IStatementExecutor;
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveStatsTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveStatsTest.java
index c51fd95..cb123bf 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveStatsTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveStatsTest.java
@@ -37,6 +37,7 @@
import org.apache.asterix.app.active.ActiveEntityEventsListener;
import org.apache.asterix.app.active.ActiveNotificationHandler;
import org.apache.asterix.app.cc.CCExtensionManager;
+import org.apache.asterix.app.cc.CcApplicationContext;
import org.apache.asterix.app.nc.NCAppRuntimeContext;
import org.apache.asterix.app.result.ResponsePrinter;
import org.apache.asterix.common.exceptions.ErrorCode;
@@ -45,7 +46,6 @@
import org.apache.asterix.external.operators.FeedIntakeOperatorNodePushable;
import org.apache.asterix.metadata.declared.MetadataProvider;
import org.apache.asterix.metadata.entities.Dataset;
-import org.apache.asterix.runtime.utils.CcApplicationContext;
import org.apache.asterix.test.runtime.ExecutionTestUtil;
import org.apache.asterix.translator.IStatementExecutor;
import org.apache.asterix.translator.SessionOutput;
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/ProfilingTestExecutor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/ProfilingTestExecutor.java
new file mode 100644
index 0000000..44502da
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/ProfilingTestExecutor.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.test.common;
+
+import java.io.InputStream;
+import java.net.URI;
+import java.nio.charset.Charset;
+import java.util.List;
+import java.util.function.Predicate;
+
+import org.apache.asterix.testframework.context.TestCaseContext;
+import org.apache.asterix.testframework.xml.ParameterTypeEnum;
+import org.apache.asterix.testframework.xml.TestCase;
+
+public class ProfilingTestExecutor extends TestExecutor {
+
+ private final TestCase.CompilationUnit.Parameter profile = new TestCase.CompilationUnit.Parameter();
+
+ public InputStream executeQueryService(String str, TestCaseContext.OutputFormat fmt, URI uri,
+ List<TestCase.CompilationUnit.Parameter> params, boolean jsonEncoded, Charset responseCharset,
+ Predicate<Integer> responseCodeValidator, boolean cancellable) throws Exception {
+ profile.setName("profile");
+ profile.setValue("timings");
+ profile.setType(ParameterTypeEnum.STRING);
+ params.add(profile);
+ return super.executeQueryService(str, fmt, uri, constructQueryParameters(str, fmt, params), jsonEncoded,
+ responseCharset, responseCodeValidator, cancellable);
+
+ }
+}
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/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/om/lazy/LazyVisitablePointableTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/om/lazy/LazyVisitablePointableTest.java
new file mode 100644
index 0000000..d981be1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/om/lazy/LazyVisitablePointableTest.java
@@ -0,0 +1,122 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.om.lazy;
+
+import static org.apache.hyracks.util.file.FileUtil.joinPath;
+
+import java.io.FileInputStream;
+import java.io.IOException;
+
+import org.apache.asterix.external.parser.JSONDataParser;
+import org.apache.asterix.om.lazy.AbstractLazyVisitablePointable;
+import org.apache.asterix.om.lazy.RecordLazyVisitablePointable;
+import org.apache.asterix.om.lazy.TypedRecordLazyVisitablePointable;
+import org.apache.asterix.om.pointables.ARecordVisitablePointable;
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+import org.apache.asterix.om.pointables.base.IVisitablePointable;
+import org.apache.asterix.om.pointables.cast.ACastVisitor;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.evaluators.comparisons.DeepEqualAssessor;
+import org.apache.hyracks.algebricks.common.utils.Triple;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.junit.Assert;
+import org.junit.Test;
+
+import com.fasterxml.jackson.core.JsonFactory;
+
+/**
+ * Test the operations of {@link AbstractLazyVisitablePointable}
+ */
+public class LazyVisitablePointableTest {
+ private static final String BASE_DIR;
+ private static final String[] FILE_PATHS;
+ private final JSONDataParser parser;
+ private final ACastVisitor castVisitor;
+ private final RecordTypeInference schemaInference;
+ private final DeepEqualAssessor deepEqualAssessor;
+ private final RecordLazyVisitablePointable openLazyPointable;
+ private final ARecordVisitablePointable openPointable;
+ private final ArrayBackedValueStorage recordStorage;
+ private final Triple<IVisitablePointable, IAType, Boolean> arg;
+
+ static {
+ BASE_DIR = "data";
+ FILE_PATHS = new String[] { joinPath(BASE_DIR, "hdfs", "parquet", "dummy_tweet.json"),
+ joinPath(BASE_DIR, "nested01", "person2.adm"), joinPath(BASE_DIR, "yelp-checkin", "use-case-1.json"),
+ joinPath(BASE_DIR, "yelp-checkin", "use-case-2.json"),
+ joinPath(BASE_DIR, "yelp-checkin", "use-case-3.json"),
+ joinPath(BASE_DIR, "yelp-checkin", "use-case-4.json") };
+ }
+
+ public LazyVisitablePointableTest() {
+ parser = new JSONDataParser(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE, new JsonFactory());
+ castVisitor = new ACastVisitor();
+ schemaInference = new RecordTypeInference();
+ deepEqualAssessor = new DeepEqualAssessor();
+ openLazyPointable = new RecordLazyVisitablePointable(true);
+ openPointable = new ARecordVisitablePointable(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
+ recordStorage = new ArrayBackedValueStorage();
+ arg = new Triple<>(null, null, null);
+ arg.third = Boolean.FALSE;
+ }
+
+ private void prepareParser(String path) throws IOException {
+ FileInputStream inputStream = new FileInputStream(path);
+ parser.setInputStream(inputStream);
+ }
+
+ private void inferCastAndCompare() throws HyracksDataException {
+ recordStorage.reset();
+ while (parser.parse(recordStorage.getDataOutput())) {
+ openLazyPointable.set(recordStorage);
+
+ //Infer the schema
+ ARecordType inferredFromOpen = (ARecordType) openLazyPointable.accept(schemaInference, "fromOpen");
+ ARecordVisitablePointable closedPointable = new ARecordVisitablePointable(inferredFromOpen);
+ arg.first = closedPointable;
+ arg.second = inferredFromOpen;
+
+ //Cast to closed using the inferred type
+ openPointable.set(recordStorage);
+ openPointable.accept(castVisitor, arg);
+ //Ensure both closed and open records are the same
+ Assert.assertTrue(deepEqualAssessor.isEqual(openPointable, closedPointable));
+
+ //Ensure lazy pointable can handle closed types
+ TypedRecordLazyVisitablePointable closedLazyPointable =
+ new TypedRecordLazyVisitablePointable(inferredFromOpen);
+ closedLazyPointable.set(closedPointable);
+ //Infer the type (again) but from a closed type
+ ARecordType inferredFromClosed = (ARecordType) closedLazyPointable.accept(schemaInference, "fromClosed");
+ //Ensure both inferred types are the same
+ Assert.assertTrue(inferredFromOpen.deepEqual(inferredFromClosed));
+ recordStorage.reset();
+ }
+ }
+
+ @Test
+ public void runTest() throws IOException {
+ for (String path : FILE_PATHS) {
+ prepareParser(path);
+ inferCastAndCompare();
+ }
+ }
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/om/lazy/RecordTypeInference.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/om/lazy/RecordTypeInference.java
new file mode 100644
index 0000000..4a1a4a3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/om/lazy/RecordTypeInference.java
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.om.lazy;
+
+import java.io.DataInputStream;
+import java.io.IOException;
+
+import org.apache.asterix.om.lazy.AbstractListLazyVisitablePointable;
+import org.apache.asterix.om.lazy.FlatLazyVisitablePointable;
+import org.apache.asterix.om.lazy.ILazyVisitablePointableVisitor;
+import org.apache.asterix.om.lazy.RecordLazyVisitablePointable;
+import org.apache.asterix.om.types.AOrderedListType;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AUnorderedListType;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.data.std.util.ByteArrayAccessibleInputStream;
+import org.apache.hyracks.util.string.UTF8StringReader;
+import org.apache.hyracks.util.string.UTF8StringUtil;
+
+/**
+ * Infers the type of single record using lazy visitable pointable
+ */
+public class RecordTypeInference implements ILazyVisitablePointableVisitor<IAType, String> {
+ private final ByteArrayAccessibleInputStream in;
+ private final DataInputStream dataIn;
+ private final UTF8StringReader utf8Reader;
+
+ public RecordTypeInference() {
+ in = new ByteArrayAccessibleInputStream(new byte[] {}, 0, 0);
+ dataIn = new DataInputStream(in);
+ utf8Reader = new UTF8StringReader();
+ }
+
+ @Override
+ public IAType visit(RecordLazyVisitablePointable pointable, String arg) throws HyracksDataException {
+ String[] fieldNames = new String[pointable.getNumberOfChildren()];
+ IAType[] fieldTypes = new IAType[pointable.getNumberOfChildren()];
+ for (int i = 0; i < pointable.getNumberOfChildren(); i++) {
+ pointable.nextChild();
+ fieldNames[i] = deserializeString(pointable.getFieldName());
+ fieldTypes[i] = pointable.getChildVisitablePointable().accept(this, fieldNames[i]);
+ }
+ // isOpen has to be false here to ensure that every field go to the closed part
+ return new ARecordType(arg, fieldNames, fieldTypes, false);
+ }
+
+ @Override
+ public IAType visit(AbstractListLazyVisitablePointable pointable, String arg) throws HyracksDataException {
+ IAType itemType = BuiltinType.ANY;
+ String itemTypeName = arg + "Item";
+ for (int i = 0; i < pointable.getNumberOfChildren(); i++) {
+ pointable.nextChild();
+ IAType ithItemType = pointable.getChildVisitablePointable().accept(this, itemTypeName);
+ if (itemType.getTypeTag() != ATypeTag.ANY && itemType.getTypeTag() != ithItemType.getTypeTag()) {
+ throw new UnsupportedOperationException("Union types are not supported");
+ }
+ itemType = ithItemType;
+ }
+ return pointable.getTypeTag() == ATypeTag.ARRAY ? new AOrderedListType(itemType, arg)
+ : new AUnorderedListType(itemType, arg);
+ }
+
+ @Override
+ public IAType visit(FlatLazyVisitablePointable pointable, String arg) throws HyracksDataException {
+ return BuiltinType.getBuiltinType(pointable.getTypeTag());
+ }
+
+ private String deserializeString(IValueReference stringValue) throws HyracksDataException {
+ in.setContent(stringValue.getByteArray(), stringValue.getStartOffset(), stringValue.getLength());
+ try {
+ return UTF8StringUtil.readUTF8(dataIn, utf8Reader);
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ }
+}
\ No newline at end of file
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/LangExecutionUtil.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/LangExecutionUtil.java
index 408882d..d704d8e 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/LangExecutionUtil.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/LangExecutionUtil.java
@@ -33,6 +33,7 @@
import java.util.List;
import org.apache.asterix.app.external.ExternalUDFLibrarian;
+import org.apache.asterix.app.external.IExternalUDFLibrarian;
import org.apache.asterix.common.utils.StorageConstants;
import org.apache.asterix.test.common.TestExecutor;
import org.apache.asterix.testframework.context.TestCaseContext;
@@ -65,11 +66,17 @@
}
public static void setUp(String configFile, TestExecutor executor, boolean startHdfs) throws Exception {
+ setUp(configFile, executor, startHdfs, false, new ExternalUDFLibrarian());
+ }
+
+ public static void setUp(String configFile, TestExecutor executor, boolean startHdfs, boolean disableLangExec,
+ IExternalUDFLibrarian librarian) throws Exception {
testExecutor = executor;
File outdir = new File(PATH_ACTUAL);
outdir.mkdirs();
- ExecutionTestUtil.setUp(cleanupOnStart, configFile, integrationUtil, startHdfs, null);
- librarian = new ExternalUDFLibrarian();
+ if (!disableLangExec) {
+ ExecutionTestUtil.setUp(cleanupOnStart, configFile, integrationUtil, startHdfs, null);
+ }
testExecutor.setLibrarian(librarian);
if (repeat != 1) {
System.out.println("FYI: each test will be run " + repeat + " times.");
@@ -151,7 +158,9 @@
NodeControllerService[] ncs = integrationUtil.ncs;
// Checks that dataset files are uniformly distributed across each io device.
for (NodeControllerService nc : ncs) {
- checkNcStore(nc);
+ if (nc != null) {
+ checkNcStore(nc);
+ }
}
}
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..6eaa09d 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
@@ -31,11 +31,11 @@
import org.junit.runners.Parameterized.Parameters;
/**
- * Runs the cluster state runtime tests with the storage parallelism.
+ * Runs the cluster runtime tests and checks the query metrics.
*/
@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/SqlppBatchPointLookupExecutionTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppBatchPointLookupExecutionTest.java
new file mode 100644
index 0000000..2608447
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppBatchPointLookupExecutionTest.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.test.runtime;
+
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.test.common.TestExecutor;
+import org.apache.asterix.testframework.context.TestCaseContext;
+import org.apache.hyracks.control.nc.NodeControllerService;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+/**
+ * Runs the SQL++ with Batched point-lookups enabled.
+ */
+@RunWith(Parameterized.class)
+public class SqlppBatchPointLookupExecutionTest {
+ protected static final String TEST_CONFIG_FILE_NAME = "src/test/resources/cc-batch-lookup.conf";
+
+ @BeforeClass
+ public static void setUp() throws Exception {
+ final TestExecutor testExecutor = new TestExecutor();
+ LangExecutionUtil.setUp(TEST_CONFIG_FILE_NAME, testExecutor);
+ setNcEndpoints(testExecutor);
+ }
+
+ @AfterClass
+ public static void tearDown() throws Exception {
+ LangExecutionUtil.tearDown();
+ }
+
+ @Parameters(name = "SqlppBatchPointLookupExecutionTest {index}: {0}")
+ public static Collection<Object[]> tests() throws Exception {
+ return LangExecutionUtil.tests("only_batch_lookup.xml", "testsuite_sqlpp_batch_lookup.xml");
+ }
+
+ protected TestCaseContext tcCtx;
+
+ public SqlppBatchPointLookupExecutionTest(TestCaseContext tcCtx) {
+ this.tcCtx = tcCtx;
+ }
+
+ @Test
+ public void test() throws Exception {
+ LangExecutionUtil.test(tcCtx);
+ }
+
+ private static void setNcEndpoints(TestExecutor testExecutor) {
+ final NodeControllerService[] ncs = ExecutionTestUtil.integrationUtil.ncs;
+ final Map<String, InetSocketAddress> ncEndPoints = new HashMap<>();
+ final String ip = InetAddress.getLoopbackAddress().getHostAddress();
+ for (NodeControllerService nc : ncs) {
+ final String nodeId = nc.getId();
+ final INcApplicationContext appCtx = (INcApplicationContext) nc.getApplicationContext();
+ int apiPort = appCtx.getExternalProperties().getNcApiPort();
+ ncEndPoints.put(nodeId, InetSocketAddress.createUnresolved(ip, apiPort));
+ }
+ testExecutor.setNcEndPoints(ncEndPoints);
+ }
+}
diff --git a/asterixdb/asterix-app/src/test/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/java/org/apache/asterix/test/runtime/SqlppProfiledExecutionTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppProfiledExecutionTest.java
new file mode 100644
index 0000000..b056d2c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppProfiledExecutionTest.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.test.runtime;
+
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.test.common.ProfilingTestExecutor;
+import org.apache.asterix.test.common.TestExecutor;
+import org.apache.asterix.testframework.context.TestCaseContext;
+import org.apache.hyracks.control.nc.NodeControllerService;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+/**
+ * Runs the SQL++ runtime tests with full runtime profiling.
+ */
+@RunWith(Parameterized.class)
+public class SqlppProfiledExecutionTest {
+ protected static final String TEST_CONFIG_FILE_NAME = "src/main/resources/cc.conf";
+
+ @BeforeClass
+ public static void setUp() throws Exception {
+ final TestExecutor testExecutor = new ProfilingTestExecutor();
+ LangExecutionUtil.setUp(TEST_CONFIG_FILE_NAME, testExecutor);
+ setNcEndpoints(testExecutor);
+ }
+
+ @AfterClass
+ public static void tearDown() throws Exception {
+ LangExecutionUtil.tearDown();
+ }
+
+ @Parameters(name = "SqlppProfiledExecutionTest {index}: {0}")
+ public static Collection<Object[]> tests() throws Exception {
+ return LangExecutionUtil.tests("only_sqlpp.xml", "testsuite_sqlpp_profiled.xml");
+ }
+
+ protected TestCaseContext tcCtx;
+
+ public SqlppProfiledExecutionTest(TestCaseContext tcCtx) {
+ this.tcCtx = tcCtx;
+ }
+
+ @Test
+ public void test() throws Exception {
+ LangExecutionUtil.test(tcCtx);
+ }
+
+ private static void setNcEndpoints(TestExecutor testExecutor) {
+ final NodeControllerService[] ncs = ExecutionTestUtil.integrationUtil.ncs;
+ final Map<String, InetSocketAddress> ncEndPoints = new HashMap<>();
+ final String ip = InetAddress.getLoopbackAddress().getHostAddress();
+ for (NodeControllerService nc : ncs) {
+ final String nodeId = nc.getId();
+ final INcApplicationContext appCtx = (INcApplicationContext) nc.getApplicationContext();
+ int apiPort = appCtx.getExternalProperties().getNcApiPort();
+ ncEndPoints.put(nodeId, InetSocketAddress.createUnresolved(ip, apiPort));
+ }
+ testExecutor.setNcEndPoints(ncEndPoints);
+ }
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java
index 8a87de7..5d10028 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java
@@ -279,6 +279,7 @@
invokeMethod(rewriter, "rewriteGroupBys");
invokeMethod(rewriter, "rewriteSetOperations");
invokeMethod(rewriter, "inlineColumnAlias");
+ invokeMethod(rewriter, "rewriteSelectExcludeSugar");
invokeMethod(rewriter, "rewriteWindowExpressions");
invokeMethod(rewriter, "rewriteGroupingSets");
invokeMethod(rewriter, "variableCheckAndRewrite");
diff --git a/asterixdb/asterix-app/src/test/resources/cc-batch-lookup.conf b/asterixdb/asterix-app/src/test/resources/cc-batch-lookup.conf
new file mode 100644
index 0000000..6e10481
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/cc-batch-lookup.conf
@@ -0,0 +1,65 @@
+; Licensed to the Apache Software Foundation (ASF) under one
+; or more contributor license agreements. See the NOTICE file
+; distributed with this work for additional information
+; regarding copyright ownership. The ASF licenses this file
+; to you under the Apache License, Version 2.0 (the
+; "License"); you may not use this file except in compliance
+; with the License. You may obtain a copy of the License at
+;
+; http://www.apache.org/licenses/LICENSE-2.0
+;
+; Unless required by applicable law or agreed to in writing,
+; software distributed under the License is distributed on an
+; "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+; KIND, either express or implied. See the License for the
+; specific language governing permissions and limitations
+; under the License.
+
+[nc/asterix_nc1]
+txn.log.dir=target/tmp/asterix_nc1/txnlog
+core.dump.dir=target/tmp/asterix_nc1/coredump
+iodevices=target/tmp/asterix_nc1/iodevice1,
+iodevices=../asterix-server/target/tmp/asterix_nc1/iodevice2
+nc.api.port=19004
+#jvm.args=-agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=5006
+
+[nc/asterix_nc2]
+ncservice.port=9091
+txn.log.dir=target/tmp/asterix_nc2/txnlog
+core.dump.dir=target/tmp/asterix_nc2/coredump
+iodevices=target/tmp/asterix_nc2/iodevice1,../asterix-server/target/tmp/asterix_nc2/iodevice2
+nc.api.port=19005
+#jvm.args=-agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=5007
+
+[nc]
+credential.file=src/test/resources/security/passwd
+python.cmd.autolocate=true
+python.env=FOO=BAR=BAZ,BAR=BAZ
+address=127.0.0.1
+command=asterixnc
+app.class=org.apache.asterix.hyracks.bootstrap.NCApplication
+jvm.args=-Xmx4096m -Dnode.Resolver="org.apache.asterix.external.util.IdentitiyResolverFactory"
+storage.buffercache.pagesize=32KB
+storage.buffercache.size=128MB
+storage.memorycomponent.globalbudget=512MB
+
+[cc]
+address = 127.0.0.1
+app.class=org.apache.asterix.hyracks.bootstrap.CCApplication
+heartbeat.period=2000
+heartbeat.max.misses=25
+credential.file=src/test/resources/security/passwd
+
+[common]
+log.dir = logs/
+log.level = INFO
+compiler.framesize=32KB
+compiler.sortmemory=320KB
+compiler.groupmemory=160KB
+compiler.joinmemory=256KB
+compiler.textsearchmemory=160KB
+compiler.windowmemory=192KB
+compiler.batch.lookup=true
+messaging.frame.size=4096
+messaging.frame.count=512
+
diff --git a/asterixdb/asterix-app/src/test/resources/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-ssl.conf b/asterixdb/asterix-app/src/test/resources/cc-ssl.conf
index 499e9fc..ff7e73b 100644
--- a/asterixdb/asterix-app/src/test/resources/cc-ssl.conf
+++ b/asterixdb/asterix-app/src/test/resources/cc-ssl.conf
@@ -59,6 +59,8 @@
[common]
log.dir = logs/
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/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-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/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/index-through-object/index-through-object.1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.1.sqlpp
new file mode 100644
index 0000000..76f8416
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.1.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.
+ */
+
+/**
+ * Index on top-level field, single OBJECT_ADD function application.
+ * Index "usersNameIdx" should be used.
+ */
+
+DROP DATAVERSE Test IF EXISTS;
+CREATE DATAVERSE Test;
+USE Test;
+
+CREATE TYPE UsersType AS { user_id: bigint };
+CREATE DATASET Users (UsersType) PRIMARY KEY user_id;
+CREATE INDEX usersNameIdx ON Users ( name : string );
+
+FROM Test.Users U
+LET augmentedUser = OBJECT_ADD(U, "favoriteColor", "Green")
+WHERE augmentedUser.name = "Glenn"
+SELECT *;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.10.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.10.sqlpp
new file mode 100644
index 0000000..7e018b9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.10.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.
+ */
+
+/**
+ * Index on top-level field, three OBJECT_ADD function applications.
+ * Primary index should used w/ INLJ.
+ */
+
+DROP DATAVERSE Test IF EXISTS;
+CREATE DATAVERSE Test;
+USE Test;
+
+CREATE TYPE UsersType AS { user_id: bigint };
+CREATE DATASET Users (UsersType) PRIMARY KEY user_id;
+
+FROM Test.Users U1, Test.Users U2
+LET augmentedUser1 = OBJECT_ADD(U1, "favoriteColor", "Green"),
+ augmentedUser2 = OBJECT_ADD(U2, "favoriteFood", "Pizza"),
+ augmentedUser3 = OBJECT_ADD(augmentedUser2, "favoriteColor", "Red"),
+ augmentedUser4 = OBJECT_ADD(augmentedUser3, "favoriteDrink", "Wine")
+WHERE augmentedUser1.name = "John" AND
+ augmentedUser2.name = "Sally" AND
+ TO_BIGINT(U1.bestFriend) /* +indexnl */ = augmentedUser4.user_id
+SELECT *;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.11.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.11.sqlpp
new file mode 100644
index 0000000..782060f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.11.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.
+ */
+
+/**
+ * Two indexes on nested fields, one OBJECT_ADD function application each.
+ * Index should be used in both cases.
+ */
+
+DROP DATAVERSE Test IF EXISTS;
+CREATE DATAVERSE Test;
+USE Test;
+
+CREATE TYPE UsersType AS { user_id: bigint, name: { first: string } };
+CREATE DATASET Users (UsersType) PRIMARY KEY user_id;
+CREATE INDEX firstUsersNameIdx ON Users ( name.first );
+CREATE INDEX lastUsersNameIdx ON Users ( name.last: string );
+
+FROM Test.Users U1
+LET augmentedUser1 = OBJECT_ADD(U1, "favoriteColor", "Green")
+WHERE augmentedUser1.name.first = "Glenn"
+SELECT augmentedUser1.*
+
+UNION ALL
+
+FROM Test.Users U2
+LET augmentedUser2 = OBJECT_ADD(U2, "favoriteFood", "Pizza")
+WHERE augmentedUser2.name.last = "John"
+SELECT augmentedUser2.*;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.2.sqlpp
new file mode 100644
index 0000000..4583e1c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.2.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.
+ */
+
+/**
+ * Index on top-level field, two OBJECT_ADD function applications.
+ * Index "usersNameIdx" should be used.
+ */
+
+DROP DATAVERSE Test IF EXISTS;
+CREATE DATAVERSE Test;
+USE Test;
+
+CREATE TYPE UsersType AS { user_id: bigint };
+CREATE DATASET Users (UsersType) PRIMARY KEY user_id;
+CREATE INDEX usersNameIdx ON Users ( name : string );
+
+FROM Test.Users U
+LET augmentedUser1 = OBJECT_ADD(U, "favoriteColor", "Green"),
+ augmentedUser2 = OBJECT_ADD(augmentedUser1, "favoriteCity", "Irvine")
+WHERE augmentedUser2.name = "Glenn"
+SELECT *;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.3.sqlpp
new file mode 100644
index 0000000..0d81c0f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.3.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.
+ */
+
+/**
+ * Index on nested field, one OBJECT_ADD function application.
+ * Index "usersNameIdx" should be used.
+ */
+
+DROP DATAVERSE Test IF EXISTS;
+CREATE DATAVERSE Test;
+USE Test;
+
+CREATE TYPE UsersType AS { user_id: bigint };
+CREATE DATASET Users (UsersType) PRIMARY KEY user_id;
+CREATE INDEX usersNameIdx ON Users ( name.first : string );
+
+FROM Test.Users U
+LET augmentedUser = OBJECT_ADD(U, "favoriteColor", "Green")
+WHERE augmentedUser.name.first = "Glenn"
+SELECT *;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.4.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.4.sqlpp
new file mode 100644
index 0000000..95a60e6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.4.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.
+ */
+
+/**
+ * Index on top-level field, one OBJECT_REMOVE function application.
+ * Index "usersNameIdx" should be used.
+ */
+
+DROP DATAVERSE Test IF EXISTS;
+CREATE DATAVERSE Test;
+USE Test;
+
+CREATE TYPE UsersType AS { user_id: bigint };
+CREATE DATASET Users (UsersType) PRIMARY KEY user_id;
+CREATE INDEX usersNameIdx ON Users ( name : string );
+
+FROM Test.Users U
+LET augmentedUser = OBJECT_REMOVE(U, "favoriteColor")
+WHERE augmentedUser.name = "Glenn"
+SELECT *;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.5.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.5.sqlpp
new file mode 100644
index 0000000..41af922
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.5.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.
+ */
+
+/**
+ * Index on top-level field, one OBJECT_PUT function application.
+ * Index should NOT used.
+ */
+
+DROP DATAVERSE Test IF EXISTS;
+CREATE DATAVERSE Test;
+USE Test;
+
+CREATE TYPE UsersType AS { user_id: bigint };
+CREATE DATASET Users (UsersType) PRIMARY KEY user_id;
+CREATE INDEX usersNameIdx ON Users ( name : string );
+
+FROM Test.Users U
+LET augmentedUser = OBJECT_PUT(U, "name", "John")
+WHERE augmentedUser.name = "Glenn"
+SELECT *;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.6.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.6.sqlpp
new file mode 100644
index 0000000..8b573b5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.6.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.
+ */
+
+/**
+ * Index on top-level field, one OBJECT_ADD function application.
+ * Primary index should used w/ INLJ.
+ */
+
+DROP DATAVERSE Test IF EXISTS;
+CREATE DATAVERSE Test;
+USE Test;
+
+CREATE TYPE UsersType AS { user_id: bigint };
+CREATE DATASET Users (UsersType) PRIMARY KEY user_id;
+
+FROM Test.Users U1, Test.Users U2
+LET augmentedUser1 = OBJECT_ADD(U1, "favoriteColor", "Green"),
+ augmentedUser2 = OBJECT_ADD(U2, "favoriteFood", "Pizza")
+WHERE augmentedUser1.name = "John" AND
+ augmentedUser2.name = "Sally" AND
+ TO_BIGINT(U1.bestFriend) /* +indexnl */ = augmentedUser2.user_id
+SELECT *;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.7.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.7.sqlpp
new file mode 100644
index 0000000..cb48c1c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.7.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.
+ */
+
+/**
+ * Index on top-level field, OBJECT_ADD followed by OBJECT_REMOVE.
+ * Index should NOT be used.
+ */
+
+DROP DATAVERSE Test IF EXISTS;
+CREATE DATAVERSE Test;
+USE Test;
+
+CREATE TYPE UsersType AS { user_id: bigint };
+CREATE DATASET Users (UsersType) PRIMARY KEY user_id;
+CREATE INDEX usersNameIdx ON Users ( name : string );
+
+FROM Test.Users U
+LET augmentedUser = OBJECT_ADD(OBJECT_REMOVE(U, "name"), "name", "Glenn")
+WHERE augmentedUser.firstName = "Glenn"
+SELECT *;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.8.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.8.sqlpp
new file mode 100644
index 0000000..8d780bf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.8.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.
+ */
+
+/**
+ * Index on top-level field, single OBJECT_CONCAT function application with multiple records.
+ * Index "usersNameIdx" should be used.
+ */
+
+DROP DATAVERSE Test IF EXISTS;
+CREATE DATAVERSE Test;
+USE Test;
+
+CREATE TYPE UsersType AS { user_id: bigint };
+CREATE DATASET Users (UsersType) PRIMARY KEY user_id;
+CREATE INDEX usersNameIdx ON Users ( name : string );
+
+FROM Test.Users U
+LET augmentedUser = OBJECT_CONCAT({"favoriteColor": "Green"}, U, {"birthdate": "10/09/1996"})
+WHERE augmentedUser.name = "Glenn"
+SELECT *;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.9.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.9.sqlpp
new file mode 100644
index 0000000..ad827e0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.9.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.
+ */
+
+/**
+ * Index on top-level field, three OBJECT_ADD function applications.
+ * Index should be used twice (and not used for E1).
+ */
+
+DROP DATAVERSE Test IF EXISTS;
+CREATE DATAVERSE Test;
+USE Test;
+
+CREATE TYPE UsersType AS { user_id: bigint };
+CREATE DATASET Users (UsersType) PRIMARY KEY user_id;
+CREATE INDEX usersNameIdx ON Users ( name : string );
+
+FROM Test.Users U1, Test.Users U2, [{"name": "Glenn"}] E1
+LET augmentedUser1 = OBJECT_ADD(U1, "favoriteColor", "Green"),
+ augmentedUser2 = OBJECT_ADD(U2, "favoriteFood", "Pizza"),
+ augmentedUser3 = OBJECT_ADD(E1, "favoriteColor", "Blue")
+WHERE augmentedUser1.name = "John" AND
+ augmentedUser2.name = "Sally" AND
+ augmentedUser3.name = "Glenn" AND
+ augmentedUser1.bestFriend = augmentedUser2.user_id
+SELECT *;
\ No newline at end of file
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/results/ASTERIXDB-2402.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ASTERIXDB-2402.plan
index 0f996d1..b596e4b 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
@@ -5,12 +5,12 @@
{
-- AGGREGATE |LOCAL|
-- ASSIGN |LOCAL|
- -- MICRO_PRE_CLUSTERED_GROUP_BY[$$215] |LOCAL|
+ -- MICRO_PRE_CLUSTERED_GROUP_BY[$$230] |LOCAL|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- MICRO_STABLE_SORT [$$215(ASC)] |LOCAL|
+ -- MICRO_STABLE_SORT [$$230(ASC)] |LOCAL|
-- ASSIGN |LOCAL|
-- UNNEST |LOCAL|
-- SUBPLAN |LOCAL|
@@ -27,105 +27,55 @@
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- INSERT_DELETE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$199] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$214] |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$267] |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 [$$267(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$267] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$266][$$237] |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[$$322] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- MICRO_PRE_CLUSTERED_GROUP_BY[$$324, $$325] |LOCAL|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$322(ASC), $$324(ASC), $$325(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$322] |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 [$$274(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 [$$274(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 [$$237] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$248, $$250][$$239, $$240] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$248, $$250] |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..4d5a668 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
@@ -30,9 +30,8 @@
-- 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|
+ -- 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/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..5e62aa2 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
@@ -39,9 +39,8 @@
-- 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|
+ -- 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/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-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/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/index-through-object/index-through-object.1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.1.plan
new file mode 100644
index 0000000..ff509f4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.1.plan
@@ -0,0 +1,18 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (Test.Users.Users) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$36(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (Test.Users.usersNameIdx) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.10.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.10.plan
new file mode 100644
index 0000000..f5ab2c2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.10.plan
@@ -0,0 +1,20 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (Test.Users.Users) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$80(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$80] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (Test.Users) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.11.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.11.plan
new file mode 100644
index 0000000..6971ab4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.11.plan
@@ -0,0 +1,39 @@
+-- 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|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (Test.Users.Users) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$78(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (Test.Users.firstUsersNameIdx) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (Test.Users.Users) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$82(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (Test.Users.lastUsersNameIdx) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.2.plan
new file mode 100644
index 0000000..90816ce
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.2.plan
@@ -0,0 +1,18 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (Test.Users.Users) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$48(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (Test.Users.usersNameIdx) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.3.plan
new file mode 100644
index 0000000..6930560
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.3.plan
@@ -0,0 +1,18 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (Test.Users.Users) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$38(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (Test.Users.usersNameIdx) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.4.plan
new file mode 100644
index 0000000..ff509f4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.4.plan
@@ -0,0 +1,18 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (Test.Users.Users) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$36(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (Test.Users.usersNameIdx) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.5.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.5.plan
new file mode 100644
index 0000000..f0f6a03
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.5.plan
@@ -0,0 +1,11 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (Test.Users) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.6.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.6.plan
new file mode 100644
index 0000000..b4aee6b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.6.plan
@@ -0,0 +1,20 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (Test.Users.Users) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$58(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$58] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (Test.Users) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.7.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.7.plan
new file mode 100644
index 0000000..f0f6a03
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.7.plan
@@ -0,0 +1,11 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (Test.Users) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.8.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.8.plan
new file mode 100644
index 0000000..6930560
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.8.plan
@@ -0,0 +1,18 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (Test.Users.Users) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$38(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (Test.Users.usersNameIdx) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.9.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.9.plan
new file mode 100644
index 0000000..1d2e55b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.9.plan
@@ -0,0 +1,47 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |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 [$$82][$$83] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$82] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (Test.Users.Users) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$106(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (Test.Users.usersNameIdx) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$83] |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (Test.Users.Users) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$110(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (Test.Users.usersNameIdx) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
\ No newline at end of file
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/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.plan
index d696f4d..ee5703a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.plan
@@ -23,28 +23,29 @@
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.TweetMessages.TweetMessages) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$86(ASC)] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- RTREE_SEARCH (test.TweetMessages.twmSndLocIx) |PARTITIONED|
- -- BROADCAST_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.TweetMessages.TweetMessages) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$86(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- RTREE_SEARCH (test.TweetMessages.twmSndLocIx) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.TweetMessages.TweetMessages) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.TweetMessages.TweetMessages) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
-- AGGREGATE |UNPARTITIONED|
-- RANDOM_MERGE_EXCHANGE |PARTITIONED|
@@ -66,25 +67,26 @@
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.TweetMessages.TweetMessages) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$86(ASC)] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- RTREE_SEARCH (test.TweetMessages.twmSndLocIx) |PARTITIONED|
- -- BROADCAST_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.TweetMessages.TweetMessages) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$86(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- RTREE_SEARCH (test.TweetMessages.twmSndLocIx) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.TweetMessages.TweetMessages) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.TweetMessages.TweetMessages) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.plan
index d696f4d..ee5703a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.plan
@@ -23,28 +23,29 @@
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.TweetMessages.TweetMessages) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$86(ASC)] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- RTREE_SEARCH (test.TweetMessages.twmSndLocIx) |PARTITIONED|
- -- BROADCAST_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.TweetMessages.TweetMessages) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$86(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- RTREE_SEARCH (test.TweetMessages.twmSndLocIx) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.TweetMessages.TweetMessages) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.TweetMessages.TweetMessages) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
-- AGGREGATE |UNPARTITIONED|
-- RANDOM_MERGE_EXCHANGE |PARTITIONED|
@@ -66,25 +67,26 @@
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.TweetMessages.TweetMessages) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$86(ASC)] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- RTREE_SEARCH (test.TweetMessages.twmSndLocIx) |PARTITIONED|
- -- BROADCAST_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.TweetMessages.TweetMessages) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$86(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- RTREE_SEARCH (test.TweetMessages.twmSndLocIx) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.TweetMessages.TweetMessages) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.TweetMessages.TweetMessages) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_01.plan
index eed20bb..f94167e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_01.plan
@@ -15,14 +15,14 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$23(ASC) ] |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$24(ASC) ] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (tpch.Orders.Orders) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$30(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$31(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/orders-index-search-conjunctive-open_01_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_01_ps.plan
index 61cf933..9c39aef 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_01_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_01_ps.plan
@@ -16,8 +16,8 @@
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$23(ASC)] |PARTITIONED|
- -- RANGE_PARTITION_EXCHANGE [$$23(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$24(ASC)] |PARTITIONED|
+ -- RANGE_PARTITION_EXCHANGE [$$24(ASC)] |PARTITIONED|
-- FORWARD |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
@@ -28,7 +28,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (tpch.Orders.Orders) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$30(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$31(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
@@ -50,7 +50,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (tpch.Orders.Orders) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$30(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$31(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/orders-index-search-conjunctive-open_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_02.plan
index 80fa709..e20acdb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_02.plan
@@ -15,14 +15,14 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$25(ASC) ] |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$26(ASC) ] |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (tpch.Orders.Orders) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$31(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$32(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/orders-index-search-conjunctive-open_02_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_02_ps.plan
index 159cc10..f79649b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_02_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_02_ps.plan
@@ -16,8 +16,8 @@
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$25(ASC)] |PARTITIONED|
- -- RANGE_PARTITION_EXCHANGE [$$25(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$26(ASC)] |PARTITIONED|
+ -- RANGE_PARTITION_EXCHANGE [$$26(ASC)] |PARTITIONED|
-- FORWARD |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
@@ -28,7 +28,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (tpch.Orders.Orders) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$31(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$32(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
@@ -50,7 +50,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (tpch.Orders.Orders) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$31(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$32(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/orders-index-search-conjunctive_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_01.plan
index eed20bb..f94167e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_01.plan
@@ -15,14 +15,14 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$23(ASC) ] |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$24(ASC) ] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (tpch.Orders.Orders) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$30(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$31(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/orders-index-search-conjunctive_01_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_01_ps.plan
index 61cf933..9c39aef 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_01_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_01_ps.plan
@@ -16,8 +16,8 @@
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$23(ASC)] |PARTITIONED|
- -- RANGE_PARTITION_EXCHANGE [$$23(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$24(ASC)] |PARTITIONED|
+ -- RANGE_PARTITION_EXCHANGE [$$24(ASC)] |PARTITIONED|
-- FORWARD |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
@@ -28,7 +28,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (tpch.Orders.Orders) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$30(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$31(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
@@ -50,7 +50,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (tpch.Orders.Orders) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$30(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$31(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/orders-index-search-conjunctive_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_02.plan
index 80fa709..e20acdb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_02.plan
@@ -15,14 +15,14 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$25(ASC) ] |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$26(ASC) ] |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (tpch.Orders.Orders) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$31(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$32(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/orders-index-search-conjunctive_02_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_02_ps.plan
index 159cc10..f79649b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_02_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_02_ps.plan
@@ -16,8 +16,8 @@
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$25(ASC)] |PARTITIONED|
- -- RANGE_PARTITION_EXCHANGE [$$25(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$26(ASC)] |PARTITIONED|
+ -- RANGE_PARTITION_EXCHANGE [$$26(ASC)] |PARTITIONED|
-- FORWARD |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
@@ -28,7 +28,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (tpch.Orders.Orders) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$31(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$32(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
@@ -50,7 +50,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (tpch.Orders.Orders) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$31(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$32(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/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/q01_pricing_summary_report_nt_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/q01_pricing_summary_report_nt_ps.plan
index 0128265..b5c8603 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/q01_pricing_summary_report_nt_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/q01_pricing_summary_report_nt_ps.plan
@@ -22,13 +22,13 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$206, $$207] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$217, $$218] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$206, $$207] |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$181, $$182] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$217, $$218] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$192, $$193] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
@@ -51,13 +51,13 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$206, $$207] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$217, $$218] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$206, $$207] |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$181, $$182] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$217, $$218] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$192, $$193] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
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_issue849-2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query_issue849-2.plan
index e0059ed..4e9ed38 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query_issue849-2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query_issue849-2.plan
@@ -20,12 +20,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$44][$$46] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$47][$$49] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN (test.s) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$46] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$49] |PARTITIONED|
-- STREAM_PROJECT |UNPARTITIONED|
-- ASSIGN |UNPARTITIONED|
-- UNNEST |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query_issue849.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query_issue849.plan
index 5d16539..3a4e8a7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query_issue849.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query_issue849.plan
@@ -20,8 +20,8 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$45][$$44] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$45] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$48][$$47] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$48] |PARTITIONED|
-- ASSIGN |UNPARTITIONED|
-- UNNEST |UNPARTITIONED|
-- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
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/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/tpcds/query-ASTERIXDB-1581-correlated.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1581-correlated.plan
index d61947a..d7bad18 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1581-correlated.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1581-correlated.plan
@@ -8,7 +8,7 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$173] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$180] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- AGGREGATE |LOCAL|
@@ -18,9 +18,9 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$173][$$215] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$180][$$222] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$157] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$164] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- AGGREGATE |LOCAL|
@@ -28,13 +28,13 @@
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$157(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$164(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$157][$$171] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$164][$$178] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$137] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$144] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- AGGREGATE |LOCAL|
@@ -44,7 +44,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$137][$$147] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$144][$$154] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -60,7 +60,7 @@
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$198] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$205] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -73,35 +73,35 @@
-- DATASOURCE_SCAN (tpcds.store_sales) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$171] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$178] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- SORT_GROUP_BY[$$220, $$221] |PARTITIONED|
+ -- SORT_GROUP_BY[$$227, $$228] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$220, $$221] |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$158, $$161] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$227, $$228] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$165, $$168] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$158(ASC), $$161(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$165(ASC), $$168(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$150][$$149] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$157][$$156] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$196] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$204] |PARTITIONED|
-- RUNNING_AGGREGATE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -110,7 +110,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$199] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$206] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- AGGREGATE |LOCAL|
@@ -120,7 +120,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$199][$$200] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$206][$$207] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
@@ -133,7 +133,7 @@
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$198] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$205] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -146,7 +146,7 @@
-- DATASOURCE_SCAN (tpcds.store_sales) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$149] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$156] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
@@ -157,30 +157,30 @@
-- DATASOURCE_SCAN (tpcds.store_sales) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$215] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$222] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- SORT_GROUP_BY[$$226, $$227] |PARTITIONED|
+ -- SORT_GROUP_BY[$$233, $$234] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$226, $$227] |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$174, $$177] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$233, $$234] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$181, $$184] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$174(ASC), $$177(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$181(ASC), $$184(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$152][$$151] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$152] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$159][$$158] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$159] |PARTITIONED|
-- RUNNING_AGGREGATE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -189,7 +189,7 @@
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$178] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$185] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- AGGREGATE |LOCAL|
@@ -197,18 +197,18 @@
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$178(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$185(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$178][$$181] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$185][$$188] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$199] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$206] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- AGGREGATE |LOCAL|
@@ -218,7 +218,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$199][$$200] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$206][$$207] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
@@ -231,7 +231,7 @@
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$198] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$205] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -244,31 +244,31 @@
-- DATASOURCE_SCAN (tpcds.store_sales) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$181] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$188] |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- SORT_GROUP_BY[$$223, $$224] |PARTITIONED|
+ -- SORT_GROUP_BY[$$230, $$231] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$223, $$224] |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$193, $$194] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$230, $$231] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$200, $$201] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$193(ASC), $$194(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$200(ASC), $$201(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$196][$$198] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$204][$$205] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$196] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$204] |PARTITIONED|
-- RUNNING_AGGREGATE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -277,7 +277,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$199] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$206] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- AGGREGATE |LOCAL|
@@ -287,7 +287,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$199][$$200] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$206][$$207] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
@@ -300,7 +300,7 @@
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$198] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$205] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -315,7 +315,7 @@
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$198] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$205] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -328,7 +328,7 @@
-- DATASOURCE_SCAN (tpcds.store_sales) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$151] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$158] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |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..4d5a668
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/atomic-and-array-queries/query8.plan
@@ -0,0 +1,37 @@
+-- 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|
+ -- 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..5e62aa2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/atomic-and-array-queries/query9.plan
@@ -0,0 +1,46 @@
+-- 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|
+ -- 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-10.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-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-use-index/hints-use-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-use-index/hints-use-index-11.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-11.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-11.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-12.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-12.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-12.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-use-index/hints-use-index-13.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-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-use-index/hints-use-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-use-index/hints-use-index-14.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-14.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-14.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-15.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-15.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-15.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-use-index/hints-use-index-16.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-16.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-16.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-use-index/hints-use-index-17.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-17.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-17.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-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-3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-3.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-3.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-4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-4.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-4.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-use-index/hints-use-index-5.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-5.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-5.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-use-index/hints-use-index-6.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-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-use-index/hints-use-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-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-selection/hints-use-index/hints-use-index-8.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-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-use-index/hints-use-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-use-index/hints-use-index-9.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-9.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-9.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_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..a357a82
--- /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[$$399] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- HASH_PARTITION_EXCHANGE [$$399] |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[$$390, $$391, $$392, $$393, $$394, $$395, $$396, $$397] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- HASH_PARTITION_EXCHANGE [$$390, $$391, $$392, $$393, $$394, $$395, $$396, $$397] |PARTITIONED|
+ -- SORT_GROUP_BY[$$352, $$378, $$377, $$354, $$355, $$351, $$350, $$379] |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 [$$354][$$365] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$354] |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|
+ -- HASH_PARTITION_EXCHANGE [$$365] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$355][$$366] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$355] |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 [$$366] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$351, $$350][$$352, $$362] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$351, $$350] |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 [$$352, $$362] |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/index-through-object/index-through-object.1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.1.plan
new file mode 100644
index 0000000..f0f6a03
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.1.plan
@@ -0,0 +1,11 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (Test.Users) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.2.plan
new file mode 100644
index 0000000..f0f6a03
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.2.plan
@@ -0,0 +1,11 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (Test.Users) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.3.plan
new file mode 100644
index 0000000..f0f6a03
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.3.plan
@@ -0,0 +1,11 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (Test.Users) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.4.plan
new file mode 100644
index 0000000..f0f6a03
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.4.plan
@@ -0,0 +1,11 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (Test.Users) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.8.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.8.plan
new file mode 100644
index 0000000..f0f6a03
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.8.plan
@@ -0,0 +1,11 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (Test.Users) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.9.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.9.plan
new file mode 100644
index 0000000..75a50b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.9.plan
@@ -0,0 +1,41 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |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 [$$88][$$89] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$88] |PARTITIONED|
+ -- ASSIGN |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.Users) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$89] |PARTITIONED|
+ -- STREAM_SELECT |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.Users) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
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..e4167f0
--- /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 [$$24(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..8e1370f
--- /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 [$$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|
+ -- 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..c87d9ce
--- /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 [$$26(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..6db1de8
--- /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 [$$26(ASC)] |PARTITIONED|
+ -- RANGE_PARTITION_EXCHANGE [$$26(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..e4167f0
--- /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 [$$24(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..8e1370f
--- /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 [$$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|
+ -- 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..c87d9ce
--- /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 [$$26(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..6db1de8
--- /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 [$$26(ASC)] |PARTITIONED|
+ -- RANGE_PARTITION_EXCHANGE [$$26(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_issue849.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/query_issue849.plan
new file mode 100644
index 0000000..59b09a9
--- /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 [$$55(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$55] |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_shipping.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpch/q12_shipping.plan
new file mode 100644
index 0000000..b20e06d
--- /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[$$135] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- HASH_PARTITION_EXCHANGE [$$135] |PARTITIONED|
+ -- SORT_GROUP_BY[$$118] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$122][$$126] |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|
+ -- HASH_PARTITION_EXCHANGE [$$126] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$124][$$118] |PARTITIONED|
+ -- RANDOM_PARTITION_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
+ -- BROADCAST_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|
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..c970de6
--- /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[$$135] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- HASH_PARTITION_EXCHANGE [$$135] |PARTITIONED|
+ -- SORT_GROUP_BY[$$118] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$122][$$125] |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 [$$124][$$118] |PARTITIONED|
+ -- RANDOM_PARTITION_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
+ -- BROADCAST_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|
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..d037732
--- /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[$$135] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- HASH_PARTITION_EXCHANGE [$$135] |PARTITIONED|
+ -- SORT_GROUP_BY[$$118] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$122][$$125] |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 [$$124][$$118] |PARTITIONED|
+ -- RANDOM_PARTITION_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
+ -- BROADCAST_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|
+ -- 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[$$135] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- HASH_PARTITION_EXCHANGE [$$135] |PARTITIONED|
+ -- SORT_GROUP_BY[$$118] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$122][$$125] |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 [$$124][$$118] |PARTITIONED|
+ -- RANDOM_PARTITION_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
+ -- BROADCAST_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|
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..2c1d311
--- /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[$$135] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- HASH_PARTITION_EXCHANGE [$$135] |PARTITIONED|
+ -- SORT_GROUP_BY[$$118] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$122][$$126] |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|
+ -- HASH_PARTITION_EXCHANGE [$$126] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$124][$$118] |PARTITIONED|
+ -- RANDOM_PARTITION_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
+ -- BROADCAST_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|
+ -- 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[$$135] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- HASH_PARTITION_EXCHANGE [$$135] |PARTITIONED|
+ -- SORT_GROUP_BY[$$118] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$122][$$126] |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|
+ -- HASH_PARTITION_EXCHANGE [$$126] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$124][$$118] |PARTITIONED|
+ -- RANDOM_PARTITION_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
+ -- BROADCAST_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|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/only_batch_lookup.xml b/asterixdb/asterix-app/src/test/resources/runtimets/only_batch_lookup.xml
new file mode 100644
index 0000000..334dd52
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/only_batch_lookup.xml
@@ -0,0 +1,23 @@
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<!--
+ ! Licensed to the Apache Software Foundation (ASF) under one
+ ! or more contributor license agreements. See the NOTICE file
+ ! distributed with this work for additional information
+ ! regarding copyright ownership. The ASF licenses this file
+ ! to you under the Apache License, Version 2.0 (the
+ ! "License"); you may not use this file except in compliance
+ ! with the License. You may obtain a copy of the License at
+ !
+ ! http://www.apache.org/licenses/LICENSE-2.0
+ !
+ ! Unless required by applicable law or agreed to in writing,
+ ! software distributed under the License is distributed on an
+ ! "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ ! KIND, either express or implied. See the License for the
+ ! specific language governing permissions and limitations
+ ! under the License.
+ !-->
+<test-suite xmlns="urn:xml.testframework.asterix.apache.org" ResultOffsetPath="results" QueryOffsetPath="queries_sqlpp" QueryFileExtension=".sqlpp">
+ <test-group name="failed">
+ </test-group>
+</test-suite>
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.1.ddl.sqlpp
new file mode 100644
index 0000000..c0aadca
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.1.ddl.sqlpp
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test1 if exists;
+create dataverse test1;
+
+use test1;
+
+create type t1 as closed {
+ c1 : bigint,
+ c2 : bigint
+};
+
+create dataset ds1(t1) primary key c1;
+
+create index idx1 on ds1(c2);
+
+create view v1 as select value v from ds1 v;
+
+create synonym s1 for v1;
+
+create function f1() {
+ select "Dataverse" as k, DataverseName as dv, DataverseName as n from Metadata.`Dataverse`
+ union all
+ select "Datatype" as k, DataverseName as dv, DatatypeName as n from Metadata.`Datatype`
+ union all
+ select "Dataset" as k, DataverseName as dv, DatasetName as n from Metadata.`Dataset`
+ union all
+ select "Index" as k, DataverseName as dv, IndexName as n from Metadata.`Index` where not(isPrimary)
+ union all
+ select "Synonym" as k, DataverseName as dv, SynonymName as n from Metadata.`Synonym`
+ union all
+ select "Function" as k, DataverseName as dv, Name as n from Metadata.`Function`
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-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/error-handling/no-field-type/no-field-type.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/no-field-type/no-field-type.1.ddl.sqlpp
new file mode 100644
index 0000000..e3cc4be
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/no-field-type/no-field-type.1.ddl.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: Verify that a proper error message is reported when the type of the indexed field could not be found
+ */
+
+DROP DATAVERSE testDv IF EXISTS;
+CREATE DATAVERSE testDv;
+USE testDv;
+
+CREATE TYPE t0 as {a: string};
+CREATE TYPE t1 as {id: int, obj_f: t0, array_f: [t0]};
+CREATE DATASET ds(t1) PRIMARY KEY id;
+
+CREATE INDEX idx1 ON ds(UNNEST open_array_f) EXCLUDE UNKNOWN KEY;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/no-field-type/no-field-type.2.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/no-field-type/no-field-type.2.ddl.sqlpp
new file mode 100644
index 0000000..141bb4c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/no-field-type/no-field-type.2.ddl.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: Verify that a proper error message is reported when the type of the indexed field could not be found
+ */
+
+DROP DATAVERSE testDv IF EXISTS;
+CREATE DATAVERSE testDv;
+USE testDv;
+
+CREATE TYPE t0 as {a: string};
+CREATE TYPE t1 as {id: int, obj_f: t0, array_f: [t0]};
+CREATE DATASET ds(t1) PRIMARY KEY id;
+
+CREATE INDEX idx1 ON ds(UNNEST open_array_f UNNEST nested_array) EXCLUDE UNKNOWN KEY;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/no-field-type/no-field-type.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/no-field-type/no-field-type.3.ddl.sqlpp
new file mode 100644
index 0000000..11f15e0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/no-field-type/no-field-type.3.ddl.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: Verify that a proper error message is reported when the type of the indexed field could not be found
+ */
+
+DROP DATAVERSE testDv IF EXISTS;
+CREATE DATAVERSE testDv;
+USE testDv;
+
+CREATE TYPE t0 as {a: string};
+CREATE TYPE t1 as {id: int, obj_f: t0, array_f: [t0]};
+CREATE DATASET ds(t1) PRIMARY KEY id;
+
+CREATE INDEX idx1 ON ds(UNNEST array_f UNNEST nested_array) EXCLUDE UNKNOWN KEY;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/no-field-type/no-field-type.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/no-field-type/no-field-type.4.ddl.sqlpp
new file mode 100644
index 0000000..996ae2d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/no-field-type/no-field-type.4.ddl.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: Verify that a proper error message is reported when the type of the indexed field could not be found
+ */
+
+DROP DATAVERSE testDv IF EXISTS;
+CREATE DATAVERSE testDv;
+USE testDv;
+
+CREATE TYPE t0 as {a: string};
+CREATE TYPE t1 as {id: int, obj_f: t0, array_f: [t0]};
+CREATE DATASET ds(t1) PRIMARY KEY id;
+
+CREATE INDEX idx1 ON ds(UNNEST array_f SELECT proj1) EXCLUDE UNKNOWN KEY;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/no-field-type/no-field-type.5.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/no-field-type/no-field-type.5.ddl.sqlpp
new file mode 100644
index 0000000..3f314a3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/no-field-type/no-field-type.5.ddl.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: Verify that a proper error message is reported when the type of the indexed field could not be found
+ */
+
+DROP DATAVERSE testDv IF EXISTS;
+CREATE DATAVERSE testDv;
+USE testDv;
+
+CREATE TYPE t0 as {a: string};
+CREATE TYPE t1 as {id: int, obj_f: t0, array_f: [t0]};
+CREATE DATASET ds(t1) PRIMARY KEY id;
+
+CREATE INDEX idx1 ON ds(UNNEST array_f UNNEST nested_array SELECT proj1) EXCLUDE UNKNOWN KEY;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/no-field-type/no-field-type.6.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/no-field-type/no-field-type.6.ddl.sqlpp
new file mode 100644
index 0000000..2dd04c1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/no-field-type/no-field-type.6.ddl.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: Verify that a proper error message is reported when the type of the indexed field could not be found
+ */
+
+DROP DATAVERSE testDv IF EXISTS;
+CREATE DATAVERSE testDv;
+USE testDv;
+
+CREATE TYPE t0 as {a: string};
+CREATE TYPE t1 as {id: int, obj_f: t0, array_f: [t0]};
+CREATE DATASET ds(t1) PRIMARY KEY id;
+
+CREATE INDEX idx1 ON ds(UNNEST open_array_f SELECT proj1) EXCLUDE UNKNOWN KEY;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/no-field-type/no-field-type.7.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/no-field-type/no-field-type.7.ddl.sqlpp
new file mode 100644
index 0000000..b5d09af
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/no-field-type/no-field-type.7.ddl.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: Verify that a proper error message is reported when the type of the indexed field could not be found
+ */
+
+DROP DATAVERSE testDv IF EXISTS;
+CREATE DATAVERSE testDv;
+USE testDv;
+
+CREATE TYPE t0 as {a: string};
+CREATE TYPE t1 as {id: int, obj_f: t0, array_f: [t0]};
+CREATE DATASET ds(t1) PRIMARY KEY id;
+
+CREATE INDEX idx1 ON ds(UNNEST open_array_f UNNEST nested_array SELECT proj1) EXCLUDE UNKNOWN KEY;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/no-field-type/no-field-type.8.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/no-field-type/no-field-type.8.ddl.sqlpp
new file mode 100644
index 0000000..8c95583
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/no-field-type/no-field-type.8.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 testDv IF EXISTS;
\ 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/array_fun/array_binary_search/array_binary_search.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_binary_search/array_binary_search.1.ddl.sqlpp
new file mode 100644
index 0000000..601eb0c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_binary_search/array_binary_search.1.ddl.sqlpp
@@ -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.
+ */
+
+drop dataverse TinySocial if exists;
+create dataverse TinySocial;
+
+use TinySocial;
+
+create type addressType as closed {state: string, country: string, zip_code: int?};
+create type openType as {id: int};
+create type closedType as closed {id: int, list_f: [addressType]};
+
+create dataset openDs(openType) primary key id;
+create dataset closedDs(closedType) primary key id;
+
+create type TinySocial.TwitterUserType as
+{
+ `screen-name` : string,
+ lang : string,
+ friends_count : bigint,
+ statuses_count : bigint,
+ name : string,
+ followers_count : bigint
+};
+
+create type TinySocial.TweetMessageType as
+ closed {
+ tweetid : string,
+ user : TwitterUserType,
+ `sender-location` : point?,
+ `send-time` : datetime,
+ `referred-topics` : {{string}},
+ `message-text` : string
+};
+
+create type t1 AS {
+
+};
+
+create type t2 AS {
+id: int,
+compType: t1
+};
+
+create dataset TweetMessages(TweetMessageType) primary key tweetid hints (`CARDINALITY`=`100`);
+create dataset d1(t2) primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_binary_search/array_binary_search.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_binary_search/array_binary_search.2.update.sqlpp
new file mode 100644
index 0000000..7e8e5ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_binary_search/array_binary_search.2.update.sqlpp
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use TinySocial;
+
+load dataset TweetMessages using localfs ((`path`=`asterix_nc1://data/tinysocial/twm.adm`),(`format`=`adm`));
+
+insert into d1([
+{"id":1, "compType":{"sth":33}},
+{"id":2, "compType":{"sth":44}, "followers":["John Green", "Emily Jones"]}
+]);
+
+insert into openDs([
+{"id": 1, "list_f": [ [1,2,1], [9999,3] ]},
+{"id": 2, "list_f": [ ["white","blue","magenta"], ["red", "black"] ]},
+{"id": 3, "list_f": [ 1 , 2 ]},
+{"id": 4, "list_f": [ {"state": "OH", "country": "US"} , {"state": "CA", "country": "US", "zip_code": 92863} ]},
+{"id": 5, "list_f": [ {"state": "OR", "country": "US", "zip_code": null} , {"state": "IL", "country": "US", "zip_code": 92863} ]},
+{"id": 6, "list_f": null},
+{"id": 7}
+]);
+
+insert into closedDs([
+{"id": 1, "list_f": [ {"state": "OH", "country": "US"} , {"state": "CA", "country": "US", "zip_code": 92863} ]},
+{"id": 2, "list_f": [ {"state": "OR", "country": "US", "zip_code": null} , {"state": "IL", "country": "US", "zip_code": 92863} ]}
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_binary_search/array_binary_search.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_binary_search/array_binary_search.3.query.sqlpp
new file mode 100644
index 0000000..eebfec8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_binary_search/array_binary_search.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.
+ */
+
+use TinySocial;
+
+FROM [
+ {"id":1, "t1": (select array_binary_search(null, 1))},
+ {"id":2, "t2": (select array_binary_search(missing, 1))},
+ {"id":3, "t3": (select array_binary_search([], 1))},
+ {"id":4, "t4": (select array_binary_search([1,2,3,4,5,6,7], 4))},
+ {"id":5, "t5": (select array_binary_search([1,2,3,4,5,6,7], 1))},
+ {"id":6, "t6": (select array_binary_search([1,2,3,4,5,6,7], 7))},
+ {"id":7, "t7": (select array_binary_search(["a", "b", "c", "d", "e", "f", "g"], "c"))},
+ {"id":8, "t8": (select array_binary_search(["a", "b", "c", "d", "e", "f", "g"], "a"))},
+ {"id":9, "t9": (select array_binary_search(["a", "b", "c", "d", "e", "f", "g"], "g"))},
+ {"id":10, "t10": (select array_binary_search([1,2,3,4,5,6,7,8], 3.0))},
+ {"id":11, "t11": (select array_binary_search([1,2,3,4,5,6,7,8], 8.0))},
+ {"id":12, "t12": (select array_binary_search([1,2,3,4,5,6,7,8], 2.5))},
+ {"id":13, "t13": (select array_binary_search("not-an-array", 3))},
+ {"id":14, "t14": (select array_binary_search([ ["a", "b"], ["b", "c"], ["c", "d"], ["d", "e"] ], ["a", "b"]))},
+ {"id":15, "t15": (select array_binary_search([ ["a", "b"], ["b", "c"], ["c", "d"], ["d", "e"] ], ["d", "e"]))},
+ {"id":16, "t16": (from openDs select array_binary_search(list_f, [9999,3]) order by id)},
+ {"id":17, "t17": (from openDs select array_binary_search(list_f, {"state": "OH", "country": "US"}) order by id)},
+ {"id":18, "t18": (from closedDs select array_binary_search(list_f, {"state": "OH", "country": "US"}) order by id)},
+ {"id":19, "t19": (select array_binary_search([{"id": 1, "age": 34}, {"id": 2, "age": 29}, {"id": 3, "age": 90}, {"id": 4, "age": 10}], {"id": 2, "age": 29} ))},
+ {"id":20, "t20": (select array_binary_search([0,0,1,1,1,2,3,3,3,3,4,5,6,6,6,7], 3))},
+ {"id":21, "t21": (select array_binary_search(["a", "b", "b", "b", "c", "d", "e", "e", "f", "f", "f", "g", "h", "i", "j"], "f"))},
+ {"id":22, "t22": (select array_binary_search( {{1, 2, 3, 4, 5}}, 3))}
+] as d
+
+SELECT VALUE d
+ORDER BY d.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_binary_search/array_binary_search.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_binary_search/array_binary_search.4.ddl.sqlpp
new file mode 100644
index 0000000..3f8c8ec
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_binary_search/array_binary_search.4.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 TinySocial;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_move/array_move.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_move/array_move.1.ddl.sqlpp
new file mode 100644
index 0000000..601eb0c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_move/array_move.1.ddl.sqlpp
@@ -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.
+ */
+
+drop dataverse TinySocial if exists;
+create dataverse TinySocial;
+
+use TinySocial;
+
+create type addressType as closed {state: string, country: string, zip_code: int?};
+create type openType as {id: int};
+create type closedType as closed {id: int, list_f: [addressType]};
+
+create dataset openDs(openType) primary key id;
+create dataset closedDs(closedType) primary key id;
+
+create type TinySocial.TwitterUserType as
+{
+ `screen-name` : string,
+ lang : string,
+ friends_count : bigint,
+ statuses_count : bigint,
+ name : string,
+ followers_count : bigint
+};
+
+create type TinySocial.TweetMessageType as
+ closed {
+ tweetid : string,
+ user : TwitterUserType,
+ `sender-location` : point?,
+ `send-time` : datetime,
+ `referred-topics` : {{string}},
+ `message-text` : string
+};
+
+create type t1 AS {
+
+};
+
+create type t2 AS {
+id: int,
+compType: t1
+};
+
+create dataset TweetMessages(TweetMessageType) primary key tweetid hints (`CARDINALITY`=`100`);
+create dataset d1(t2) primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_move/array_move.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_move/array_move.2.update.sqlpp
new file mode 100644
index 0000000..7e8e5ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_move/array_move.2.update.sqlpp
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use TinySocial;
+
+load dataset TweetMessages using localfs ((`path`=`asterix_nc1://data/tinysocial/twm.adm`),(`format`=`adm`));
+
+insert into d1([
+{"id":1, "compType":{"sth":33}},
+{"id":2, "compType":{"sth":44}, "followers":["John Green", "Emily Jones"]}
+]);
+
+insert into openDs([
+{"id": 1, "list_f": [ [1,2,1], [9999,3] ]},
+{"id": 2, "list_f": [ ["white","blue","magenta"], ["red", "black"] ]},
+{"id": 3, "list_f": [ 1 , 2 ]},
+{"id": 4, "list_f": [ {"state": "OH", "country": "US"} , {"state": "CA", "country": "US", "zip_code": 92863} ]},
+{"id": 5, "list_f": [ {"state": "OR", "country": "US", "zip_code": null} , {"state": "IL", "country": "US", "zip_code": 92863} ]},
+{"id": 6, "list_f": null},
+{"id": 7}
+]);
+
+insert into closedDs([
+{"id": 1, "list_f": [ {"state": "OH", "country": "US"} , {"state": "CA", "country": "US", "zip_code": 92863} ]},
+{"id": 2, "list_f": [ {"state": "OR", "country": "US", "zip_code": null} , {"state": "IL", "country": "US", "zip_code": 92863} ]}
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_move/array_move.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_move/array_move.3.query.sqlpp
new file mode 100644
index 0000000..8659ed4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_move/array_move.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.
+ */
+
+use TinySocial;
+
+FROM [
+ {"id":1, "t1" : (select array_move([0,1,2,3,4,5], 1, 3))},
+ {"id":2, "t2" : (select array_move([0,1,2,3,4,5], -1, -3))},
+ {"id":3, "t3" : (select array_move(["a", "b", "c", "d", "e"], 0, 2))},
+ {"id":4, "t4" : (select array_move(["a", "b", "c", "d", "e"], -2, -4))},
+ {"id":5, "t5" : (select array_move(null, 0, 1))},
+ {"id":6, "t6" : (select array_move([], 0, 1))},
+ {"id":7, "t7" : (select array_move([1,2,3,4,5], null, 1))},
+ {"id":8, "t8" : (select array_move([1,2,3,4,5], 0, null))},
+ {"id":9, "t9": (from openDs select array_move(list_f, 0, 1) order by id)},
+ {"id":10, "t10": (from openDs select array_move(list_f, -1, -2) order by id)},
+ {"id":11, "t11": (from openDs select array_move(list_f, null, 1) order by id)},
+ {"id":12, "t12": (from openDs select array_move(list_f, missing, 1) order by id)},
+ {"id":13, "t13": (from closedDs select array_move(list_f, 0, 1) order by id)},
+ {"id":14, "t14": (from closedDs select array_move(list_f, -1, -2) order by id)},
+ {"id":15, "t15": (from closedDs select array_move(list_f, null, 1) order by id)},
+ {"id":16, "t16": (from closedDs select array_move(list_f, missing, 1) order by id)},
+ {"id":17, "t17": (select array_move("not-an-array", 0, 1))},
+ {"id":18, "t18": (select array_move([1,2,3,4,5], 7, 8))},
+ {"id":19, "t19": (select array_move([1,2,3,4,5], "not-an-int", 1))},
+ {"id":20, "t20": (select array_move([1,2,3,4,5], 0, "not-an-int"))},
+ {"id":21, "t21": (select array_move(missing, 0, 1))},
+ {"id":22, "t22": (select array_move([1,2,3,4,5], missing, 1))},
+ {"id":23, "t23": (select array_move([1,2,3,4,5], 0, missing))},
+ {"id":24, "t24": (select array_move([1,2,3,4,5], 4, 0))},
+ {"id":25, "t25": (select array_move([1,2,3,4,5], 3, 1))},
+ {"id":26, "t26": (select array_move( {{1,2,3,4,5}}, 0, 3 ))}
+] as d
+
+SELECT VALUE d
+ORDER BY d.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_move/array_move.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_move/array_move.4.ddl.sqlpp
new file mode 100644
index 0000000..3f8c8ec
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_move/array_move.4.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 TinySocial;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_swap/array_swap.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_swap/array_swap.1.ddl.sqlpp
new file mode 100644
index 0000000..601eb0c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_swap/array_swap.1.ddl.sqlpp
@@ -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.
+ */
+
+drop dataverse TinySocial if exists;
+create dataverse TinySocial;
+
+use TinySocial;
+
+create type addressType as closed {state: string, country: string, zip_code: int?};
+create type openType as {id: int};
+create type closedType as closed {id: int, list_f: [addressType]};
+
+create dataset openDs(openType) primary key id;
+create dataset closedDs(closedType) primary key id;
+
+create type TinySocial.TwitterUserType as
+{
+ `screen-name` : string,
+ lang : string,
+ friends_count : bigint,
+ statuses_count : bigint,
+ name : string,
+ followers_count : bigint
+};
+
+create type TinySocial.TweetMessageType as
+ closed {
+ tweetid : string,
+ user : TwitterUserType,
+ `sender-location` : point?,
+ `send-time` : datetime,
+ `referred-topics` : {{string}},
+ `message-text` : string
+};
+
+create type t1 AS {
+
+};
+
+create type t2 AS {
+id: int,
+compType: t1
+};
+
+create dataset TweetMessages(TweetMessageType) primary key tweetid hints (`CARDINALITY`=`100`);
+create dataset d1(t2) primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_swap/array_swap.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_swap/array_swap.2.update.sqlpp
new file mode 100644
index 0000000..7e8e5ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_swap/array_swap.2.update.sqlpp
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use TinySocial;
+
+load dataset TweetMessages using localfs ((`path`=`asterix_nc1://data/tinysocial/twm.adm`),(`format`=`adm`));
+
+insert into d1([
+{"id":1, "compType":{"sth":33}},
+{"id":2, "compType":{"sth":44}, "followers":["John Green", "Emily Jones"]}
+]);
+
+insert into openDs([
+{"id": 1, "list_f": [ [1,2,1], [9999,3] ]},
+{"id": 2, "list_f": [ ["white","blue","magenta"], ["red", "black"] ]},
+{"id": 3, "list_f": [ 1 , 2 ]},
+{"id": 4, "list_f": [ {"state": "OH", "country": "US"} , {"state": "CA", "country": "US", "zip_code": 92863} ]},
+{"id": 5, "list_f": [ {"state": "OR", "country": "US", "zip_code": null} , {"state": "IL", "country": "US", "zip_code": 92863} ]},
+{"id": 6, "list_f": null},
+{"id": 7}
+]);
+
+insert into closedDs([
+{"id": 1, "list_f": [ {"state": "OH", "country": "US"} , {"state": "CA", "country": "US", "zip_code": 92863} ]},
+{"id": 2, "list_f": [ {"state": "OR", "country": "US", "zip_code": null} , {"state": "IL", "country": "US", "zip_code": 92863} ]}
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_swap/array_swap.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_swap/array_swap.3.query.sqlpp
new file mode 100644
index 0000000..6689330
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_swap/array_swap.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.
+ */
+
+use TinySocial;
+
+FROM [
+ {"id":1, "t1" : (select array_swap([0,1,2,3,4,5], 1, 3))},
+ {"id":2, "t2" : (select array_swap([0,1,2,3,4,5], -1, -3))},
+ {"id":3, "t3" : (select array_swap(["a", "b", "c", "d", "e"], 0, 2))},
+ {"id":4, "t4" : (select array_swap(["a", "b", "c", "d", "e"], -2, -4))},
+ {"id":5, "t5" : (select array_swap(null, 0, 1))},
+ {"id":6, "t6" : (select array_swap([], 0, 1))},
+ {"id":7, "t7" : (select array_swap([1,2,3,4,5], null, 1))},
+ {"id":8, "t8" : (select array_swap([1,2,3,4,5], 0, null))},
+ {"id":9, "t9": (from openDs select array_swap(list_f, 0, 1) order by id)},
+ {"id":10, "t10": (from openDs select array_swap(list_f, -1, -2) order by id)},
+ {"id":11, "t11": (from openDs select array_swap(list_f, null, 1) order by id)},
+ {"id":12, "t12": (from openDs select array_swap(list_f, missing, 1) order by id)},
+ {"id":13, "t13": (from closedDs select array_swap(list_f, 0, 1) order by id)},
+ {"id":14, "t14": (from closedDs select array_swap(list_f, -1, -2) order by id)},
+ {"id":15, "t15": (from closedDs select array_swap(list_f, null, 1) order by id)},
+ {"id":16, "t16": (from closedDs select array_swap(list_f, missing, 1) order by id)},
+ {"id":17, "t17" : (select array_swap("not-an-array", 0, 1))},
+ {"id":18, "t18" : (select array_swap([1,2,3,4,5], 7, 8))},
+ {"id":19, "t19" : (select array_swap([1,2,3,4,5], "not-an-int", 1))},
+ {"id":20, "t20" : (select array_swap([1,2,3,4,5], 0, "not-an-int"))},
+ {"id":21, "t21" : (select array_swap(missing, 0, 1))},
+ {"id":22, "t22" : (select array_swap([1,2,3,4,5], missing, 1))},
+ {"id":23, "t23" : (select array_swap([1,2,3,4,5], 0, missing))},
+ {"id":24, "t24": (select array_swap([1,2,3,4,5], 4, 0))},
+ {"id":25, "t25": (select array_swap([1,2,3,4,5], 3, 1))},
+ {"id":26, "t26": (select array_swap({{1,2,3,4,5}}, 1, 3))}
+] as d
+
+SELECT VALUE d
+ORDER BY d.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_swap/array_swap.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_swap/array_swap.4.ddl.sqlpp
new file mode 100644
index 0000000..3f8c8ec
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_swap/array_swap.4.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 TinySocial;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/AsyncDeferredQueries.xml b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/AsyncDeferredQueries.xml
index 47eb4395..9a5e17e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/AsyncDeferredQueries.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/AsyncDeferredQueries.xml
@@ -20,44 +20,52 @@
<test-case FilePath="async-deferred">
<compilation-unit name="async-failed">
<output-dir compare="Clean-JSON">async-failed</output-dir>
+ <parameter name="profile" value="timings" type="string"/>
<expected-error>Injected failure in inject-failure</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="async-deferred">
<compilation-unit name="async-compilation-failed">
+ <parameter name="profile" value="timings" type="string"/>
<output-dir compare="Clean-JSON">async-compilation-failed</output-dir>
<expected-error>Cannot find dataset gargel</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="async-deferred">
<compilation-unit name="deferred">
+ <parameter name="profile" value="timings" type="string"/>
<output-dir compare="Clean-JSON">deferred</output-dir>
</compilation-unit>
</test-case>
<test-case FilePath="async-deferred">
<compilation-unit name="async">
+ <parameter name="profile" value="timings" type="string"/>
<output-dir compare="Clean-JSON">async</output-dir>
</compilation-unit>
</test-case>
<test-case FilePath="async-deferred">
<compilation-unit name="async-repeated">
+ <parameter name="profile" value="timings" type="string"/>
<output-dir compare="Clean-JSON">async-repeated</output-dir>
</compilation-unit>
</test-case>
<test-case FilePath="async-deferred">
<compilation-unit name="async-running">
+ <parameter name="profile" value="timings" type="string"/>
<output-dir compare="Clean-JSON">async-running</output-dir>
</compilation-unit>
</test-case>
<test-case FilePath="async-deferred">
<compilation-unit name="async-exhausted-result">
<output-dir compare="Clean-JSON">async-exhausted-result</output-dir>
+ <parameter name="profile" value="timings" type="string"/>
<expected-error>Premature end of chunk</expected-error> <!--TODO:REVISIT -->
<source-location>false</source-location>
</compilation-unit>
</test-case>
<test-case FilePath="async-deferred">
<compilation-unit name="async-json">
+ <parameter name="profile" value="timings" type="string"/>
<output-dir compare="Clean-JSON">async-json</output-dir>
</compilation-unit>
</test-case>
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..3190b2d
--- /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,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.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..759fc3f
--- /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,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.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..7d6bf92
--- /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,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.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..a593df3
--- /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 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.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..151309a
--- /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,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.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..759fc3f
--- /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 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..9c01681
--- /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;
+
+FROM `Metadata`.`Index` t WHERE t.IndexStructure = "SAMPLE"
+SELECT t.* EXCLUDE DataverseName, SearchKey, IsPrimary, Timestamp, PendingOp, SampleSeed
+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..9c01681
--- /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;
+
+FROM `Metadata`.`Index` t WHERE t.IndexStructure = "SAMPLE"
+SELECT t.* EXCLUDE DataverseName, SearchKey, IsPrimary, Timestamp, PendingOp, SampleSeed
+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.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/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/external-dataset/common/parquet/invalid-parquet-files/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/invalid-parquet-files/test.000.ddl.sqlpp
new file mode 100644
index 0000000..ca5868c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/invalid-parquet-files/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;
+
+CREATE TYPE ParquetType as {
+};
+
+CREATE EXTERNAL DATASET ParquetDataset(ParquetType) USING %adapter%
+(
+ %template%,
+ ("container"="playground"),
+ ("definition"="json-data/reviews/single-line/json"),
+ ("format" = "parquet")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/invalid-parquet-files/test.001.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/invalid-parquet-files/test.001.query.sqlpp
new file mode 100644
index 0000000..a178663
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/invalid-parquet-files/test.001.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 VALUE COUNT(*)
+FROM ParquetDataset p
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/invalid-parquet-files/test.999.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/invalid-parquet-files/test.999.ddl.sqlpp
new file mode 100644
index 0000000..20dc6fd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/invalid-parquet-files/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/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/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..10378f2 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,6 @@
* 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":30, "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.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/metadata_only_02/metadata_only_02.1.ddl.sqlpp
new file mode 100644
index 0000000..ab247a3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/metadata_only_02/metadata_only_02.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.
+ */
+
+/*
+ * The query tests fix for ASTERIXDB-3038
+ */
+
+drop dataverse test1 if exists;
+drop dataverse test2 if exists;
+
+create dataverse test1;
+create dataverse test2;
+
+create dataset test1.ds1 (id1 integer not unknown) primary key id1;
+create dataset test2.ds2 (id2 integer not unknown) primary key id2;
+
+create synonym test1.syn1 for test1.ds1;
+create synonym test2.syn2 for test2.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.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/metadata_only_02/metadata_only_02.2.query.sqlpp
new file mode 100644
index 0000000..a7e1b93
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/metadata_only_02/metadata_only_02.2.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.
+ */
+
+/*
+ * The query tests fix for ASTERIXDB-3038
+ */
+
+select syn.DataverseName, syn.SynonymName
+from Metadata.`Synonym` as syn
+where syn.ObjectDataverseName in
+ ["test1", "test2"]
+and syn.ObjectName in (
+ select value ds.DatasetName
+ from Metadata.`Dataset` as ds
+ where ds.DataverseName in
+ ["test1", "test2"]
+ and ds.DatasetName in
+ ["ds1", "ds2"])
+ or syn.ObjectName in (
+ select value syn1.SynonymName
+ from Metadata.`Synonym` as syn1
+ where syn1.SynonymName in
+ ["syn1", "syn2"]
+ and syn1.ObjectName in (
+ select value ds1.DatasetName
+ from Metadata.`Dataset` as ds1
+ where ds1.DataverseName in
+ ["test1", "test2"]
+ and ds1.DatasetName in
+ ["ds1", "ds2"]
+ )
+ )
+order by syn.DataverseName, syn.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.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/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 bf48a7b..77fc7c5 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
@@ -120,9 +120,14 @@
<expected-error>ASX1001: Syntax error: Cannot infer field name</expected-error>
</compilation-unit>
</test-case>
- <test-case FilePath="objects">
+ <test-case FilePath="objects" check-warnings="true">
<compilation-unit name="object_concat">
<output-dir compare="Text">object_concat</output-dir>
+ <expected-warn>ASX0013: Duplicate field name 'v'</expected-warn>
+ <expected-warn>ASX0013: Duplicate field name 'f'</expected-warn>
+ <expected-warn>ASX0013: Duplicate field name 'id'</expected-warn>
+ <expected-warn>ASX0013: Duplicate field name 'id'</expected-warn>
+ <expected-warn>ASX0013: Duplicate field name 'dup'</expected-warn>
</compilation-unit>
</test-case>
<test-case FilePath="objects">
@@ -203,6 +208,11 @@
</compilation-unit>
</test-case>
<test-case FilePath="objects" check-warnings="true">
+ <compilation-unit name="open-object-constructor-with-missing-field-name">
+ <output-dir compare="Text">open-object-constructor-with-missing-field-name</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="objects" check-warnings="true">
<compilation-unit name="open-closed-fieldname-conflict_issue173">
<output-dir compare="Text">open-closed-fieldname-conflict_issue173</output-dir>
<source-location>false</source-location>
@@ -238,4 +248,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/objects/object_add/object_add.5.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_add/object_add.5.ddl.sqlpp
new file mode 100644
index 0000000..2eefbe8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_add/object_add.5.ddl.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.
+ */
+
+DROP DATAVERSE TestDataverse IF EXISTS;
+CREATE DATAVERSE TestDataverse;
+USE TestDataverse;
+
+CREATE TYPE UsersBaseTypeClosed AS CLOSED { _id: int };
+CREATE TYPE UsersBaseTypeOpen AS { _id: int };
+CREATE TYPE UsersFriendsTypeClosed AS CLOSED {
+ _id: int,
+ best_friend: UsersBaseTypeClosed,
+ friends: [UsersBaseTypeClosed]
+};
+CREATE TYPE UsersFriendsTypeOpen AS { _id: int };
+
+CREATE DATASET UsersClosed (UsersFriendsTypeClosed) PRIMARY KEY _id;
+CREATE DATASET UsersOpen (UsersFriendsTypeOpen) PRIMARY KEY _id;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_add/object_add.6.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_add/object_add.6.update.sqlpp
new file mode 100644
index 0000000..bda29eb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_add/object_add.6.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 TestDataverse;
+
+INSERT INTO UsersClosed [
+ { "_id": 1, "best_friend": { "_id": 1 }, "friends": [] },
+ { "_id": 2, "best_friend": { "_id": 1 }, "friends": [{ "_id": 1 }] },
+ { "_id": 3, "best_friend": { "_id": 2 }, "friends": [{ "_id": 1 }, { "_id": 2 }] }
+];
+INSERT INTO UsersOpen [
+ { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [] },
+ { "_id": 5, "favorite_color": "Blue", "best_friend": { "_id": 4 }, "friends": [{ "_id": 4 }] },
+ { "_id": 6, "favorite_color": "Orange", "best_friend": { "_id": 5 }, "friends": [{ "_id": 4 }, { "_id": 5 }] }
+];
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_add/object_add.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_add/object_add.7.query.sqlpp
new file mode 100644
index 0000000..71f6974
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_add/object_add.7.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.
+ */
+
+// For OBJECT_ADD.
+USE TestDataverse;
+LET openUValues = (
+ FROM UsersOpen U2
+ WHERE U2._id = 4
+ SELECT VALUE U2
+ ),
+ closedUValues = (
+ FROM UsersClosed U1
+ WHERE U1._id = 1
+ SELECT VALUE U1
+ )
+SELECT VALUE {
+ // New constant field into closed and open record.
+ "t1c": ( FROM UsersClosed U SELECT VALUE OBJECT_ADD(U, "name", "John") ORDER BY U._id ),
+ "t1o": ( FROM UsersOpen U SELECT VALUE OBJECT_ADD(U, "name", "John") ORDER BY U._id ),
+
+ // New record field from open record into closed and open record.
+ "t2c": ( FROM UsersClosed U SELECT VALUE OBJECT_ADD(U, "newFriend", openUValues[0]) ORDER BY U._id ) ,
+ "t2o": ( FROM UsersOpen U SELECT VALUE OBJECT_ADD(U, "newFriend", openUValues[0]) ORDER BY U._id ) ,
+
+ // New record field from closed record into closed and open record.
+ "t3c": ( FROM UsersClosed U SELECT VALUE OBJECT_ADD(U, "newFriend", closedUValues[0]) ORDER BY U._id ),
+ "t3o": ( FROM UsersOpen U SELECT VALUE OBJECT_ADD(U, "newFriend", closedUValues[0]) ORDER BY U._id ),
+
+ // New list field from open dataset into closed and open record.
+ "t4c": ( FROM UsersClosed U SELECT VALUE OBJECT_ADD(U, "newFriends", openUValues) ORDER BY U._id ),
+ "t4o": ( FROM UsersOpen U SELECT VALUE OBJECT_ADD(U, "newFriends", openUValues) ORDER BY U._id ),
+
+ // New list field from closed dataset into closed and open record.
+ "t5c": ( FROM UsersClosed U SELECT VALUE OBJECT_ADD(U, "newFriends", closedUValues) ORDER BY U._id ),
+ "t5o": ( FROM UsersOpen U SELECT VALUE OBJECT_ADD(U, "newFriends", closedUValues) ORDER BY U._id )
+};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_concat/object_concat.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_concat/object_concat.4.query.sqlpp
new file mode 100644
index 0000000..ba22920
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_concat/object_concat.4.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.
+ */
+
+// test that object_concat() issues a warning when encountering a duplicate field
+
+// requesttype=application/json
+// param max-warnings:json=10
+WITH t AS (
+SELECT v AS v
+FROM [{"id": 1, "f": 3}, {"id": 2, "f": 3}] AS v
+)
+SELECT OBJECT_CONCAT(t);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_concat/object_concat.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_concat/object_concat.5.query.sqlpp
new file mode 100644
index 0000000..1b61e66
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_concat/object_concat.5.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.
+ */
+
+// test that object_concat() issues a warning when encountering a duplicate field
+
+// requesttype=application/json
+// param max-warnings:json=10
+
+SET `import-private-functions` `true`;
+WITH t AS ([{"id": 1, "f": 3}, {"id": 2, "f": 4}])
+SELECT OBJECT_CONCAT_STRICT(t) AS res;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_concat/object_concat.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_concat/object_concat.6.query.sqlpp
new file mode 100644
index 0000000..8ef2ebbb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_concat/object_concat.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.
+ */
+
+// requesttype=application/json
+// param max-warnings:json=10
+
+WITH a AS (SELECT VALUE x FROM [{"id": 1, "a1": 3, "dup": 0}] AS x),
+b AS (SELECT VALUE y FROM [{"id": 1, "b1": 3, "dup": 5}] AS y)
+FROM a, b
+WHERE a.id = b.id
+SELECT a.*, b.*
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_put/object_put.5.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_put/object_put.5.ddl.sqlpp
new file mode 100644
index 0000000..2eefbe8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_put/object_put.5.ddl.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.
+ */
+
+DROP DATAVERSE TestDataverse IF EXISTS;
+CREATE DATAVERSE TestDataverse;
+USE TestDataverse;
+
+CREATE TYPE UsersBaseTypeClosed AS CLOSED { _id: int };
+CREATE TYPE UsersBaseTypeOpen AS { _id: int };
+CREATE TYPE UsersFriendsTypeClosed AS CLOSED {
+ _id: int,
+ best_friend: UsersBaseTypeClosed,
+ friends: [UsersBaseTypeClosed]
+};
+CREATE TYPE UsersFriendsTypeOpen AS { _id: int };
+
+CREATE DATASET UsersClosed (UsersFriendsTypeClosed) PRIMARY KEY _id;
+CREATE DATASET UsersOpen (UsersFriendsTypeOpen) PRIMARY KEY _id;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_put/object_put.6.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_put/object_put.6.update.sqlpp
new file mode 100644
index 0000000..bda29eb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_put/object_put.6.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 TestDataverse;
+
+INSERT INTO UsersClosed [
+ { "_id": 1, "best_friend": { "_id": 1 }, "friends": [] },
+ { "_id": 2, "best_friend": { "_id": 1 }, "friends": [{ "_id": 1 }] },
+ { "_id": 3, "best_friend": { "_id": 2 }, "friends": [{ "_id": 1 }, { "_id": 2 }] }
+];
+INSERT INTO UsersOpen [
+ { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [] },
+ { "_id": 5, "favorite_color": "Blue", "best_friend": { "_id": 4 }, "friends": [{ "_id": 4 }] },
+ { "_id": 6, "favorite_color": "Orange", "best_friend": { "_id": 5 }, "friends": [{ "_id": 4 }, { "_id": 5 }] }
+];
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_put/object_put.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_put/object_put.7.query.sqlpp
new file mode 100644
index 0000000..52943a0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_put/object_put.7.query.sqlpp
@@ -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.
+ */
+
+// For OBJECT_PUT.
+USE TestDataverse;
+LET openUValues = (
+ FROM UsersOpen U2
+ WHERE U2._id = 4
+ SELECT VALUE U2
+ ),
+ closedUValues = (
+ FROM UsersClosed U1
+ WHERE U1._id = 1
+ SELECT VALUE U1
+ )
+SELECT VALUE {
+ // New constant field into closed and open record.
+ "t1c": ( FROM UsersClosed U SELECT VALUE OBJECT_PUT(U, "name", "John") ORDER BY U._id ),
+ "t1o": ( FROM UsersOpen U SELECT VALUE OBJECT_PUT(U, "name", "John") ORDER BY U._id ),
+
+ // New record field from open record into closed and open record.
+ "t2c": ( FROM UsersClosed U SELECT VALUE OBJECT_PUT(U, "newFriend", openUValues[0]) ORDER BY U._id ) ,
+ "t2o": ( FROM UsersOpen U SELECT VALUE OBJECT_PUT(U, "newFriend", openUValues[0]) ORDER BY U._id ) ,
+
+ // New record field from closed record into closed and open record.
+ "t3c": ( FROM UsersClosed U SELECT VALUE OBJECT_PUT(U, "newFriend", closedUValues[0]) ORDER BY U._id ),
+ "t3o": ( FROM UsersOpen U SELECT VALUE OBJECT_PUT(U, "newFriend", closedUValues[0]) ORDER BY U._id ),
+
+ // New list field from open dataset into closed and open record.
+ "t4c": ( FROM UsersClosed U SELECT VALUE OBJECT_PUT(U, "newFriends", openUValues) ORDER BY U._id ),
+ "t4o": ( FROM UsersOpen U SELECT VALUE OBJECT_PUT(U, "newFriends", openUValues) ORDER BY U._id ),
+
+ // New list field from closed dataset into closed and open record.
+ "t5c": ( FROM UsersClosed U SELECT VALUE OBJECT_PUT(U, "newFriends", closedUValues) ORDER BY U._id ),
+ "t5o": ( FROM UsersOpen U SELECT VALUE OBJECT_PUT(U, "newFriends", closedUValues) ORDER BY U._id ),
+
+ // Old field of open type into closed record (we should overwrite).
+ "t6c": ( FROM UsersClosed U
+ LET newFriends = [
+ { "_id": 8 }, { "_id": 9 }
+ ]
+ SELECT VALUE OBJECT_PUT(U, "friends", newFriends)
+ ORDER BY U._id )
+};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/open-object-constructor-with-missing-field-name/open-object-constructor-with-missing-field-name.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/open-object-constructor-with-missing-field-name/open-object-constructor-with-missing-field-name.1.query.sqlpp
new file mode 100644
index 0000000..8a892ef
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/open-object-constructor-with-missing-field-name/open-object-constructor-with-missing-field-name.1.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.
+ */
+
+FROM [{"fieldName": "status", "fieldValue": "shipped"}, {"fieldValue": "pending"}] AS g
+SELECT VALUE { g.fieldName : g.fieldValue};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/open-object-constructor-with-missing-field-name/open-object-constructor-with-missing-field-name.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/open-object-constructor-with-missing-field-name/open-object-constructor-with-missing-field-name.2.query.sqlpp
new file mode 100644
index 0000000..e132460
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/open-object-constructor-with-missing-field-name/open-object-constructor-with-missing-field-name.2.query.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.
+ */
+
+WITH src AS [
+ { "category_name": "Video Card", "status": "Shipped", "order_id": 1, "order_value": 10 },
+ { "category_name": "Video Card", "status": "Shipped", "order_id": 2, "order_value": 20 },
+ { "category_name": "Video Card", "status": "Cancelled", "order_id": 3, "order_value": 30 },
+ { "category_name": "Video Card", "status": "Cancelled", "order_id": 4, "order_value": 40 },
+ { "category_name": "Video Card", "status": "Pending", "order_id": 5, "order_value": 50 },
+ { "category_name": "Video Card", "status": "Pending", "order_id": 6, "order_value": 50 },
+ { "category_name": "Storage", "status": "Shipped", "order_id": 11, "order_value": 110 },
+ { "category_name": "Storage", "status": "Shipped", "order_id": 12, "order_value": 120 },
+ { "category_name": "Storage", "status": "Cancelled", "order_id": 13, "order_value": 130 },
+ { "category_name": "Storage", "status": "Cancelled", "order_id": 14, "order_value": 140 },
+ { "category_name": "Storage", "status": "Pending", "order_id": 15, "order_value": 150 },
+ { "category_name": "Storage", "status": "Pending", "order_id": 16, "order_value": 160 }
+],
+t1 AS (
+ select category_name, status, count(order_id) orders, sum(order_value) sales
+ from src
+ group by category_name, status
+)
+SELECT status, v.*
+FROM t1 group BY status GROUP AS g
+let v = OBJECT_CONCAT((
+ SELECT VALUE { g.t1.category_name || " Orders": g.t1.orders, g.t1.UNKNOWN_FIELD || " Sales": g.t1.sales } FROM g
+))
+ORDER BY status;
\ No newline at end of file
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/select-exclude/complex-exclude/complex-exclude.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.1.ddl.sqlpp
new file mode 100644
index 0000000..8fc7172
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.1.ddl.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE ComplexExclude IF EXISTS;
+CREATE DATAVERSE ComplexExclude;
+USE ComplexExclude;
+
+CREATE TYPE UsersType AS { user_id: bigint };
+CREATE DATASET Users (UsersType) PRIMARY KEY user_id;
+
+CREATE VIEW UsersWithoutPII AS
+ FROM ComplexExclude.Users U
+ SELECT U.* EXCLUDE phones, address, ssn;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.10.query.sqlpp
new file mode 100644
index 0000000..67d91fb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.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.
+ */
+
+// Query with RIGHT JOIN and EXCLUDE.
+FROM ComplexExclude.Users U2
+RIGHT JOIN ComplexExclude.Users U1
+ON U1.best_friend = U2.user_id
+SELECT * EXCLUDE U1.address, U2.address
+ORDER BY U1.user_id, U2.user_id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.11.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.11.query.sqlpp
new file mode 100644
index 0000000..099c6c5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.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.
+ */
+
+// Query with DISTINCT (applies after EXCLUDE) and normal WHERE.
+FROM ComplexExclude.Users U
+WHERE U.best_friend = 1
+SELECT DISTINCT U.* EXCLUDE address, favorite_color, user_id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.12.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.12.query.sqlpp
new file mode 100644
index 0000000..6b5ac14
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.12.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.
+ */
+
+// Query using view with EXCLUDE + EXCLUDE on top.
+FROM ComplexExclude.UsersWithoutPII U
+SELECT U.* EXCLUDE favorite_color
+ORDER BY U.user_id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.13.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.13.query.sqlpp
new file mode 100644
index 0000000..46bac5d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.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.
+ */
+
+// Query using EXCLUDE as a field-name.
+// (We disallow EXCLUDE as a field-name if not preceded with an 'AS').
+WITH A AS [ { "a": 1, "b": 2, "EXCLUDE": 3 } ]
+FROM A
+SELECT A AS EXCLUDE
+UNION ALL
+FROM A
+SELECT A.EXCLUDE
+UNION ALL
+FROM A
+SELECT A EXCLUDE A.a
+ORDER BY EXCLUDE;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.14.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.14.query.sqlpp
new file mode 100644
index 0000000..20975a4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.14.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.
+ */
+
+// Query using all clauses + EXCLUDE.
+WITH otherUsers AS ( FROM ComplexExclude.Users U SELECT VALUE U )
+FROM ComplexExclude.Users U1
+JOIN otherUsers U2
+ON U2.user_id = U1.best_friend
+LEFT UNNEST U1.phones U1P
+LET bestFriend = U2
+WHERE U1.user_id = U2.best_friend
+GROUP BY U1
+GROUP AS G
+LET bestFriends = ( FROM G SELECT VALUE bestFriend )
+HAVING COUNT(*) > 0
+SELECT bestFriends, U1.* EXCLUDE address, phones
+ORDER BY U1.user_id
+LIMIT 10;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.2.update.sqlpp
new file mode 100644
index 0000000..4c770f6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.2.update.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 ComplexExclude;
+
+INSERT INTO Users [
+ { "user_id": 1, "best_friend": 2, "phones": [ { "kind": "MOBILE", "number": "222-222-2222" } ] },
+ { "user_id": 2, "best_friend": 1, "address": { "zip_code": "99929", "street": "2341 Apple Street" } },
+ { "user_id": 3, "best_friend": 1, "address": { "zip_code": "99929", "street": "2341 Apple Street" }, "favorite_color": "Green" },
+ { "user_id": 4, "best_friend": null }
+];
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.3.query.sqlpp
new file mode 100644
index 0000000..bc01577
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.3.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.
+ */
+
+// Query using view with EXCLUDE.
+FROM ComplexExclude.UsersWithoutPII U
+SELECT U.*
+ORDER BY U.user_id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.4.query.sqlpp
new file mode 100644
index 0000000..6aa8a33
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.4.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.
+ */
+
+// Query with extraneous EXCLUDE terms ("address.zip_code" is unnecessary).
+FROM ComplexExclude.Users U
+SELECT U.* EXCLUDE address, address.zip_code
+ORDER BY U.user_id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.5.query.sqlpp
new file mode 100644
index 0000000..8ef1df6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.5.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.
+ */
+
+// Query with EXCLUDE in subquery.
+FROM ComplexExclude.Users U1
+WHERE U1.best_friend = 1
+SELECT VALUE ( FROM ComplexExclude.Users U2
+ WHERE U2.user_id = U1.user_id
+ SELECT U2.* EXCLUDE address )
+ORDER BY U1.user_id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.6.query.sqlpp
new file mode 100644
index 0000000..8e0b660
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.6.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.
+ */
+
+// Query with EXCLUDE on the projections w/o star.
+FROM ComplexExclude.Users U
+SELECT U.user_id, U.best_friend, U.address EXCLUDE best_friend, address
+ORDER BY U.user_id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.7.query.sqlpp
new file mode 100644
index 0000000..ba6588e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.7.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.
+ */
+
+// Query with GROUP-BY and EXCLUDE.
+FROM ComplexExclude.Users U
+GROUP BY U.best_friend
+GROUP AS G
+SELECT * EXCLUDE G
+ORDER BY U.best_friend NULLS LAST;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.8.query.sqlpp
new file mode 100644
index 0000000..437c7af
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.8.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.
+ */
+
+// Query with CTE, EXCLUDE, and explicit JOIN.
+LET SpecialUsers = (
+ FROM ComplexExclude.Users U
+ WHERE U.user_id = 1
+ SELECT VALUE U
+ )
+FROM ComplexExclude.Users U
+INNER JOIN SpecialUsers S
+ON U.user_id = S.user_id
+SELECT U.* EXCLUDE phones;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.9.query.sqlpp
new file mode 100644
index 0000000..ff507a3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.9.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.
+ */
+
+// Query with LET after the FROM clause.
+FROM ComplexExclude.Users U
+LET miscInfo = { "24as": "23412", "address": "2341 Orange Street" }
+SELECT * EXCLUDE U.address, miscInfo.address, U.phones
+ORDER BY U.user_id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/exclude-negative/exclude-negative.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/exclude-negative/exclude-negative.1.ddl.sqlpp
new file mode 100644
index 0000000..7c54e82
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/exclude-negative/exclude-negative.1.ddl.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.
+ */
+
+DROP DATAVERSE ExcludeNegative IF EXISTS;
+CREATE DATAVERSE ExcludeNegative;
+USE ExcludeNegative;
+
+CREATE TYPE UsersType AS { user_id: bigint };
+CREATE DATASET Users (UsersType) PRIMARY KEY user_id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/exclude-negative/exclude-negative.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/exclude-negative/exclude-negative.2.query.sqlpp
new file mode 100644
index 0000000..8e8c021
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/exclude-negative/exclude-negative.2.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.
+ */
+
+// EXCLUDE cannot be used as an alias without 'AS'.
+FROM NegativeExclude.Users U
+SELECT U.user_id EXCLUDE;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/exclude-negative/exclude-negative.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/exclude-negative/exclude-negative.3.query.sqlpp
new file mode 100644
index 0000000..eee1023
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/exclude-negative/exclude-negative.3.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.
+ */
+
+// EXCLUDE cannot be used with SELECT VALUE.
+FROM NegativeExclude.Users U
+SELECT VALUE U EXCLUDE user_id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/tiny-social/tiny-social.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/tiny-social/tiny-social.1.ddl.sqlpp
new file mode 100644
index 0000000..3cb07d0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/tiny-social/tiny-social.1.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.
+ */
+
+DROP DATAVERSE TinySocial IF EXISTS;
+CREATE DATAVERSE TinySocial;
+USE TinySocial;
+
+CREATE TYPE TwitterUserType AS {
+ `screen-name` : string,
+ lang : string,
+ friends_count : bigint,
+ statuses_count : bigint
+};
+
+CREATE TYPE TweetMessageType AS {
+ tweetid : string,
+ user : TwitterUserType,
+ `sender-location` : point?,
+ `send-time` : datetime,
+ `referred-topics` : {{string}},
+ `message-text` : string
+};
+
+CREATE DATASET TwitterUsers(TwitterUserType) PRIMARY KEY `screen-name`;
+CREATE DATASET TweetMessages(TweetMessageType) PRIMARY KEY tweetid;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/tiny-social/tiny-social.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/tiny-social/tiny-social.2.update.sqlpp
new file mode 100644
index 0000000..a60ef4d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/tiny-social/tiny-social.2.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 TinySocial;
+LOAD DATASET TwitterUsers USING localfs ((`path`=`asterix_nc1://data/tinysocial/twu.adm`),(`format`=`adm`));
+LOAD DATASET TweetMessages USING localfs ((`path`=`asterix_nc1://data/tinysocial/twm.adm`),(`format`=`adm`));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/tiny-social/tiny-social.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/tiny-social/tiny-social.3.query.sqlpp
new file mode 100644
index 0000000..4d7489a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/tiny-social/tiny-social.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.
+ */
+
+FROM TinySocial.TweetMessages TM
+
+ // SELECT-var.*.
+SELECT TM.*
+
+ // Nested field that does not exist.
+EXCLUDE user.does_not_exist,
+ // Nested field that does exist.
+ user.lang,
+ // Field with quotes.
+ `send-time`,
+ // Non-nested field that does not exist.
+ does_not_exist,
+ // Non-nested field that exists.
+ tweetid
+
+ORDER BY TM.tweetid;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/tiny-social/tiny-social.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/tiny-social/tiny-social.4.query.sqlpp
new file mode 100644
index 0000000..06c6df0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/tiny-social/tiny-social.4.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.
+ */
+
+FROM TinySocial.TweetMessages TM,
+ TinySocial.TwitterUsers TU
+
+WHERE TM.user.`screen-name` = TU.`screen-name`
+
+ // SELECT * returns { "TM": ..., "TU": ... }
+SELECT *
+
+ // Identifier that exists in scope.
+EXCLUDE TU,
+ // Nested identifier that exists in scope.
+ TM.user.`screen-name`,
+ // Identifier that does not exist in scope.
+ TI
+
+ORDER BY TM.tweetid;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/tiny-social/tiny-social.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/tiny-social/tiny-social.5.query.sqlpp
new file mode 100644
index 0000000..d41b3a7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/tiny-social/tiny-social.5.query.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.
+ */
+
+FROM TinySocial.TweetMessages TM1
+SELECT TM1.*
+EXCLUDE user.does_not_exist,
+ user.lang,
+ `send-time`,
+ does_not_exist
+
+UNION ALL
+
+FROM TinySocial.TweetMessages TM2
+SELECT TM2.*
+EXCLUDE user.lang
+
+ORDER BY tweetid, `send-time` NULLS FIRST;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/tiny-social/tiny-social.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/tiny-social/tiny-social.6.query.sqlpp
new file mode 100644
index 0000000..e5453ef
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/tiny-social/tiny-social.6.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.
+ */
+
+FROM TinySocial.TweetMessages TM
+
+ // SELECT-* (single-variable rule case).
+SELECT *
+
+ // Nested field that exists, anchored by TM.
+EXCLUDE TM.user.lang,
+ // Field with quotes (single-variable rule applies).
+ `send-time`,
+ // Non-nested field that exists (single-variable rule applies).
+ tweetid
+
+ORDER BY tweetid;
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/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/union/union_type_cast/union_type_cast.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/union/union_type_cast/union_type_cast.1.ddl.sqlpp
new file mode 100644
index 0000000..2a616ea
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/union/union_type_cast/union_type_cast.1.ddl.sqlpp
@@ -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.
+ */
+
+DROP DATAVERSE TinySocial IF EXISTS;
+CREATE DATAVERSE TinySocial;
+
+USE TinySocial;
+
+CREATE TYPE EmploymentType AS CLOSED
+{
+ `organization-name` : string,
+ `start-date` : date,
+ `end-date` : date?
+};
+
+CREATE TYPE FacebookUserType AS CLOSED {
+ id : bigint,
+ alias : string,
+ name : string,
+ `user-since` : datetime,
+ `friend-ids` : {{bigint}},
+ employment : [EmploymentType]
+};
+
+CREATE TYPE FacebookMessageType AS CLOSED {
+ `message-id` : bigint,
+ `author-id` : bigint,
+ `in-response-to` : bigint?,
+ `sender-location` : point?,
+ message : string
+};
+
+CREATE DATASET FacebookUsers(FacebookUserType) PRIMARY KEY id;
+
+CREATE DATASET FacebookMessages(FacebookMessageType) PRIMARY KEY `message-id`;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/union/union_type_cast/union_type_cast.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/union/union_type_cast/union_type_cast.2.update.sqlpp
new file mode 100644
index 0000000..3d785ad
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/union/union_type_cast/union_type_cast.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 TinySocial;
+
+LOAD DATASET FacebookUsers using localfs (("path"="asterix_nc1://data/tinysocial/fbu.adm"),("format"="adm"));
+
+LOAD DATASET FacebookMessages using localfs (("path"="asterix_nc1://data/tinysocial/fbm.adm"),("format"="adm"));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/union/union_type_cast/union_type_cast.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/union/union_type_cast/union_type_cast.3.query.sqlpp
new file mode 100644
index 0000000..003ec26
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/union/union_type_cast/union_type_cast.3.query.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.
+ */
+
+USE TinySocial;
+-- Disabled for a simpler plan
+SET `compiler.sort.parallel` "false";
+
+
+ SELECT "message1" ds, s.`message-id` id, object_remove(s, "in-response-to") no_in_response_to
+ FROM FacebookMessages AS s
+UNION ALL
+ SELECT "user" ds, t.id id, t user
+ FROM FacebookUsers t
+UNION ALL
+ SELECT "message2" ds, s.`message-id` id, object_remove(s, "author-id") no_author_id
+ FROM FacebookMessages s
+ORDER BY id, ds;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/union/union_type_cast/union_type_cast.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/union/union_type_cast/union_type_cast.4.query.sqlpp
new file mode 100644
index 0000000..cc63b57
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/union/union_type_cast/union_type_cast.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.
+ */
+
+USE TinySocial;
+-- Disabled for a simpler plan
+SET `compiler.sort.parallel` "false";
+
+-- To prevent the plan from changing
+SET `compiler.parallelism` "0";
+
+EXPLAIN
+ SELECT s.`message-id` id, object_remove(s, "in-response-to") no_in_response_to
+ FROM FacebookMessages AS s
+UNION ALL
+ SELECT t.id id, t user
+ FROM FacebookUsers t
+UNION ALL
+ SELECT s.`message-id` id, object_remove(s, "author-id") no_author_id
+ FROM FacebookMessages s
+ORDER BY id;
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/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..14ff86d
--- /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,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.
+ */
+/*
+* 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
+};
+
+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/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/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..389b120 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,10 @@
"active\.suspend\.timeout" : 3600,
"azure.request.timeout" : 120,
"compiler\.arrayindex" : true,
+ "compiler.batch.lookup" : false,
+ "compiler.cbo" : false,
"compiler\.external\.field\.pushdown" : true,
+ "compiler.forcejoinorder" : false,
"compiler\.framesize" : 32768,
"compiler\.groupmemory" : 163840,
"compiler\.indexonly" : true,
@@ -18,6 +21,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,
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..e68df5f 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,10 @@
"active\.suspend\.timeout" : 3600,
"azure.request.timeout" : 120,
"compiler\.arrayindex" : true,
+ "compiler.batch.lookup" : false,
+ "compiler.cbo" : false,
"compiler\.external\.field\.pushdown" : true,
+ "compiler.forcejoinorder" : false,
"compiler\.framesize" : 32768,
"compiler\.groupmemory" : 163840,
"compiler\.indexonly" : true,
@@ -18,6 +21,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,
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..a6fa211 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,10 @@
"active\.suspend\.timeout" : 3600,
"azure.request.timeout" : 120,
"compiler\.arrayindex" : true,
+ "compiler.batch.lookup" : false,
+ "compiler.cbo" : false,
"compiler\.external\.field\.pushdown" : true,
+ "compiler.forcejoinorder" : false,
"compiler\.framesize" : 32768,
"compiler\.groupmemory" : 163840,
"compiler\.indexonly" : true,
@@ -18,6 +21,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,
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/api/request-param-validation-400-BAD/request-param-validation-400-BAD.01.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/request-param-validation-400-BAD/request-param-validation-400-BAD.01.regexjson
index ad95b7b..6fa99e2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/request-param-validation-400-BAD/request-param-validation-400-BAD.01.regexjson
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/request-param-validation-400-BAD/request-param-validation-400-BAD.01.regexjson
@@ -6,6 +6,7 @@
"metrics": {
"elapsedTime": "R{.*}",
"executionTime": "R{.*}",
+ "compileTime": "R{.*}",
"resultCount": 0,
"resultSize": 0,
"processedObjects": 0,
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_binary_search/array_binary_search.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_binary_search/array_binary_search.3.adm
new file mode 100644
index 0000000..ae6f41f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_binary_search/array_binary_search.3.adm
@@ -0,0 +1,23 @@
+{ "id": 1, "t1": [ { "$1": null } ] }
+{ "id": 2, "t2": [ { } ] }
+{ "id": 3, "t3": [ { "$3": -1 } ] }
+{ "id": 4, "t4": [ { "$4": 3 } ] }
+{ "id": 5, "t5": [ { "$5": 0 } ] }
+{ "id": 6, "t6": [ { "$6": 6 } ] }
+{ "id": 7, "t7": [ { "$7": 2 } ] }
+{ "id": 8, "t8": [ { "$8": 0 } ] }
+{ "id": 9, "t9": [ { "$9": 6 } ] }
+{ "id": 10, "t10": [ { "$10": 2 } ] }
+{ "id": 11, "t11": [ { "$11": 7 } ] }
+{ "id": 12, "t12": [ { "$12": -1 } ] }
+{ "id": 13, "t13": [ { "$13": null } ] }
+{ "id": 14, "t14": [ { "$14": 0 } ] }
+{ "id": 15, "t15": [ { "$15": 3 } ] }
+{ "id": 16, "t16": [ { "$16": 1 }, { "$16": -1 }, { "$16": -1 }, { "$16": -1 }, { "$16": -1 }, { "$16": null }, { } ] }
+{ "id": 17, "t17": [ { "$17": -1 }, { "$17": -1 }, { "$17": -1 }, { "$17": 0 }, { "$17": -1 }, { "$17": null }, { } ] }
+{ "id": 18, "t18": [ { "$18": 0 }, { "$18": -1 } ] }
+{ "id": 19, "t19": [ { "$19": 1 } ] }
+{ "id": 20, "t20": [ { "$20": 6 } ] }
+{ "id": 21, "t21": [ { "$21": 8 } ] }
+{ "id": 22, "t22": [ { "$22": null } ] }
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_move/array_move.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_move/array_move.3.adm
new file mode 100644
index 0000000..f8057c6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_move/array_move.3.adm
@@ -0,0 +1,26 @@
+{ "id": 1, "t1": [ { "$1": [ 0, 2, 3, 1, 4, 5 ] } ] }
+{ "id": 2, "t2": [ { "$2": [ 0, 1, 2, 5, 3, 4 ] } ] }
+{ "id": 3, "t3": [ { "$3": [ "b", "c", "a", "d", "e" ] } ] }
+{ "id": 4, "t4": [ { "$4": [ "a", "d", "b", "c", "e" ] } ] }
+{ "id": 5, "t5": [ { "$5": null } ] }
+{ "id": 6, "t6": [ { "$6": null } ] }
+{ "id": 7, "t7": [ { "$7": null } ] }
+{ "id": 8, "t8": [ { "$8": null } ] }
+{ "id": 9, "t9": [ { "$9": [ [ 9999, 3 ], [ 1, 2, 1 ] ] }, { "$9": [ [ "red", "black" ], [ "white", "blue", "magenta" ] ] }, { "$9": [ 2, 1 ] }, { "$9": [ { "state": "CA", "country": "US", "zip_code": 92863 }, { "state": "OH", "country": "US" } ] }, { "$9": [ { "state": "IL", "country": "US", "zip_code": 92863 }, { "state": "OR", "country": "US", "zip_code": null } ] }, { "$9": null }, { } ] }
+{ "id": 10, "t10": [ { "$10": [ [ 9999, 3 ], [ 1, 2, 1 ] ] }, { "$10": [ [ "red", "black" ], [ "white", "blue", "magenta" ] ] }, { "$10": [ 2, 1 ] }, { "$10": [ { "state": "CA", "country": "US", "zip_code": 92863 }, { "state": "OH", "country": "US" } ] }, { "$10": [ { "state": "IL", "country": "US", "zip_code": 92863 }, { "state": "OR", "country": "US", "zip_code": null } ] }, { "$10": null }, { } ] }
+{ "id": 11, "t11": [ { "$11": null }, { "$11": null }, { "$11": null }, { "$11": null }, { "$11": null }, { "$11": null }, { } ] }
+{ "id": 12, "t12": [ { }, { }, { }, { }, { }, { }, { } ] }
+{ "id": 13, "t13": [ { "$13": [ { "state": "CA", "country": "US", "zip_code": 92863 }, { "state": "OH", "country": "US" } ] }, { "$13": [ { "state": "IL", "country": "US", "zip_code": 92863 }, { "state": "OR", "country": "US", "zip_code": null } ] } ] }
+{ "id": 14, "t14": [ { "$14": [ { "state": "CA", "country": "US", "zip_code": 92863 }, { "state": "OH", "country": "US" } ] }, { "$14": [ { "state": "IL", "country": "US", "zip_code": 92863 }, { "state": "OR", "country": "US", "zip_code": null } ] } ] }
+{ "id": 15, "t15": [ { "$15": null }, { "$15": null } ] }
+{ "id": 16, "t16": [ { }, { } ] }
+{ "id": 17, "t17": [ { "$17": null } ] }
+{ "id": 18, "t18": [ { "$18": null } ] }
+{ "id": 19, "t19": [ { "$19": null } ] }
+{ "id": 20, "t20": [ { "$20": null } ] }
+{ "id": 21, "t21": [ { } ] }
+{ "id": 22, "t22": [ { } ] }
+{ "id": 23, "t23": [ { } ] }
+{ "id": 24, "t24": [ { "$24": [ 5, 1, 2, 3, 4 ] } ] }
+{ "id": 25, "t25": [ { "$25": [ 1, 4, 2, 3, 5 ] } ] }
+{ "id": 26, "t26": [ { "$26": null } ] }
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/array_fun/array_swap/array_swap.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_swap/array_swap.3.adm
new file mode 100644
index 0000000..97ccdc4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_swap/array_swap.3.adm
@@ -0,0 +1,26 @@
+{ "id": 1, "t1": [ { "$1": [ 0, 3, 2, 1, 4, 5 ] } ] }
+{ "id": 2, "t2": [ { "$2": [ 0, 1, 2, 5, 4, 3 ] } ] }
+{ "id": 3, "t3": [ { "$3": [ "c", "b", "a", "d", "e" ] } ] }
+{ "id": 4, "t4": [ { "$4": [ "a", "d", "c", "b", "e" ] } ] }
+{ "id": 5, "t5": [ { "$5": null } ] }
+{ "id": 6, "t6": [ { "$6": null } ] }
+{ "id": 7, "t7": [ { "$7": null } ] }
+{ "id": 8, "t8": [ { "$8": null } ] }
+{ "id": 9, "t9": [ { "$9": [ [ 9999, 3 ], [ 1, 2, 1 ] ] }, { "$9": [ [ "red", "black" ], [ "white", "blue", "magenta" ] ] }, { "$9": [ 2, 1 ] }, { "$9": [ { "state": "CA", "country": "US", "zip_code": 92863 }, { "state": "OH", "country": "US" } ] }, { "$9": [ { "state": "IL", "country": "US", "zip_code": 92863 }, { "state": "OR", "country": "US", "zip_code": null } ] }, { "$9": null }, { } ] }
+{ "id": 10, "t10": [ { "$10": [ [ 9999, 3 ], [ 1, 2, 1 ] ] }, { "$10": [ [ "red", "black" ], [ "white", "blue", "magenta" ] ] }, { "$10": [ 2, 1 ] }, { "$10": [ { "state": "CA", "country": "US", "zip_code": 92863 }, { "state": "OH", "country": "US" } ] }, { "$10": [ { "state": "IL", "country": "US", "zip_code": 92863 }, { "state": "OR", "country": "US", "zip_code": null } ] }, { "$10": null }, { } ] }
+{ "id": 11, "t11": [ { "$11": null }, { "$11": null }, { "$11": null }, { "$11": null }, { "$11": null }, { "$11": null }, { } ] }
+{ "id": 12, "t12": [ { }, { }, { }, { }, { }, { }, { } ] }
+{ "id": 13, "t13": [ { "$13": [ { "state": "CA", "country": "US", "zip_code": 92863 }, { "state": "OH", "country": "US" } ] }, { "$13": [ { "state": "IL", "country": "US", "zip_code": 92863 }, { "state": "OR", "country": "US", "zip_code": null } ] } ] }
+{ "id": 14, "t14": [ { "$14": [ { "state": "CA", "country": "US", "zip_code": 92863 }, { "state": "OH", "country": "US" } ] }, { "$14": [ { "state": "IL", "country": "US", "zip_code": 92863 }, { "state": "OR", "country": "US", "zip_code": null } ] } ] }
+{ "id": 15, "t15": [ { "$15": null }, { "$15": null } ] }
+{ "id": 16, "t16": [ { }, { } ] }
+{ "id": 17, "t17": [ { "$17": null } ] }
+{ "id": 18, "t18": [ { "$18": null } ] }
+{ "id": 19, "t19": [ { "$19": null } ] }
+{ "id": 20, "t20": [ { "$20": null } ] }
+{ "id": 21, "t21": [ { } ] }
+{ "id": 22, "t22": [ { } ] }
+{ "id": 23, "t23": [ { } ] }
+{ "id": 24, "t24": [ { "$24": [ 5, 2, 3, 4, 1 ] } ] }
+{ "id": 25, "t25": [ { "$25": [ 1, 4, 3, 2, 5 ] } ] }
+{ "id": 26, "t26": [ { "$26": null } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/async-deferred/async-exhausted-result/async-exhausted-result.3.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/async-deferred/async-exhausted-result/async-exhausted-result.3.regexjson
index df5177b..7fe2d93 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/async-deferred/async-exhausted-result/async-exhausted-result.3.regexjson
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/async-deferred/async-exhausted-result/async-exhausted-result.3.regexjson
@@ -11,5 +11,6 @@
{ "i": 9, "i2": 81 },
{ "i": 10, "i2": 100 }
],
- "metrics": "R{.*}"
+ "metrics": "R{.*}",
+ "profile": "R{.*}"
}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/async-deferred/async-repeated/async-repeated.3.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/async-deferred/async-repeated/async-repeated.3.regexjson
index df5177b..7fe2d93 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/async-deferred/async-repeated/async-repeated.3.regexjson
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/async-deferred/async-repeated/async-repeated.3.regexjson
@@ -11,5 +11,6 @@
{ "i": 9, "i2": 81 },
{ "i": 10, "i2": 100 }
],
- "metrics": "R{.*}"
+ "metrics": "R{.*}",
+ "profile": "R{.*}"
}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/async-deferred/async-repeated/async-repeated.4.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/async-deferred/async-repeated/async-repeated.4.regexjson
index df5177b..7fe2d93 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/async-deferred/async-repeated/async-repeated.4.regexjson
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/async-deferred/async-repeated/async-repeated.4.regexjson
@@ -11,5 +11,6 @@
{ "i": 9, "i2": 81 },
{ "i": 10, "i2": 100 }
],
- "metrics": "R{.*}"
+ "metrics": "R{.*}",
+ "profile": "R{.*}"
}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/async-deferred/async-running/async-running.4.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/async-deferred/async-running/async-running.4.regexjson
index a2a5f0a8..3e0193d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/async-deferred/async-running/async-running.4.regexjson
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/async-deferred/async-running/async-running.4.regexjson
@@ -2,5 +2,6 @@
"results": [
"result"
],
- "metrics": "R{.*}"
+ "metrics": "R{.*}",
+ "profile": "R{.*}"
}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/async-deferred/async/async.3.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/async-deferred/async/async.3.regexjson
index df5177b..7fe2d93 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/async-deferred/async/async.3.regexjson
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/async-deferred/async/async.3.regexjson
@@ -11,5 +11,6 @@
{ "i": 9, "i2": 81 },
{ "i": 10, "i2": 100 }
],
- "metrics": "R{.*}"
+ "metrics": "R{.*}",
+ "profile": "R{.*}"
}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/async-deferred/deferred/deferred.2.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/async-deferred/deferred/deferred.2.regexjson
index df5177b..7fe2d93 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/async-deferred/deferred/deferred.2.regexjson
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/async-deferred/deferred/deferred.2.regexjson
@@ -11,5 +11,6 @@
{ "i": 9, "i2": 81 },
{ "i": 10, "i2": 100 }
],
- "metrics": "R{.*}"
+ "metrics": "R{.*}",
+ "profile": "R{.*}"
}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.11.adm
new file mode 100644
index 0000000..58f454b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.11.adm
@@ -0,0 +1 @@
+{ "metadata": { "DatasetName": "ds1", "IndexName": "sample_idx_2_ds1", "SampleCardinalityTarget": 4252, "SourceCardinality": 1100, "SourceAvgItemSize": true, "SampleSeed": 234 }, "stats": { "cnt": 1100, "min_pk": 1, "max_pk": 1100, "min_x": -1100, "max_x": -1 } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.14.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.14.adm
new file mode 100644
index 0000000..6ef756a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.14.adm
@@ -0,0 +1 @@
+{ "metadata": { "DatasetName": "ds1", "IndexName": "sample_idx_1_ds1", "SampleCardinalityTarget": 4252, "SourceCardinality": 4400, "SourceAvgItemSize": true, "SampleSeed": true }, "stats": { "cnt": 4246, "min_pk": true, "max_pk": true, "min_x": true, "max_x": true } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.16.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.16.adm
new file mode 100644
index 0000000..01eb5b7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.16.adm
@@ -0,0 +1 @@
+{ "metadata": { "DatasetName": "ds1", "IndexName": "sample_idx_2_ds1", "SampleCardinalityTarget": 17008, "SourceCardinality": 4400, "SourceAvgItemSize": true, "SampleSeed": 345 }, "stats": { "cnt": 4400, "min_pk": 1, "max_pk": 4400, "min_x": -4400, "max_x": -1 } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.19.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.19.adm
new file mode 100644
index 0000000..60b969f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.19.adm
@@ -0,0 +1 @@
+{ "metadata": { "DatasetName": "ds1", "IndexName": "sample_idx_1_ds1", "SampleCardinalityTarget": 17008, "SourceCardinality": 17100, "SourceAvgItemSize": true, "SampleSeed": true }, "stats": { "cnt": 16972, "min_pk": true, "max_pk": true, "min_x": true, "max_x": true } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.2.adm
new file mode 100644
index 0000000..e3cefee
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.2.adm
@@ -0,0 +1 @@
+{ "DatasetName": "ds1", "IndexName": "sample_idx_1_ds1", "SampleCardinalityTarget": 1063, "SourceCardinality": 0, "SourceAvgItemSize": 0, "SampleSeed": true }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.21.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.21.adm
new file mode 100644
index 0000000..bacb60c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.21.adm
@@ -0,0 +1 @@
+{ "cnt": 0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.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..f86e66b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.23.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.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..bacb60c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.25.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..00aafda
--- /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: 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 ([$$47]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$47] <- [{"deptId": $#1, "star_cost": $$50}] [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|
+ 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: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- SORT_GROUP_BY[$$55] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.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: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- SORT_GROUP_BY[$$48] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$45, $$48]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$48, $$45] <- [substring($$e.getField("dept").getField("department_id"), 0), $$e.getField("salary")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$e]) [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 []<-[$$49, $$e] <- gby.Employee [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/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..fd2deff
--- /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: 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 ([$$46]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$46] <- [{"deptId": $#1, "star_cost": $$49}] [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|
+ 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: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- SORT_GROUP_BY[$$53] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.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: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- SORT_GROUP_BY[$$47] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$44, $$47]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$47, $$44] <- [substring($$e.getField(1), 0), $$e.getField(2)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$e]) [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 []<-[$$48, $$e] <- gby.Employee [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/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_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..7390e2a
--- /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: 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 ([$$17]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$17] <- [{"display_url": $$20}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$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]
+ -- SORT_MERGE_EXCHANGE [$$19(ASC) ] |PARTITIONED|
+ order (ASC, $$19) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STABLE_SORT [$$19(ASC)] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$20, $$19]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$20, $$19] <- [get-item($$p.getField("entities").getField("urls"), 0).getField("display_url"), $$p.getField("id")] [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 []<-[$$p] <- test.ParquetDataset [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/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..d18185b
--- /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: 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 ([$$17]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$17] <- [{"display_url": $$20}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$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]
+ -- SORT_MERGE_EXCHANGE [$$19(ASC) ] |PARTITIONED|
+ order (ASC, $$19) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STABLE_SORT [$$19(ASC)] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$20, $$19]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$20, $$19] <- [get-item($$p.getField("entities").getField("urls"), 0).getField("display_url"), $$p.getField("id")] [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 []<-[$$p] <- test.ParquetDataset project ({entities:{urls:[{display_url:any}]},id: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/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..e52844d
--- /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: 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 ([$$21]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$21] <- [{"display_url": $$25}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$25]) [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 [$$24(ASC) ] |PARTITIONED|
+ order (ASC, $$24) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STABLE_SORT [$$24(ASC)] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$25, $$24]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$25] <- [array-star($$23).getField("display_url")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ select (not(is-missing($$23))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_SELECT |PARTITIONED|
+ project ([$$23, $$24]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$23, $$24] <- [$$p.getField("entities").getField("urls"), $$p.getField("id")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p] <- test.ParquetDataset [cardinality: 1000000.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/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..357aeff
--- /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: 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 ([$$21]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$21] <- [{"display_url": $$25}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$25]) [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 [$$24(ASC) ] |PARTITIONED|
+ order (ASC, $$24) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STABLE_SORT [$$24(ASC)] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$25, $$24]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$25] <- [array-star($$23).getField("display_url")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ select (not(is-missing($$23))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_SELECT |PARTITIONED|
+ project ([$$23, $$24]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$23, $$24] <- [$$p.getField("entities").getField("urls"), $$p.getField("id")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p] <- test.ParquetDataset project ({entities:{urls:[{display_url:any}]},id:any}) [cardinality: 1000000.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/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..4b19e00
--- /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: 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 ([$$26]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$26] <- [{"display_url": $$urls.getField("display_url")}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$urls]) [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 [$$29(ASC) ] |PARTITIONED|
+ order (ASC, $$29) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STABLE_SORT [$$29(ASC)] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$urls, $$29]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ unnest $$urls <- scan-collection($$28) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- UNNEST |PARTITIONED|
+ project ([$$28, $$29]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$28, $$29] <- [$$p.getField("entities").getField("urls"), $$p.getField("id")] [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 []<-[$$p] <- test.ParquetDataset [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/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..fd5eea3
--- /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: 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 ([$$26]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$26] <- [{"display_url": $$urls.getField("display_url")}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$urls]) [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 [$$29(ASC) ] |PARTITIONED|
+ order (ASC, $$29) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STABLE_SORT [$$29(ASC)] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$urls, $$29]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ unnest $$urls <- scan-collection($$28) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- UNNEST |PARTITIONED|
+ project ([$$28, $$29]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$28, $$29] <- [$$p.getField("entities").getField("urls"), $$p.getField("id")] [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 []<-[$$p] <- test.ParquetDataset project ({entities:{urls:[{display_url:any}]},id: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/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..23cb456
--- /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: 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|
+ aggregate [$$46] <- [agg-sql-sum($$52)] [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|
+ aggregate [$$52] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- AGGREGATE |PARTITIONED|
+ select ($$39) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_SELECT |PARTITIONED|
+ project ([$$39]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- 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: 0.0]
+ -- SUBPLAN |PARTITIONED|
+ project ([$$47]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$47] <- [$$p.getField("entities").getField("urls")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p] <- test.ParquetDataset [cardinality: 1000000.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/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..5d07555
--- /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: 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|
+ aggregate [$$46] <- [agg-sql-sum($$52)] [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|
+ aggregate [$$52] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- AGGREGATE |PARTITIONED|
+ select ($$39) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_SELECT |PARTITIONED|
+ project ([$$39]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- 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: 0.0]
+ -- SUBPLAN |PARTITIONED|
+ project ([$$47]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$47] <- [$$p.getField("entities").getField("urls")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p] <- test.ParquetDataset project ({entities:{urls:[{display_url:any}]}}) [cardinality: 1000000.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/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..62eeb47
--- /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: 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 ([$$18]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$18] <- [get-item(get-item(get-item($$21, 0), 0), 0)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$21]) [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 [$$20(ASC) ] |PARTITIONED|
+ order (ASC, $$20) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$21, $$20]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$21, $$20] <- [$$p.getField("place").getField("bounding_box").getField("coordinates"), $$p.getField("id")] [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 []<-[$$p] <- test.ParquetDataset [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/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..7143a1c
--- /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: 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 ([$$18]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$18] <- [get-item(get-item(get-item($$21, 0), 0), 0)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$21]) [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 [$$20(ASC) ] |PARTITIONED|
+ order (ASC, $$20) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$21, $$20]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$21, $$20] <- [$$p.getField("place").getField("bounding_box").getField("coordinates"), $$p.getField("id")] [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 []<-[$$p] <- test.ParquetDataset project ({place:{bounding_box:{coordinates:[[[any]]]}},id: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/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..5ddd589
--- /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: 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 ([$$p]) [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 [$$13(ASC) ] |PARTITIONED|
+ order (ASC, $$13) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STABLE_SORT [$$13(ASC)] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ assign [$$13] <- [$$p.getField("id")] [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 []<-[$$p] <- test.ParquetDataset [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/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..5ddd589
--- /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: 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 ([$$p]) [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 [$$13(ASC) ] |PARTITIONED|
+ order (ASC, $$13) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STABLE_SORT [$$13(ASC)] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ assign [$$13] <- [$$p.getField("id")] [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 []<-[$$p] <- test.ParquetDataset [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/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..51e4b81
--- /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: 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 ([$$17]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$17] <- [{"id": $$21, "name": $$22}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |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]
+ -- SORT_MERGE_EXCHANGE [$$20(ASC) ] |PARTITIONED|
+ order (ASC, $$20) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$21, $$22, $$20]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$22, $$21] <- [$$19.getField("name"), $$19.getField("id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$19, $$20]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$19, $$20] <- [$$p.getField("user"), $$p.getField("id")] [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 []<-[$$p] <- test.ParquetDataset4 [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/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..65a4217
--- /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: 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 ([$$17]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$17] <- [{"id": $$21, "name": $$22}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |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]
+ -- SORT_MERGE_EXCHANGE [$$20(ASC) ] |PARTITIONED|
+ order (ASC, $$20) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$21, $$22, $$20]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$22, $$21] <- [$$19.getField("name"), $$19.getField("id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$19, $$20]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$19, $$20] <- [$$p.getField("user"), $$p.getField("id")] [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 []<-[$$p] <- test.ParquetDataset4 project ({id:any,user:{name:any,id: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/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..67c9a9f
--- /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: 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|
+ aggregate [$$31] <- [agg-sql-sum($$32)] [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|
+ aggregate [$$32] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- AGGREGATE |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p] <- test.ParquetDataset4 [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/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..5229312
--- /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: 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|
+ aggregate [$$31] <- [agg-sql-sum($$32)] [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|
+ aggregate [$$32] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- AGGREGATE |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p] <- test.ParquetDataset4 project ({}) [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/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..65a4217
--- /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: 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 ([$$17]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$17] <- [{"id": $$21, "name": $$22}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |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]
+ -- SORT_MERGE_EXCHANGE [$$20(ASC) ] |PARTITIONED|
+ order (ASC, $$20) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$21, $$22, $$20]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$22, $$21] <- [$$19.getField("name"), $$19.getField("id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$19, $$20]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$19, $$20] <- [$$p.getField("user"), $$p.getField("id")] [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 []<-[$$p] <- test.ParquetDataset4 project ({id:any,user:{name:any,id: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/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..7269401
--- /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: 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 ([$$20]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$20] <- [switch-case(true, is-array($$22), $$24, $$25)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$22, $$24, $$25]) [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 [$$23(ASC) ] |PARTITIONED|
+ order (ASC, $$23) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STABLE_SORT [$$23(ASC)] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ assign [$$24, $$25] <- [array-star($$22).getField("text"), $$22.getField("text")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$22, $$23]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$22, $$23] <- [$$p.getField("arrayOrObject"), $$p.getField("id")] [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 []<-[$$p] <- test.ParquetDataset [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/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..cf67bc7
--- /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: 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 ([$$20]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$20] <- [switch-case(true, is-array($$22), $$24, $$25)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$22, $$24, $$25]) [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 [$$23(ASC) ] |PARTITIONED|
+ order (ASC, $$23) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STABLE_SORT [$$23(ASC)] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ assign [$$24, $$25] <- [array-star($$22).getField("text"), $$22.getField("text")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$22, $$23]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$22, $$23] <- [$$p.getField("arrayOrObject"), $$p.getField("id")] [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 []<-[$$p] <- test.ParquetDataset project ({arrayOrObject:<[{text:any}],{text:any}>,id: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/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..2fe8496
--- /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: 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 ([$$16]) [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 [$$18(ASC) ] |PARTITIONED|
+ order (ASC, $$18) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STABLE_SORT [$$18(ASC)] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$16, $$18]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$16, $$18] <- [object-concat($$p.getField("coordinates"), $$p.getField("user")).getField("name"), $$p.getField("id")] [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 []<-[$$p] <- test.ParquetDataset project ({coordinates:any,id:any,user: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/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..1e4c4e1
--- /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: 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 ([$$15]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$15] <- [object-concat($$18, $$19)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$18, $$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 [$$17(ASC) ] |PARTITIONED|
+ order (ASC, $$17) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STABLE_SORT [$$17(ASC)] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$18, $$19, $$17]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$19, $$18, $$17] <- [$$p.getField("user"), $$p.getField("coordinates"), $$p.getField("id")] [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 []<-[$$p] <- test.ParquetDataset project ({coordinates:any,id:any,user: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/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..196c0ce
--- /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: 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 3 [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|
+ project ([$$13]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$13] <- [{"test": $$test}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ limit 3 [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|
+ data-scan []<-[$$test] <- test.test limit 3 [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-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/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..911eafe
--- /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: 1.0, op-cost: 0.0, total-cost: 1.1E7]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 1.1E7]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$48]) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.1E7]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$48] <- [{"n_nationkey": $$55, "s_nationkey": $$53, "c_nationkey": $$52}] [cardinality: 1.0, op-cost: 0.0, total-cost: 1.1E7]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1.0, 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: 1.0, op-cost: 0.0, total-cost: 1.1E7]
+ -- STABLE_SORT [$$55(ASC), $$53(ASC), $$52(ASC)] |PARTITIONED|
+ exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 1.1E7]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ join (eq($$52, $$55)) [cardinality: 1.0, 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..43c3697
--- /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: 1.0, op-cost: 0.0, total-cost: 1.1E7]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 1.1E7]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$48]) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.1E7]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$48] <- [{"n_nationkey": $$56, "s_nationkey": $$53, "c_nationkey": $$52}] [cardinality: 1.0, op-cost: 0.0, total-cost: 1.1E7]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1.0, 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: 1.0, op-cost: 0.0, total-cost: 1.1E7]
+ -- STABLE_SORT [$$56(ASC), $$53(ASC), $$52(ASC)] |PARTITIONED|
+ exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 1.1E7]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$56, $$53, $$52]) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.1E7]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 1.1E7]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ join (and(eq($$52, $$56), eq($$53, $$63))) [cardinality: 1.0, 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..f4c0080
--- /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: 1.0, op-cost: 0.0, total-cost: 1.0E7]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0E7]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$48]) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0E7]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$48] <- [{"n_nationkey": $$56, "s_nationkey": $$53, "c_nationkey": $$52}] [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0E7]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1.0, 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: 1.0, op-cost: 0.0, total-cost: 1.0E7]
+ -- STABLE_SORT [$$56(ASC), $$53(ASC), $$52(ASC)] |PARTITIONED|
+ exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0E7]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$56, $$53, $$52]) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0E7]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0E7]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ join (and(eq($$52, $$56), eq($$53, $$63))) [cardinality: 1.0, 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..37a1ef4
--- /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: 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 offset 98 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_LIMIT |UNPARTITIONED|
+ project ([$$15]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$15] <- [{"id": $$17, "dblpid": $$paper.getField(1)}] [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 [$$17(ASC) ] |PARTITIONED|
+ data-scan []<-[$$17, $$paper] <- test.DBLP1 [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-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..d1d6400
--- /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: 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|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ project ([$$13]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$13] <- [$$t.getField(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|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$t] <- test.ds1 limit 5 [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-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..d97ef17
--- /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: 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 ([$$paper]) [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 [$$12(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|
+ data-scan []<-[$$12, $$paper] <- test.DBLP1 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.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..50f6e55
--- /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: 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 ([$$paper]) [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) ] |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|
+ data-scan []<-[$$14, $$paper] <- test.DBLP1 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.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..a7095d6
--- /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: 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 ([$$75]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$75] <- [get-item($$73, 0)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$73]) [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 [$$77(ASC) ] |PARTITIONED|
+ project ([$$73, $$77]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.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: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- SUBPLAN |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|
+ data-scan []<-[$$77, $$paper] <- test.DBLP1 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/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/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..a52ceae
--- /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": 30, "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..7bc4c42 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+917504/
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..acf8405 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+1146880/
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/metadata_only_02/metadata_only_02.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/metadata_only_02/metadata_only_02.2.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.2.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/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/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/objects/object_add/object_add.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_add/object_add.3.adm
index 800859b..94bb2b5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_add/object_add.3.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_add/object_add.3.adm
@@ -1 +1 @@
-{ "t1": [ true, true, true, true, true, true, true ], "t2": { "a": 1, "field": "value" }, "t3": { "a": 1, "field": null }, "t4": { "a": 1, "field": { "x": [ "y", "z" ] } }, "t5": { "a": 1 }, "t6": { "a": 1 }, "t7": [ { "screen-name": "ChangEwing_573", "lang": "en", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136, "field": "value" } ], "t8": [ { "tweetid": "1", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:00.000"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)", "field": { "a": 1 } } ] }
+{ "t1": [ true, true, true, true, true, true, true ], "t2": { "a": 1, "field": "value" }, "t3": { "a": 1, "field": null }, "t4": { "a": 1, "field": { "x": [ "y", "z" ] } }, "t5": { "a": 1 }, "t6": { "a": 1 }, "t7": [ { "screen-name": "ChangEwing_573", "lang": "en", "friends_count": 182, "statuses_count": 394, "field": "value", "name": "Chang Ewing", "followers_count": 32136 } ], "t8": [ { "tweetid": "1", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:00.000"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)", "field": { "a": 1 } } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_add/object_add.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_add/object_add.7.adm
new file mode 100644
index 0000000..3bbbabb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_add/object_add.7.adm
@@ -0,0 +1 @@
+{ "t1c": [ { "_id": 1, "best_friend": { "_id": 1 }, "friends": [ ], "name": "John" }, { "_id": 2, "best_friend": { "_id": 1 }, "friends": [ { "_id": 1 } ], "name": "John" }, { "_id": 3, "best_friend": { "_id": 2 }, "friends": [ { "_id": 1 }, { "_id": 2 } ], "name": "John" } ], "t1o": [ { "_id": 4, "name": "John", "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [ ] }, { "_id": 5, "name": "John", "favorite_color": "Blue", "best_friend": { "_id": 4 }, "friends": [ { "_id": 4 } ] }, { "_id": 6, "name": "John", "favorite_color": "Orange", "best_friend": { "_id": 5 }, "friends": [ { "_id": 4 }, { "_id": 5 } ] } ], "t2c": [ { "_id": 1, "best_friend": { "_id": 1 }, "friends": [ ], "newFriend": { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [ ] } }, { "_id": 2, "best_friend": { "_id": 1 }, "friends": [ { "_id": 1 } ], "newFriend": { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [ ] } }, { "_id": 3, "best_friend": { "_id": 2 }, "friends": [ { "_id": 1 }, { "_id": 2 } ], "newFriend": { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [ ] } } ], "t2o": [ { "_id": 4, "newFriend": { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [ ] }, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [ ] }, { "_id": 5, "newFriend": { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [ ] }, "favorite_color": "Blue", "best_friend": { "_id": 4 }, "friends": [ { "_id": 4 } ] }, { "_id": 6, "newFriend": { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [ ] }, "favorite_color": "Orange", "best_friend": { "_id": 5 }, "friends": [ { "_id": 4 }, { "_id": 5 } ] } ], "t3c": [ { "_id": 1, "best_friend": { "_id": 1 }, "friends": [ ], "newFriend": { "_id": 1, "best_friend": { "_id": 1 }, "friends": [ ] } }, { "_id": 2, "best_friend": { "_id": 1 }, "friends": [ { "_id": 1 } ], "newFriend": { "_id": 1, "best_friend": { "_id": 1 }, "friends": [ ] } }, { "_id": 3, "best_friend": { "_id": 2 }, "friends": [ { "_id": 1 }, { "_id": 2 } ], "newFriend": { "_id": 1, "best_friend": { "_id": 1 }, "friends": [ ] } } ], "t3o": [ { "_id": 4, "newFriend": { "_id": 1, "best_friend": { "_id": 1 }, "friends": [ ] }, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [ ] }, { "_id": 5, "newFriend": { "_id": 1, "best_friend": { "_id": 1 }, "friends": [ ] }, "favorite_color": "Blue", "best_friend": { "_id": 4 }, "friends": [ { "_id": 4 } ] }, { "_id": 6, "newFriend": { "_id": 1, "best_friend": { "_id": 1 }, "friends": [ ] }, "favorite_color": "Orange", "best_friend": { "_id": 5 }, "friends": [ { "_id": 4 }, { "_id": 5 } ] } ], "t4c": [ { "_id": 1, "best_friend": { "_id": 1 }, "friends": [ ], "newFriends": [ { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [ ] } ] }, { "_id": 2, "best_friend": { "_id": 1 }, "friends": [ { "_id": 1 } ], "newFriends": [ { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [ ] } ] }, { "_id": 3, "best_friend": { "_id": 2 }, "friends": [ { "_id": 1 }, { "_id": 2 } ], "newFriends": [ { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [ ] } ] } ], "t4o": [ { "_id": 4, "newFriends": [ { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [ ] } ], "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [ ] }, { "_id": 5, "newFriends": [ { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [ ] } ], "favorite_color": "Blue", "best_friend": { "_id": 4 }, "friends": [ { "_id": 4 } ] }, { "_id": 6, "newFriends": [ { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [ ] } ], "favorite_color": "Orange", "best_friend": { "_id": 5 }, "friends": [ { "_id": 4 }, { "_id": 5 } ] } ], "t5c": [ { "_id": 1, "best_friend": { "_id": 1 }, "friends": [ ], "newFriends": [ { "_id": 1, "best_friend": { "_id": 1 }, "friends": [ ] } ] }, { "_id": 2, "best_friend": { "_id": 1 }, "friends": [ { "_id": 1 } ], "newFriends": [ { "_id": 1, "best_friend": { "_id": 1 }, "friends": [ ] } ] }, { "_id": 3, "best_friend": { "_id": 2 }, "friends": [ { "_id": 1 }, { "_id": 2 } ], "newFriends": [ { "_id": 1, "best_friend": { "_id": 1 }, "friends": [ ] } ] } ], "t5o": [ { "_id": 4, "newFriends": [ { "_id": 1, "best_friend": { "_id": 1 }, "friends": [ ] } ], "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [ ] }, { "_id": 5, "newFriends": [ { "_id": 1, "best_friend": { "_id": 1 }, "friends": [ ] } ], "favorite_color": "Blue", "best_friend": { "_id": 4 }, "friends": [ { "_id": 4 } ] }, { "_id": 6, "newFriends": [ { "_id": 1, "best_friend": { "_id": 1 }, "friends": [ ] } ], "favorite_color": "Orange", "best_friend": { "_id": 5 }, "friends": [ { "_id": 4 }, { "_id": 5 } ] } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_concat/object_concat.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_concat/object_concat.4.adm
new file mode 100644
index 0000000..ad527c3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_concat/object_concat.4.adm
@@ -0,0 +1 @@
+{ "$1": { "v": { "id": 2, "f": 3 } } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_concat/object_concat.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_concat/object_concat.5.adm
new file mode 100644
index 0000000..e20e752
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_concat/object_concat.5.adm
@@ -0,0 +1 @@
+{ "res": { "id": 2, "f": 4 } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_concat/object_concat.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_concat/object_concat.6.adm
new file mode 100644
index 0000000..80023c6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_concat/object_concat.6.adm
@@ -0,0 +1 @@
+{ "id": 1, "b1": 3, "dup": 5, "a1": 3 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_put/object_put.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_put/object_put.3.adm
index 4b97741..2947405 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_put/object_put.3.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_put/object_put.3.adm
@@ -1 +1 @@
-{ "t1": [ true, true, true, true, true, true, true ], "t2": { "a": 1, "field": "value" }, "t3": { "a": 1, "field": null }, "t4": { "a": 1, "field": { "x": [ "y", "z" ] } }, "t5": { "a": "replaced" }, "t6": { "a": 1 }, "t7": [ { "screen-name": "ChangEwing_573", "lang": "en", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136, "field": "value" } ], "t8": [ { "tweetid": "1", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:00.000"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)", "field": { "a": 1 } } ] }
+{ "t1": [ true, true, true, true, true, true, true ], "t2": { "a": 1, "field": "value" }, "t3": { "a": 1, "field": null }, "t4": { "a": 1, "field": { "x": [ "y", "z" ] } }, "t5": { "a": "replaced" }, "t6": { "a": 1 }, "t7": [ { "screen-name": "ChangEwing_573", "lang": "en", "friends_count": 182, "statuses_count": 394, "field": "value", "name": "Chang Ewing", "followers_count": 32136 } ], "t8": [ { "tweetid": "1", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:00.000"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)", "field": { "a": 1 } } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_put/object_put.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_put/object_put.7.adm
new file mode 100644
index 0000000..32b297d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_put/object_put.7.adm
@@ -0,0 +1 @@
+{ "t1c": [ { "_id": 1, "best_friend": { "_id": 1 }, "friends": [ ], "name": "John" }, { "_id": 2, "best_friend": { "_id": 1 }, "friends": [ { "_id": 1 } ], "name": "John" }, { "_id": 3, "best_friend": { "_id": 2 }, "friends": [ { "_id": 1 }, { "_id": 2 } ], "name": "John" } ], "t1o": [ { "_id": 4, "name": "John", "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [ ] }, { "_id": 5, "name": "John", "favorite_color": "Blue", "best_friend": { "_id": 4 }, "friends": [ { "_id": 4 } ] }, { "_id": 6, "name": "John", "favorite_color": "Orange", "best_friend": { "_id": 5 }, "friends": [ { "_id": 4 }, { "_id": 5 } ] } ], "t2c": [ { "_id": 1, "best_friend": { "_id": 1 }, "friends": [ ], "newFriend": { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [ ] } }, { "_id": 2, "best_friend": { "_id": 1 }, "friends": [ { "_id": 1 } ], "newFriend": { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [ ] } }, { "_id": 3, "best_friend": { "_id": 2 }, "friends": [ { "_id": 1 }, { "_id": 2 } ], "newFriend": { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [ ] } } ], "t2o": [ { "_id": 4, "newFriend": { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [ ] }, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [ ] }, { "_id": 5, "newFriend": { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [ ] }, "favorite_color": "Blue", "best_friend": { "_id": 4 }, "friends": [ { "_id": 4 } ] }, { "_id": 6, "newFriend": { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [ ] }, "favorite_color": "Orange", "best_friend": { "_id": 5 }, "friends": [ { "_id": 4 }, { "_id": 5 } ] } ], "t3c": [ { "_id": 1, "best_friend": { "_id": 1 }, "friends": [ ], "newFriend": { "_id": 1, "best_friend": { "_id": 1 }, "friends": [ ] } }, { "_id": 2, "best_friend": { "_id": 1 }, "friends": [ { "_id": 1 } ], "newFriend": { "_id": 1, "best_friend": { "_id": 1 }, "friends": [ ] } }, { "_id": 3, "best_friend": { "_id": 2 }, "friends": [ { "_id": 1 }, { "_id": 2 } ], "newFriend": { "_id": 1, "best_friend": { "_id": 1 }, "friends": [ ] } } ], "t3o": [ { "_id": 4, "newFriend": { "_id": 1, "best_friend": { "_id": 1 }, "friends": [ ] }, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [ ] }, { "_id": 5, "newFriend": { "_id": 1, "best_friend": { "_id": 1 }, "friends": [ ] }, "favorite_color": "Blue", "best_friend": { "_id": 4 }, "friends": [ { "_id": 4 } ] }, { "_id": 6, "newFriend": { "_id": 1, "best_friend": { "_id": 1 }, "friends": [ ] }, "favorite_color": "Orange", "best_friend": { "_id": 5 }, "friends": [ { "_id": 4 }, { "_id": 5 } ] } ], "t4c": [ { "_id": 1, "best_friend": { "_id": 1 }, "friends": [ ], "newFriends": [ { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [ ] } ] }, { "_id": 2, "best_friend": { "_id": 1 }, "friends": [ { "_id": 1 } ], "newFriends": [ { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [ ] } ] }, { "_id": 3, "best_friend": { "_id": 2 }, "friends": [ { "_id": 1 }, { "_id": 2 } ], "newFriends": [ { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [ ] } ] } ], "t4o": [ { "_id": 4, "newFriends": [ { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [ ] } ], "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [ ] }, { "_id": 5, "newFriends": [ { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [ ] } ], "favorite_color": "Blue", "best_friend": { "_id": 4 }, "friends": [ { "_id": 4 } ] }, { "_id": 6, "newFriends": [ { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [ ] } ], "favorite_color": "Orange", "best_friend": { "_id": 5 }, "friends": [ { "_id": 4 }, { "_id": 5 } ] } ], "t5c": [ { "_id": 1, "best_friend": { "_id": 1 }, "friends": [ ], "newFriends": [ { "_id": 1, "best_friend": { "_id": 1 }, "friends": [ ] } ] }, { "_id": 2, "best_friend": { "_id": 1 }, "friends": [ { "_id": 1 } ], "newFriends": [ { "_id": 1, "best_friend": { "_id": 1 }, "friends": [ ] } ] }, { "_id": 3, "best_friend": { "_id": 2 }, "friends": [ { "_id": 1 }, { "_id": 2 } ], "newFriends": [ { "_id": 1, "best_friend": { "_id": 1 }, "friends": [ ] } ] } ], "t5o": [ { "_id": 4, "newFriends": [ { "_id": 1, "best_friend": { "_id": 1 }, "friends": [ ] } ], "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [ ] }, { "_id": 5, "newFriends": [ { "_id": 1, "best_friend": { "_id": 1 }, "friends": [ ] } ], "favorite_color": "Blue", "best_friend": { "_id": 4 }, "friends": [ { "_id": 4 } ] }, { "_id": 6, "newFriends": [ { "_id": 1, "best_friend": { "_id": 1 }, "friends": [ ] } ], "favorite_color": "Orange", "best_friend": { "_id": 5 }, "friends": [ { "_id": 4 }, { "_id": 5 } ] } ], "t6c": [ { "_id": 1, "best_friend": { "_id": 1 }, "friends": [ { "_id": 8 }, { "_id": 9 } ] }, { "_id": 2, "best_friend": { "_id": 1 }, "friends": [ { "_id": 8 }, { "_id": 9 } ] }, { "_id": 3, "best_friend": { "_id": 2 }, "friends": [ { "_id": 8 }, { "_id": 9 } ] } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/open-object-constructor-with-missing-field-name/open-object-constructor-with-missing-field-name.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/open-object-constructor-with-missing-field-name/open-object-constructor-with-missing-field-name.1.adm
new file mode 100644
index 0000000..ca46492
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/open-object-constructor-with-missing-field-name/open-object-constructor-with-missing-field-name.1.adm
@@ -0,0 +1,2 @@
+{ "status": "shipped" }
+{ }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/open-object-constructor-with-missing-field-name/open-object-constructor-with-missing-field-name.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/open-object-constructor-with-missing-field-name/open-object-constructor-with-missing-field-name.2.adm
new file mode 100644
index 0000000..008b550
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/open-object-constructor-with-missing-field-name/open-object-constructor-with-missing-field-name.2.adm
@@ -0,0 +1,3 @@
+{ "Video Card Orders": 2, "Storage Orders": 2, "status": "Cancelled" }
+{ "Video Card Orders": 2, "Storage Orders": 2, "status": "Pending" }
+{ "Video Card Orders": 2, "Storage Orders": 2, "status": "Shipped" }
\ No newline at end of file
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/profile/full-scan/full-scan.3.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/profile/full-scan/full-scan.3.regexjson
index c6f99d1b..0c4fae1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/profile/full-scan/full-scan.3.regexjson
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/profile/full-scan/full-scan.3.regexjson
@@ -1,5 +1,9 @@
{
"job-id": "R{[A-Z0-9.:]+}",
+ "create-time": "R{[0-9.]+}",
+ "start-time": "R{[0-9.]+}",
+ "queued-time": "R{.+}",
+ "end-time": "R{[0-9.]+}",
"counters": [],
"joblets": [
{
@@ -29,19 +33,29 @@
],
"counters": [
{
- "name": "Empty Tuple Source",
+ "name": "R{.+}",
"time": "R{[0-9.]+}",
- "disk-io": "R{[0-9.]+}"
- },
- {
- "name": "Index Search",
- "time": "R{[0-9.]+}",
- "disk-io": "R{[0-9.]+}"
+ "runtime-id": "R{.+}"
},
{
"name": "R{.+}",
"time": "R{[0-9.]+}",
- "disk-io": "R{[0-9.]+}"
+ "runtime-id": "R{.+}",
+ "pages-read": "R{[0-9.]+}",
+ "pages-read-cold": "R{[0-9.]+}",
+ "cardinality-out": "R{[0-9.]+}",
+ "avg-tuple-size": "R{[0-9.]+}",
+ "min-tuple-size": "R{[0-9.]+}",
+ "max-tuple-size": "R{[0-9.]+}"
+ },
+ {
+ "name": "R{.+}",
+ "time": "R{[0-9.]+}",
+ "runtime-id": "R{.+}",
+ "cardinality-out": "R{[0-9.]+}",
+ "avg-tuple-size": "R{[0-9.]+}",
+ "min-tuple-size": "R{[0-9.]+}",
+ "max-tuple-size": "R{[0-9.]+}"
}
]
},
@@ -52,18 +66,21 @@
"partition-send-profile": [],
"counters": [
{
- "name": "R{.+}",
- "time": "R{[0-9.]+}",
- "disk-io": "R{[0-9.]+}"
+ "name": "R{.+}",
+ "time": "R{[0-9.]+}",
+ "runtime-id": "R{.+}",
+ "cardinality-out": "R{[0-9.]+}",
+ "avg-tuple-size": "R{[0-9.]+}",
+ "min-tuple-size": "R{[0-9.]+}",
+ "max-tuple-size": "R{[0-9.]+}"
},
{
- "name": "Result Writer",
- "time": "R{[0-9.]+}",
- "disk-io": "R{[0-9.]+}"
+ "name": "R{.+}",
+ "time": "R{[0-9.]+}",
+ "runtime-id": "R{.+}"
}
]
}
]
- }
- ]
+ }]
}
\ 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..45c7028
--- /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: 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 ([$$v]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$v] <- [{"SK0": $$13, "PK0": $$14}] [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 [$$14(ASC) ] |PARTITIONED|
+ order (ASC, $$14) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STABLE_SORT [$$14(ASC)] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$13, $$14] <- test.ds1.ds1_age.query-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/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..0308f76
--- /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: 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 ([$$v]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$v] <- [{"SK0": $$13, "SK1": $$14, "PK0": $$15}] [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 [$$15(ASC) ] |PARTITIONED|
+ order (ASC, $$15) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STABLE_SORT [$$15(ASC)] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$13, $$14, $$15] <- test.ds1.ds1_age_dept.query-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/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..2648cf3
--- /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: 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 ([$$53]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$53] <- [{"age": $$SK0, "dept": $$SK1, "cnt": $$59}] [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 [$$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: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- SORT_GROUP_BY[$$61, $$62] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.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: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- SORT_GROUP_BY[$$57, $$56] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$57, $$56]) [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, $$57, $$58] <- test.ds1.ds1_age_dept.query-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|
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..58e4a62
--- /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: 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 ([$$31]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |UNPARTITIONED|
+ assign [$$31] <- [{"cnt": $$34}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |UNPARTITIONED|
+ aggregate [$$34] <- [agg-sql-sum($$35)] [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|
+ aggregate [$$35] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- AGGREGATE |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$32, $$33] <- test.ds1.ds1_dept.query-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/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..c86f9fb
--- /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: 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 ([$$42]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$42] <- [{"age": $$SK0, "cnt": $$46}] [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 [$$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: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- SORT_GROUP_BY[$$48] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.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: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- PRE_CLUSTERED_GROUP_BY[$$44] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$44]) [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 []<-[$$44, $$45] <- test.ds1.ds1_age.query-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/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..e537bf3
--- /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: 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 ([$$42]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$42] <- [{"age": $$SK0, "cnt": $$47}] [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 [$$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: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- SORT_GROUP_BY[$$49] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.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: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- PRE_CLUSTERED_GROUP_BY[$$44] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$44]) [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 []<-[$$44, $$45, $$46] <- test.ds1.ds1_age_dept.query-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/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..24aa758
--- /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: 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 ([$$42]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$42] <- [{"age": $$SK1, "cnt": $$47}] [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 [$$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: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- SORT_GROUP_BY[$$49] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.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: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- SORT_GROUP_BY[$$45] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$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|
+ data-scan []<-[$$44, $$45, $$46] <- test.ds1.ds1_dept_age.query-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/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..f2046b8
--- /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: 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 ([$$53]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$53] <- [{"age": $$SK0, "dept": $$SK1, "cnt": $$59}] [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 [$$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: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- SORT_GROUP_BY[$$61, $$62] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.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: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- PRE_CLUSTERED_GROUP_BY[$$56, $$57] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$56, $$57]) [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, $$57, $$58] <- test.ds1.ds1_age_dept.query-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/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..c69e8a0
--- /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 [$$17(ASC), $$18(ASC) ] |PARTITIONED|
+ select (eq($$l.getField(10), "1994-01-20"))
+ -- STREAM_SELECT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ unnest-map [$$17, $$18, $$l] <- index-search("LineItem", 0, "tpch", "LineItem", false, false, 2, $$25, $$26, 2, $$25, $$26, true, true, true)
+ -- BTREE_SEARCH |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ order (ASC, $$25) (ASC, $$26)
+ -- STABLE_SORT [$$25(ASC), $$26(ASC)] |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$25, $$26])
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ unnest-map [$$24, $$25, $$26] <- index-search("lineitem_shipdateIx", 0, "tpch", "LineItem", false, false, 1, $$22, 1, $$23, true, true, true)
+ -- BTREE_SEARCH |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ assign [$$22, $$23] <- ["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..c69e8a0
--- /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 [$$17(ASC), $$18(ASC) ] |PARTITIONED|
+ select (eq($$l.getField(10), "1994-01-20"))
+ -- STREAM_SELECT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ unnest-map [$$17, $$18, $$l] <- index-search("LineItem", 0, "tpch", "LineItem", false, false, 2, $$25, $$26, 2, $$25, $$26, true, true, true)
+ -- BTREE_SEARCH |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ order (ASC, $$25) (ASC, $$26)
+ -- STABLE_SORT [$$25(ASC), $$26(ASC)] |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$25, $$26])
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ unnest-map [$$24, $$25, $$26] <- index-search("lineitem_shipdateIx", 0, "tpch", "LineItem", false, false, 1, $$22, 1, $$23, true, true, true)
+ -- BTREE_SEARCH |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ assign [$$22, $$23] <- ["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/select-exclude/complex-exclude/complex-exclude.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.10.adm
new file mode 100644
index 0000000..254828e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.10.adm
@@ -0,0 +1,4 @@
+{ "U1": { "user_id": 1, "best_friend": 2, "phones": [ { "kind": "MOBILE", "number": "222-222-2222" } ] }, "U2": { "user_id": 2, "best_friend": 1 } }
+{ "U1": { "user_id": 2, "best_friend": 1 }, "U2": { "user_id": 1, "best_friend": 2, "phones": [ { "kind": "MOBILE", "number": "222-222-2222" } ] } }
+{ "U1": { "user_id": 3, "best_friend": 1, "favorite_color": "Green" }, "U2": { "user_id": 1, "best_friend": 2, "phones": [ { "kind": "MOBILE", "number": "222-222-2222" } ] } }
+{ "U1": { "user_id": 4, "best_friend": null } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.11.adm
new file mode 100644
index 0000000..a177d29
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.11.adm
@@ -0,0 +1 @@
+{ "best_friend": 1 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.12.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.12.adm
new file mode 100644
index 0000000..eab8cee
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.12.adm
@@ -0,0 +1,4 @@
+{ "user_id": 1, "best_friend": 2 }
+{ "user_id": 2, "best_friend": 1 }
+{ "user_id": 3, "best_friend": 1 }
+{ "user_id": 4, "best_friend": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.13.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.13.adm
new file mode 100644
index 0000000..62782bc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.13.adm
@@ -0,0 +1,3 @@
+{ "A": { "b": 2, "EXCLUDE": 3 } }
+{ "EXCLUDE": 3 }
+{ "EXCLUDE": { "a": 1, "b": 2, "EXCLUDE": 3 } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.14.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.14.adm
new file mode 100644
index 0000000..d44fb9c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.14.adm
@@ -0,0 +1,2 @@
+{ "user_id": 1, "best_friend": 2, "bestFriends": [ { "user_id": 2, "best_friend": 1, "address": { "zip_code": "99929", "street": "2341 Apple Street" } } ] }
+{ "user_id": 2, "best_friend": 1, "bestFriends": [ { "user_id": 1, "best_friend": 2, "phones": [ { "kind": "MOBILE", "number": "222-222-2222" } ] } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.3.adm
new file mode 100644
index 0000000..f72a7e5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.3.adm
@@ -0,0 +1,4 @@
+{ "user_id": 1, "best_friend": 2 }
+{ "user_id": 2, "best_friend": 1 }
+{ "user_id": 3, "best_friend": 1, "favorite_color": "Green" }
+{ "user_id": 4, "best_friend": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.4.adm
new file mode 100644
index 0000000..ae07f96
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.4.adm
@@ -0,0 +1,4 @@
+{ "user_id": 1, "best_friend": 2, "phones": [ { "kind": "MOBILE", "number": "222-222-2222" } ] }
+{ "user_id": 2, "best_friend": 1 }
+{ "user_id": 3, "best_friend": 1, "favorite_color": "Green" }
+{ "user_id": 4, "best_friend": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.5.adm
new file mode 100644
index 0000000..16d3516
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.5.adm
@@ -0,0 +1,2 @@
+[ { "user_id": 2, "best_friend": 1 } ]
+[ { "user_id": 3, "best_friend": 1, "favorite_color": "Green" } ]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.6.adm
new file mode 100644
index 0000000..bf58f6a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.6.adm
@@ -0,0 +1,4 @@
+{ "user_id": 1 }
+{ "user_id": 2 }
+{ "user_id": 3 }
+{ "user_id": 4 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.7.adm
new file mode 100644
index 0000000..71d7a13
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.7.adm
@@ -0,0 +1,3 @@
+{ "best_friend": 1 }
+{ "best_friend": 2 }
+{ "best_friend": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.8.adm
new file mode 100644
index 0000000..3c5ba18
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.8.adm
@@ -0,0 +1 @@
+{ "user_id": 1, "best_friend": 2 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.9.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.9.adm
new file mode 100644
index 0000000..c8cbba5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.9.adm
@@ -0,0 +1,4 @@
+{ "U": { "user_id": 1, "best_friend": 2 }, "miscInfo": { "24as": "23412" } }
+{ "U": { "user_id": 2, "best_friend": 1 }, "miscInfo": { "24as": "23412" } }
+{ "U": { "user_id": 3, "best_friend": 1, "favorite_color": "Green" }, "miscInfo": { "24as": "23412" } }
+{ "U": { "user_id": 4, "best_friend": null }, "miscInfo": { "24as": "23412" } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/tiny-social/tiny-social.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/tiny-social/tiny-social.3.adm
new file mode 100644
index 0000000..c27da48
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/tiny-social/tiny-social.3.adm
@@ -0,0 +1,12 @@
+{ "user": { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("47.44,80.65"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)" }
+{ "user": { "screen-name": "ColineGeyer@63", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("29.15,76.53"), "referred-topics": {{ "verizon", "voice-clarity" }}, "message-text": " hate verizon its voice-clarity is OMG:(" }
+{ "user": { "screen-name": "NilaMilliron_tw", "friends_count": 445, "statuses_count": 164, "name": "Nila Milliron", "followers_count": 22649 }, "sender-location": point("37.59,68.42"), "referred-topics": {{ "iphone", "platform" }}, "message-text": " can't stand iphone its platform is terrible" }
+{ "user": { "screen-name": "OliJackson_512", "friends_count": 445, "statuses_count": 164, "name": "Oli Jackson", "followers_count": 22649 }, "sender-location": point("24.82,94.63"), "referred-topics": {{ "samsung", "voice-command" }}, "message-text": " like samsung the voice-command is amazing:)" }
+{ "user": { "screen-name": "ColineGeyer@63", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("32.84,67.14"), "referred-topics": {{ "verizon", "shortcut-menu" }}, "message-text": " like verizon its shortcut-menu is awesome:)" }
+{ "user": { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("29.72,75.8"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " like motorola the speed is good:)" }
+{ "user": { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("39.28,70.48"), "referred-topics": {{ "sprint", "voice-command" }}, "message-text": " like sprint the voice-command is mind-blowing:)" }
+{ "user": { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("40.09,92.69"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " can't stand motorola its speed is terrible:(" }
+{ "user": { "screen-name": "ColineGeyer@63", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("47.51,83.99"), "referred-topics": {{ "iphone", "voice-clarity" }}, "message-text": " like iphone the voice-clarity is good:)" }
+{ "user": { "screen-name": "ChangEwing_573", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136 }, "sender-location": point("36.21,72.6"), "referred-topics": {{ "samsung", "platform" }}, "message-text": " like samsung the platform is good" }
+{ "user": { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("46.05,93.34"), "referred-topics": {{ "t-mobile", "shortcut-menu" }}, "message-text": " like t-mobile the shortcut-menu is awesome:)" }
+{ "user": { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("36.86,74.62"), "referred-topics": {{ "verizon", "voicemail-service" }}, "message-text": " love verizon its voicemail-service is awesome" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/tiny-social/tiny-social.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/tiny-social/tiny-social.4.adm
new file mode 100644
index 0000000..0b3ee31
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/tiny-social/tiny-social.4.adm
@@ -0,0 +1,11 @@
+{ "TM": { "tweetid": "1", "user": { "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:00.000"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)" } }
+{ "TM": { "tweetid": "10", "user": { "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("29.15,76.53"), "send-time": datetime("2008-01-26T10:10:00.000"), "referred-topics": {{ "verizon", "voice-clarity" }}, "message-text": " hate verizon its voice-clarity is OMG:(" } }
+{ "TM": { "tweetid": "11", "user": { "lang": "en", "friends_count": 445, "statuses_count": 164, "name": "Nila Milliron", "followers_count": 22649 }, "sender-location": point("37.59,68.42"), "send-time": datetime("2008-03-09T10:10:00.000"), "referred-topics": {{ "iphone", "platform" }}, "message-text": " can't stand iphone its platform is terrible" } }
+{ "TM": { "tweetid": "2", "user": { "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("32.84,67.14"), "send-time": datetime("2010-05-13T10:10:00.000"), "referred-topics": {{ "verizon", "shortcut-menu" }}, "message-text": " like verizon its shortcut-menu is awesome:)" } }
+{ "TM": { "tweetid": "3", "user": { "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("29.72,75.8"), "send-time": datetime("2006-11-04T10:10:00.000"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " like motorola the speed is good:)" } }
+{ "TM": { "tweetid": "4", "user": { "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("39.28,70.48"), "send-time": datetime("2011-12-26T10:10:00.000"), "referred-topics": {{ "sprint", "voice-command" }}, "message-text": " like sprint the voice-command is mind-blowing:)" } }
+{ "TM": { "tweetid": "5", "user": { "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("40.09,92.69"), "send-time": datetime("2006-08-04T10:10:00.000"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " can't stand motorola its speed is terrible:(" } }
+{ "TM": { "tweetid": "6", "user": { "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("47.51,83.99"), "send-time": datetime("2010-05-07T10:10:00.000"), "referred-topics": {{ "iphone", "voice-clarity" }}, "message-text": " like iphone the voice-clarity is good:)" } }
+{ "TM": { "tweetid": "7", "user": { "lang": "en", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136 }, "sender-location": point("36.21,72.6"), "send-time": datetime("2011-08-25T10:10:00.000"), "referred-topics": {{ "samsung", "platform" }}, "message-text": " like samsung the platform is good" } }
+{ "TM": { "tweetid": "8", "user": { "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("46.05,93.34"), "send-time": datetime("2005-10-14T10:10:00.000"), "referred-topics": {{ "t-mobile", "shortcut-menu" }}, "message-text": " like t-mobile the shortcut-menu is awesome:)" } }
+{ "TM": { "tweetid": "9", "user": { "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("36.86,74.62"), "send-time": datetime("2012-07-21T10:10:00.000"), "referred-topics": {{ "verizon", "voicemail-service" }}, "message-text": " love verizon its voicemail-service is awesome" } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/tiny-social/tiny-social.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/tiny-social/tiny-social.5.adm
new file mode 100644
index 0000000..0621911
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/tiny-social/tiny-social.5.adm
@@ -0,0 +1,24 @@
+{ "tweetid": "1", "user": { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("47.44,80.65"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)" }
+{ "tweetid": "1", "user": { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("47.44,80.65"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)", "send-time": datetime("2008-04-26T10:10:00.000") }
+{ "tweetid": "10", "user": { "screen-name": "ColineGeyer@63", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("29.15,76.53"), "referred-topics": {{ "verizon", "voice-clarity" }}, "message-text": " hate verizon its voice-clarity is OMG:(" }
+{ "tweetid": "10", "user": { "screen-name": "ColineGeyer@63", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("29.15,76.53"), "referred-topics": {{ "verizon", "voice-clarity" }}, "message-text": " hate verizon its voice-clarity is OMG:(", "send-time": datetime("2008-01-26T10:10:00.000") }
+{ "tweetid": "11", "user": { "screen-name": "NilaMilliron_tw", "friends_count": 445, "statuses_count": 164, "name": "Nila Milliron", "followers_count": 22649 }, "sender-location": point("37.59,68.42"), "referred-topics": {{ "iphone", "platform" }}, "message-text": " can't stand iphone its platform is terrible" }
+{ "tweetid": "11", "user": { "screen-name": "NilaMilliron_tw", "friends_count": 445, "statuses_count": 164, "name": "Nila Milliron", "followers_count": 22649 }, "sender-location": point("37.59,68.42"), "referred-topics": {{ "iphone", "platform" }}, "message-text": " can't stand iphone its platform is terrible", "send-time": datetime("2008-03-09T10:10:00.000") }
+{ "tweetid": "12", "user": { "screen-name": "OliJackson_512", "friends_count": 445, "statuses_count": 164, "name": "Oli Jackson", "followers_count": 22649 }, "sender-location": point("24.82,94.63"), "referred-topics": {{ "samsung", "voice-command" }}, "message-text": " like samsung the voice-command is amazing:)" }
+{ "tweetid": "12", "user": { "screen-name": "OliJackson_512", "friends_count": 445, "statuses_count": 164, "name": "Oli Jackson", "followers_count": 22649 }, "sender-location": point("24.82,94.63"), "referred-topics": {{ "samsung", "voice-command" }}, "message-text": " like samsung the voice-command is amazing:)", "send-time": datetime("2010-02-13T10:10:00.000") }
+{ "tweetid": "2", "user": { "screen-name": "ColineGeyer@63", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("32.84,67.14"), "referred-topics": {{ "verizon", "shortcut-menu" }}, "message-text": " like verizon its shortcut-menu is awesome:)" }
+{ "tweetid": "2", "user": { "screen-name": "ColineGeyer@63", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("32.84,67.14"), "referred-topics": {{ "verizon", "shortcut-menu" }}, "message-text": " like verizon its shortcut-menu is awesome:)", "send-time": datetime("2010-05-13T10:10:00.000") }
+{ "tweetid": "3", "user": { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("29.72,75.8"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " like motorola the speed is good:)" }
+{ "tweetid": "3", "user": { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("29.72,75.8"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " like motorola the speed is good:)", "send-time": datetime("2006-11-04T10:10:00.000") }
+{ "tweetid": "4", "user": { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("39.28,70.48"), "referred-topics": {{ "sprint", "voice-command" }}, "message-text": " like sprint the voice-command is mind-blowing:)" }
+{ "tweetid": "4", "user": { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("39.28,70.48"), "referred-topics": {{ "sprint", "voice-command" }}, "message-text": " like sprint the voice-command is mind-blowing:)", "send-time": datetime("2011-12-26T10:10:00.000") }
+{ "tweetid": "5", "user": { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("40.09,92.69"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " can't stand motorola its speed is terrible:(" }
+{ "tweetid": "5", "user": { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("40.09,92.69"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " can't stand motorola its speed is terrible:(", "send-time": datetime("2006-08-04T10:10:00.000") }
+{ "tweetid": "6", "user": { "screen-name": "ColineGeyer@63", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("47.51,83.99"), "referred-topics": {{ "iphone", "voice-clarity" }}, "message-text": " like iphone the voice-clarity is good:)" }
+{ "tweetid": "6", "user": { "screen-name": "ColineGeyer@63", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("47.51,83.99"), "referred-topics": {{ "iphone", "voice-clarity" }}, "message-text": " like iphone the voice-clarity is good:)", "send-time": datetime("2010-05-07T10:10:00.000") }
+{ "tweetid": "7", "user": { "screen-name": "ChangEwing_573", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136 }, "sender-location": point("36.21,72.6"), "referred-topics": {{ "samsung", "platform" }}, "message-text": " like samsung the platform is good" }
+{ "tweetid": "7", "user": { "screen-name": "ChangEwing_573", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136 }, "sender-location": point("36.21,72.6"), "referred-topics": {{ "samsung", "platform" }}, "message-text": " like samsung the platform is good", "send-time": datetime("2011-08-25T10:10:00.000") }
+{ "tweetid": "8", "user": { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("46.05,93.34"), "referred-topics": {{ "t-mobile", "shortcut-menu" }}, "message-text": " like t-mobile the shortcut-menu is awesome:)" }
+{ "tweetid": "8", "user": { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("46.05,93.34"), "referred-topics": {{ "t-mobile", "shortcut-menu" }}, "message-text": " like t-mobile the shortcut-menu is awesome:)", "send-time": datetime("2005-10-14T10:10:00.000") }
+{ "tweetid": "9", "user": { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("36.86,74.62"), "referred-topics": {{ "verizon", "voicemail-service" }}, "message-text": " love verizon its voicemail-service is awesome" }
+{ "tweetid": "9", "user": { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("36.86,74.62"), "referred-topics": {{ "verizon", "voicemail-service" }}, "message-text": " love verizon its voicemail-service is awesome", "send-time": datetime("2012-07-21T10:10:00.000") }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/tiny-social/tiny-social.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/tiny-social/tiny-social.6.adm
new file mode 100644
index 0000000..81592e9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/tiny-social/tiny-social.6.adm
@@ -0,0 +1,12 @@
+{ "TM": { "user": { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("47.44,80.65"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)" } }
+{ "TM": { "user": { "screen-name": "ColineGeyer@63", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("29.15,76.53"), "referred-topics": {{ "verizon", "voice-clarity" }}, "message-text": " hate verizon its voice-clarity is OMG:(" } }
+{ "TM": { "user": { "screen-name": "NilaMilliron_tw", "friends_count": 445, "statuses_count": 164, "name": "Nila Milliron", "followers_count": 22649 }, "sender-location": point("37.59,68.42"), "referred-topics": {{ "iphone", "platform" }}, "message-text": " can't stand iphone its platform is terrible" } }
+{ "TM": { "user": { "screen-name": "OliJackson_512", "friends_count": 445, "statuses_count": 164, "name": "Oli Jackson", "followers_count": 22649 }, "sender-location": point("24.82,94.63"), "referred-topics": {{ "samsung", "voice-command" }}, "message-text": " like samsung the voice-command is amazing:)" } }
+{ "TM": { "user": { "screen-name": "ColineGeyer@63", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("32.84,67.14"), "referred-topics": {{ "verizon", "shortcut-menu" }}, "message-text": " like verizon its shortcut-menu is awesome:)" } }
+{ "TM": { "user": { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("29.72,75.8"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " like motorola the speed is good:)" } }
+{ "TM": { "user": { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("39.28,70.48"), "referred-topics": {{ "sprint", "voice-command" }}, "message-text": " like sprint the voice-command is mind-blowing:)" } }
+{ "TM": { "user": { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("40.09,92.69"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " can't stand motorola its speed is terrible:(" } }
+{ "TM": { "user": { "screen-name": "ColineGeyer@63", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("47.51,83.99"), "referred-topics": {{ "iphone", "voice-clarity" }}, "message-text": " like iphone the voice-clarity is good:)" } }
+{ "TM": { "user": { "screen-name": "ChangEwing_573", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136 }, "sender-location": point("36.21,72.6"), "referred-topics": {{ "samsung", "platform" }}, "message-text": " like samsung the platform is good" } }
+{ "TM": { "user": { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("46.05,93.34"), "referred-topics": {{ "t-mobile", "shortcut-menu" }}, "message-text": " like t-mobile the shortcut-menu is awesome:)" } }
+{ "TM": { "user": { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("36.86,74.62"), "referred-topics": {{ "verizon", "voicemail-service" }}, "message-text": " love verizon its voicemail-service is awesome" } }
\ 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.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/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/union/union_type_cast/union_type_cast.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/union/union_type_cast/union_type_cast.3.adm
new file mode 100644
index 0000000..cefb946
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/union/union_type_cast/union_type_cast.3.adm
@@ -0,0 +1,40 @@
+{ "ds": "message1", "id": 1, "no_in_response_to": { "message-id": 1, "author-id": 3, "sender-location": point("47.16,77.75"), "message": " love sprint its shortcut-menu is awesome:)" } }
+{ "ds": "message2", "id": 1, "no_author_id": { "message-id": 1, "in-response-to": 2, "sender-location": point("47.16,77.75"), "message": " love sprint its shortcut-menu is awesome:)" } }
+{ "ds": "user", "id": 1, "user": { "id": 1, "alias": "Margarita", "name": "MargaritaStoddard", "user-since": datetime("2012-08-20T10:10:00.000"), "friend-ids": {{ 2, 3, 6, 10 }}, "employment": [ { "organization-name": "Codetechno", "start-date": date("2006-08-06") } ] } }
+{ "ds": "message1", "id": 2, "no_in_response_to": { "message-id": 2, "author-id": 1, "sender-location": point("41.66,80.87"), "message": " dislike iphone its touch-screen is horrible" } }
+{ "ds": "message2", "id": 2, "no_author_id": { "message-id": 2, "in-response-to": 4, "sender-location": point("41.66,80.87"), "message": " dislike iphone its touch-screen is horrible" } }
+{ "ds": "user", "id": 2, "user": { "id": 2, "alias": "Isbel", "name": "IsbelDull", "user-since": datetime("2011-01-22T10:10:00.000"), "friend-ids": {{ 1, 4 }}, "employment": [ { "organization-name": "Hexviafind", "start-date": date("2010-04-27") } ] } }
+{ "ds": "message1", "id": 3, "no_in_response_to": { "message-id": 3, "author-id": 2, "sender-location": point("48.09,81.01"), "message": " like samsung the plan is amazing" } }
+{ "ds": "message2", "id": 3, "no_author_id": { "message-id": 3, "in-response-to": 4, "sender-location": point("48.09,81.01"), "message": " like samsung the plan is amazing" } }
+{ "ds": "user", "id": 3, "user": { "id": 3, "alias": "Emory", "name": "EmoryUnk", "user-since": datetime("2012-07-10T10:10:00.000"), "friend-ids": {{ 1, 5, 8, 9 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2010-06-17"), "end-date": date("2010-01-26") } ] } }
+{ "ds": "message1", "id": 4, "no_in_response_to": { "message-id": 4, "author-id": 1, "sender-location": point("37.73,97.04"), "message": " can't stand at&t the network is horrible:(" } }
+{ "ds": "message2", "id": 4, "no_author_id": { "message-id": 4, "in-response-to": 2, "sender-location": point("37.73,97.04"), "message": " can't stand at&t the network is horrible:(" } }
+{ "ds": "user", "id": 4, "user": { "id": 4, "alias": "Nicholas", "name": "NicholasStroh", "user-since": datetime("2010-12-27T10:10:00.000"), "friend-ids": {{ 2 }}, "employment": [ { "organization-name": "Zamcorporation", "start-date": date("2010-06-08") } ] } }
+{ "ds": "message1", "id": 5, "no_in_response_to": { "message-id": 5, "author-id": 6, "sender-location": point("34.7,90.76"), "message": " love sprint the customization is mind-blowing" } }
+{ "ds": "message2", "id": 5, "no_author_id": { "message-id": 5, "in-response-to": 2, "sender-location": point("34.7,90.76"), "message": " love sprint the customization is mind-blowing" } }
+{ "ds": "user", "id": 5, "user": { "id": 5, "alias": "Von", "name": "VonKemble", "user-since": datetime("2010-01-05T10:10:00.000"), "friend-ids": {{ 3, 6, 10 }}, "employment": [ { "organization-name": "Kongreen", "start-date": date("2010-11-27") } ] } }
+{ "ds": "message1", "id": 6, "no_in_response_to": { "message-id": 6, "author-id": 2, "sender-location": point("31.5,75.56"), "message": " like t-mobile its platform is mind-blowing" } }
+{ "ds": "message2", "id": 6, "no_author_id": { "message-id": 6, "in-response-to": 1, "sender-location": point("31.5,75.56"), "message": " like t-mobile its platform is mind-blowing" } }
+{ "ds": "user", "id": 6, "user": { "id": 6, "alias": "Willis", "name": "WillisWynne", "user-since": datetime("2005-01-17T10:10:00.000"), "friend-ids": {{ 1, 3, 7 }}, "employment": [ { "organization-name": "jaydax", "start-date": date("2009-05-15") } ] } }
+{ "ds": "message1", "id": 7, "no_in_response_to": { "message-id": 7, "author-id": 5, "sender-location": point("32.91,85.05"), "message": " dislike sprint the speed is horrible" } }
+{ "ds": "message2", "id": 7, "no_author_id": { "message-id": 7, "in-response-to": 15, "sender-location": point("32.91,85.05"), "message": " dislike sprint the speed is horrible" } }
+{ "ds": "user", "id": 7, "user": { "id": 7, "alias": "Suzanna", "name": "SuzannaTillson", "user-since": datetime("2012-08-07T10:10:00.000"), "friend-ids": {{ 6 }}, "employment": [ { "organization-name": "Labzatron", "start-date": date("2011-04-19") } ] } }
+{ "ds": "message1", "id": 8, "no_in_response_to": { "message-id": 8, "author-id": 1, "sender-location": point("40.33,80.87"), "message": " like verizon the 3G is awesome:)" } }
+{ "ds": "message2", "id": 8, "no_author_id": { "message-id": 8, "in-response-to": 11, "sender-location": point("40.33,80.87"), "message": " like verizon the 3G is awesome:)" } }
+{ "ds": "user", "id": 8, "user": { "id": 8, "alias": "Nila", "name": "NilaMilliron", "user-since": datetime("2008-01-01T10:10:00.000"), "friend-ids": {{ 3 }}, "employment": [ { "organization-name": "Plexlane", "start-date": date("2010-02-28") } ] } }
+{ "ds": "message1", "id": 9, "no_in_response_to": { "message-id": 9, "author-id": 3, "sender-location": point("34.45,96.48"), "message": " love verizon its wireless is good" } }
+{ "ds": "message2", "id": 9, "no_author_id": { "message-id": 9, "in-response-to": 12, "sender-location": point("34.45,96.48"), "message": " love verizon its wireless is good" } }
+{ "ds": "user", "id": 9, "user": { "id": 9, "alias": "Woodrow", "name": "WoodrowNehling", "user-since": datetime("2005-09-20T10:10:00.000"), "friend-ids": {{ 3, 10 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2003-04-22"), "end-date": date("2009-12-13") } ] } }
+{ "ds": "message1", "id": 10, "no_in_response_to": { "message-id": 10, "author-id": 1, "sender-location": point("42.5,70.01"), "message": " can't stand motorola the touch-screen is terrible" } }
+{ "ds": "message2", "id": 10, "no_author_id": { "message-id": 10, "in-response-to": 12, "sender-location": point("42.5,70.01"), "message": " can't stand motorola the touch-screen is terrible" } }
+{ "ds": "user", "id": 10, "user": { "id": 10, "alias": "Bram", "name": "BramHatch", "user-since": datetime("2010-10-16T10:10:00.000"), "friend-ids": {{ 1, 5, 9 }}, "employment": [ { "organization-name": "physcane", "start-date": date("2007-06-05"), "end-date": date("2011-11-05") } ] } }
+{ "ds": "message1", "id": 11, "no_in_response_to": { "message-id": 11, "author-id": 1, "sender-location": point("38.97,77.49"), "message": " can't stand at&t its plan is terrible" } }
+{ "ds": "message2", "id": 11, "no_author_id": { "message-id": 11, "in-response-to": 1, "sender-location": point("38.97,77.49"), "message": " can't stand at&t its plan is terrible" } }
+{ "ds": "message1", "id": 12, "no_in_response_to": { "message-id": 12, "author-id": 10, "sender-location": point("42.26,77.76"), "message": " can't stand t-mobile its voicemail-service is OMG:(" } }
+{ "ds": "message2", "id": 12, "no_author_id": { "message-id": 12, "in-response-to": 6, "sender-location": point("42.26,77.76"), "message": " can't stand t-mobile its voicemail-service is OMG:(" } }
+{ "ds": "message1", "id": 13, "no_in_response_to": { "message-id": 13, "author-id": 10, "sender-location": point("42.77,78.92"), "message": " dislike iphone the voice-command is bad:(" } }
+{ "ds": "message2", "id": 13, "no_author_id": { "message-id": 13, "in-response-to": 4, "sender-location": point("42.77,78.92"), "message": " dislike iphone the voice-command is bad:(" } }
+{ "ds": "message1", "id": 14, "no_in_response_to": { "message-id": 14, "author-id": 9, "sender-location": point("41.33,85.28"), "message": " love at&t its 3G is good:)" } }
+{ "ds": "message2", "id": 14, "no_author_id": { "message-id": 14, "in-response-to": 12, "sender-location": point("41.33,85.28"), "message": " love at&t its 3G is good:)" } }
+{ "ds": "message1", "id": 15, "no_in_response_to": { "message-id": 15, "author-id": 7, "sender-location": point("44.47,67.11"), "message": " like iphone the voicemail-service is awesome" } }
+{ "ds": "message2", "id": 15, "no_author_id": { "message-id": 15, "in-response-to": 11, "sender-location": point("44.47,67.11"), "message": " like iphone the voicemail-service is awesome" } }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/union/union_type_cast/union_type_cast.4.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/union/union_type_cast/union_type_cast.4.plan
new file mode 100644
index 0000000..d0aa765
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/union/union_type_cast/union_type_cast.4.plan
@@ -0,0 +1,72 @@
+distribute result [$#1] [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 ([$#1]) [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 [$$102(ASC) ] |PARTITIONED|
+ order (ASC, $$102) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STABLE_SORT [$$102(ASC)] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ union ($$64, $$140, $#1) ($$103, $$70, $$102) [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 ($$139, $$141, $$64) ($$68, $$69, $$103) [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|
+ project ([$$139, $$68]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$139] <- [cast({"id": $$68, "no_in_response_to": object-remove($$s, "in-response-to")})] [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|
+ replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- REPLICATE |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$68, $$s] <- TinySocial.FacebookMessages [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|
+ project ([$$141, $$69]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$141] <- [cast({"id": $$69, "user": $$t})] [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 []<-[$$69, $$t] <- TinySocial.FacebookUsers [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|
+ project ([$$140, $$70]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$140] <- [cast({"id": $$70, "no_author_id": object-remove($$s, "author-id")})] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$70, $$s]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$70, $$s] <- [$$68, $$s] [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|
+ replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- REPLICATE |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$68, $$s] <- TinySocial.FacebookMessages [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/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..48db536
--- /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: 0.0]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ project ([$$91]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |UNPARTITIONED|
+ assign [$$91] <- [{"id": $$106, "review": $$111}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ order (ASC, $$106) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- STABLE_SORT [$$106(ASC)] |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ limit 3 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_LIMIT |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ project ([$$111, $$106]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$106] <- [int64-default-null($$d.getField("id"))] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ limit 3 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_LIMIT |PARTITIONED|
+ assign [$$111] <- [string-default-null($$d.getField("review"))] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- 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: 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/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..0a33ec8
--- /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: 0.0]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ limit 3 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_LIMIT |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ project ([$$67]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- 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: 0.0]
+ -- ASSIGN |PARTITIONED|
+ limit 3 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_LIMIT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$d] <- test.ExternalDataset limit 3 [cardinality: 1000000.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/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..9df5619
--- /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: 0.0]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ project ([$$85]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |UNPARTITIONED|
+ assign [$$85] <- [{"id": $$88, "review": $$92}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |UNPARTITIONED|
+ limit 3 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_LIMIT |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- SORT_MERGE_EXCHANGE [$$88(ASC) ] |PARTITIONED|
+ limit 3 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_LIMIT |PARTITIONED|
+ project ([$$88, $$92]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$92] <- [$$d.getField(1)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- 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: 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/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..eaf38d1
--- /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: 0.0]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ limit 3 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_LIMIT |UNPARTITIONED|
+ project ([$$61]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$61] <- [{"id": $$63, "review": $$67}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- SORT_MERGE_EXCHANGE [$$63(ASC) ] |PARTITIONED|
+ project ([$$63, $$67]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$67] <- [$$d.getField(1)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ limit 3 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_LIMIT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$63, $$d] <- test.DatasetWithKnownField limit 3 [cardinality: 1000000.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/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..9389f52
--- /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: 0.0]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ project ([$$91]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |UNPARTITIONED|
+ assign [$$91] <- [{"id": $$94, "review": $$98}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |UNPARTITIONED|
+ limit 3 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_LIMIT |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- SORT_MERGE_EXCHANGE [$$94(ASC) ] |PARTITIONED|
+ limit 3 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_LIMIT |PARTITIONED|
+ project ([$$94, $$98]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$98] <- [$$d.getField(1)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- 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: 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/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..b0427d0
--- /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: 0.0]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ limit 3 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_LIMIT |UNPARTITIONED|
+ project ([$$67]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$67] <- [{"id": $$69, "review": $$73}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- SORT_MERGE_EXCHANGE [$$69(ASC) ] |PARTITIONED|
+ project ([$$69, $$73]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$73] <- [$$d.getField(1)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ limit 3 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_LIMIT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$69, $$d] <- test.DatasetWithKnownField limit 3 [cardinality: 1000000.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/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/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/warnings-limit/warnings-limit.03.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/warnings-limit/warnings-limit.03.regexadm
index 519019a..7029ce2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/warnings-limit/warnings-limit.03.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/warnings-limit/warnings-limit.03.regexadm
@@ -31,6 +31,7 @@
\s*\Q"metrics": {\E
\s*\Q"elapsedTime": "\E[^"]+\Q",\E
\s*\Q"executionTime": "\E[^"]+\Q",\E
+\s*\Q"compileTime": "\E[^"]+\Q",\E
\s*\Q"resultCount": \E[0-9]+\Q,\E
\s*\Q"resultSize": \E[0-9]+\Q,\E
\s*\Q"processedObjects": \E[0-9]+\Q,\E
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/warnings-limit/warnings-limit.04.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/warnings-limit/warnings-limit.04.regexadm
index 8eb28ea..8478d17 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/warnings-limit/warnings-limit.04.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/warnings-limit/warnings-limit.04.regexadm
@@ -24,6 +24,7 @@
\s*\Q"metrics": {\E
\s*\Q"elapsedTime": "\E[^"]+\Q",\E
\s*\Q"executionTime": "\E[^"]+\Q",\E
+\s*\Q"compileTime": "\E[^"]+\Q",\E
\s*\Q"resultCount": \E[0-9]+\Q,\E
\s*\Q"resultSize": \E[0-9]+\Q,\E
\s*\Q"processedObjects": \E[0-9]+\Q,\E
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/warnings-limit/warnings-limit.05.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/warnings-limit/warnings-limit.05.regexadm
index 5c477e7..5399ed8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/warnings-limit/warnings-limit.05.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/warnings-limit/warnings-limit.05.regexadm
@@ -24,6 +24,7 @@
\s*\Q"metrics": {\E
\s*\Q"elapsedTime": "\E[^"]+\Q",\E
\s*\Q"executionTime": "\E[^"]+\Q",\E
+\s*\Q"compileTime": "\E[^"]+\Q",\E
\s*\Q"resultCount": \E[0-9]+\Q,\E
\s*\Q"resultSize": \E[0-9]+\Q,\E
\s*\Q"processedObjects": \E[0-9]+\Q,\E
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/warnings-limit/warnings-limit.06.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/warnings-limit/warnings-limit.06.regexadm
index 5eeb3df..aa58fb4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/warnings-limit/warnings-limit.06.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/warnings-limit/warnings-limit.06.regexadm
@@ -12,6 +12,7 @@
\s*\Q"metrics": {\E
\s*\Q"elapsedTime": "\E[^"]+\Q",\E
\s*\Q"executionTime": "\E[^"]+\Q",\E
+\s*\Q"compileTime": "\E[^"]+\Q",\E
\s*\Q"resultCount": \E[0-9]+\Q,\E
\s*\Q"resultSize": \E[0-9]+\Q,\E
\s*\Q"processedObjects": \E[0-9]+\Q,\E
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/warnings-limit/warnings-limit.07.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/warnings-limit/warnings-limit.07.regexadm
index 5c477e7..5399ed8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/warnings-limit/warnings-limit.07.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/warnings-limit/warnings-limit.07.regexadm
@@ -24,6 +24,7 @@
\s*\Q"metrics": {\E
\s*\Q"elapsedTime": "\E[^"]+\Q",\E
\s*\Q"executionTime": "\E[^"]+\Q",\E
+\s*\Q"compileTime": "\E[^"]+\Q",\E
\s*\Q"resultCount": \E[0-9]+\Q,\E
\s*\Q"resultSize": \E[0-9]+\Q,\E
\s*\Q"processedObjects": \E[0-9]+\Q,\E
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/warnings-limit/warnings-limit.08.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/warnings-limit/warnings-limit.08.regexadm
index 519019a..7029ce2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/warnings-limit/warnings-limit.08.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/warnings-limit/warnings-limit.08.regexadm
@@ -31,6 +31,7 @@
\s*\Q"metrics": {\E
\s*\Q"elapsedTime": "\E[^"]+\Q",\E
\s*\Q"executionTime": "\E[^"]+\Q",\E
+\s*\Q"compileTime": "\E[^"]+\Q",\E
\s*\Q"resultCount": \E[0-9]+\Q,\E
\s*\Q"resultSize": \E[0-9]+\Q,\E
\s*\Q"processedObjects": \E[0-9]+\Q,\E
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/testsuite_external_dataset_s3.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_external_dataset_s3.xml
index 12a8ae2..7242984 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_external_dataset_s3.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_external_dataset_s3.xml
@@ -172,6 +172,14 @@
<expected-warn>The provided external dataset configuration returned no files from the external source</expected-warn>
</compilation-unit>
</test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="common/parquet/invalid-parquet-files">
+ <placeholder name="adapter" value="S3" />
+ <output-dir compare="Text">none</output-dir>
+ <source-location>false</source-location>
+ <expected-error>20-records.json. Reason: not a Parquet file</expected-error>
+ </compilation-unit>
+ </test-case>
<test-case FilePath="external-dataset" check-warnings="true">
<compilation-unit name="common/parquet/parquet-types/unset-flags">
<placeholder name="adapter" value="S3" />
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..284c2fd 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">
@@ -76,8 +68,8 @@
<test-case FilePath="external-library" check-warnings="true">
<compilation-unit name="crash">
<output-dir compare="Text">crash</output-dir>
- <expected-warn>ASX0201: External UDF returned exception. Returned exception was: Function externallibtest:crash#0 failed to execute (in line 23, at column 1)</expected-warn>
- <expected-warn>ASX0201: External UDF returned exception. Returned exception was: java.io.IOException: Python process exited with code: 1 (in line 23, at column 1)</expected-warn>
+ <expected-warn>ASX0201: External UDF returned exception.</expected-warn>
+ <expected-warn>ASX0201: External UDF returned exception.</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 1267995..b184eff 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -34,6 +34,11 @@
</compilation-unit>
</test-case>
<test-case FilePath="api">
+ <compilation-unit name="compileonly-2">
+ <output-dir compare="Text">compileonly-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="api">
<compilation-unit name="multiple-param-values">
<output-dir compare="Text">multiple-param-values</output-dir>
</compilation-unit>
@@ -3101,6 +3106,21 @@
<expected-error>Input contains different list types (in line 25, at column 8)</expected-error>
</compilation-unit>
</test-case>
+ <test-case FilePath="array_fun">
+ <compilation-unit name="array_binary_search">
+ <output-dir compare="Text">array_binary_search</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array_fun">
+ <compilation-unit name="array_move">
+ <output-dir compare="Text">array_move</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array_fun">
+ <compilation-unit name="array_swap">
+ <output-dir compare="Text">array_swap</output-dir>
+ </compilation-unit>
+ </test-case>
</test-group>
<test-group name="bitwise">
<test-case FilePath="bitwise">
@@ -3603,10 +3623,10 @@
<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: 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 24, at column 46)</expected-warn>
+ <expected-warn>Incomparable input types: bigint and string (in line 25, at column 46)</expected-warn>
</compilation-unit>
</test-case>
</test-group>
@@ -4120,6 +4140,11 @@
-->
</test-group>
<test-group name="ddl">
+ <test-case FilePath="ddl">
+ <compilation-unit name="analyze-dataset-1">
+ <output-dir compare="Text">analyze-dataset-1</output-dir>
+ </compilation-unit>
+ </test-case>
<test-case FilePath="ddl/create-index">
<compilation-unit name="create-index-1">
<output-dir compare="Text">create-index-1</output-dir>
@@ -6637,6 +6662,11 @@
</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="hash_join_array">
<output-dir compare="Text">hash_join_array</output-dir>
</compilation-unit>
@@ -6656,6 +6686,16 @@
<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">
@@ -7207,6 +7247,11 @@
</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>
@@ -7243,6 +7288,11 @@
<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-group>
<test-group name="multipart-dataverse">
<test-case FilePath="multipart-dataverse">
@@ -8165,6 +8215,18 @@
<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">
@@ -8549,6 +8611,11 @@
<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">
@@ -8581,6 +8648,11 @@
<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">
@@ -9663,6 +9735,22 @@
</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">
@@ -9887,6 +9975,25 @@
</compilation-unit>
</test-case>
</test-group>
+ <test-group name="select-exclude">
+ <test-case FilePath="select-exclude">
+ <compilation-unit name="tiny-social">
+ <output-dir compare="Text">tiny-social</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="select-exclude">
+ <compilation-unit name="complex-exclude">
+ <output-dir compare="Text">complex-exclude</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="select-exclude">
+ <compilation-unit name="exclude-negative">
+ <output-dir compare="Text">exclude-negative</output-dir>
+ <expected-error>ASX1001: Syntax error</expected-error>
+ <expected-error>ASX1001: Syntax error</expected-error>
+ </compilation-unit>
+ </test-case>
+ </test-group>
<test-group name="semistructured">
<test-case FilePath="semistructured">
<compilation-unit name="count-nullable">
@@ -10354,6 +10461,11 @@
<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">
@@ -13512,6 +13624,11 @@
<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">
@@ -14951,6 +15068,11 @@
<output-dir compare="Text">query-ASTERIXDB-1354</output-dir>
</compilation-unit>
</test-case>
+ <test-case FilePath="union">
+ <compilation-unit name="union_type_cast">
+ <output-dir compare="Text">union_type_cast</output-dir>
+ </compilation-unit>
+ </test-case>
</test-group>
<test-group name="upsert">
<test-case FilePath="upsert">
@@ -15350,13 +15472,36 @@
</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="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. "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>
@@ -15377,12 +15522,12 @@
<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_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", "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>
+ <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">
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_batch_lookup.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_batch_lookup.xml
new file mode 100644
index 0000000..fd8b886
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_batch_lookup.xml
@@ -0,0 +1,862 @@
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<!--
+ ! Licensed to the Apache Software Foundation (ASF) under one
+ ! or more contributor license agreements. See the NOTICE file
+ ! distributed with this work for additional information
+ ! regarding copyright ownership. The ASF licenses this file
+ ! to you under the Apache License, Version 2.0 (the
+ ! "License"); you may not use this file except in compliance
+ ! with the License. You may obtain a copy of the License at
+ !
+ ! http://www.apache.org/licenses/LICENSE-2.0
+ !
+ ! Unless required by applicable law or agreed to in writing,
+ ! software distributed under the License is distributed on an
+ ! "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ ! KIND, either express or implied. See the License for the
+ ! specific language governing permissions and limitations
+ ! under the License.
+ !-->
+<test-suite xmlns="urn:xml.testframework.asterix.apache.org" ResultOffsetPath="results" QueryOffsetPath="queries_sqlpp" QueryFileExtension=".sqlpp">
+ <test-group name="batched-lookups">
+ <test-case FilePath="dml">
+ <compilation-unit name="compact-dataset-and-its-indexes">
+ <output-dir compare="Text">compact-dataset-and-its-indexes</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="using-constant-merge-policy">
+ <output-dir compare="Text">using-constant-merge-policy</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="using-prefix-merge-policy">
+ <output-dir compare="Text">using-prefix-merge-policy</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="using-concurrent-merge-policy">
+ <output-dir compare="Text">using-concurrent-merge-policy</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="using-correlated-prefix-merge-policy">
+ <output-dir compare="Text">using-correlated-prefix-merge-policy</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="using-correlated-prefix-merge-policy-with-feed">
+ <output-dir compare="Text">using-correlated-prefix-merge-policy</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="using-no-merge-policy">
+ <output-dir compare="Text">using-no-merge-policy</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="delete-from-loaded-dataset-with-index">
+ <output-dir compare="Text">delete-from-loaded-dataset-with-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="load-with-index">
+ <output-dir compare="Text">load-with-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-delete-btree-secondary-index-nullable">
+ <output-dir compare="Text">scan-delete-btree-secondary-index-nullable</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-delete-btree-correlated-secondary-index-nullable">
+ <output-dir compare="Text">scan-delete-btree-secondary-index-nullable</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-insert-btree-correlated-secondary-index-nullable">
+ <output-dir compare="Text">scan-delete-btree-secondary-index-nullable</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-delete-btree-secondary-index-open">
+ <output-dir compare="Text">scan-delete-btree-secondary-index-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-delete-btree-correlated-secondary-index-open">
+ <output-dir compare="Text">scan-delete-btree-secondary-index-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-delete-rtree-secondary-index-open">
+ <output-dir compare="Text">scan-delete-rtree-secondary-index-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-delete-rtree-correlated-secondary-index-open">
+ <output-dir compare="Text">scan-delete-rtree-secondary-index-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-insert-btree-correlated-secondary-index-open">
+ <output-dir compare="Text">scan-delete-btree-secondary-index-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-insert-rtree-correlated-secondary-index-open">
+ <output-dir compare="Text">scan-insert-rtree-secondary-index-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="fulltext">
+ <compilation-unit name="fulltext-index-01">
+ <output-dir compare="Text">fulltext-index-01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="fulltext">
+ <compilation-unit name="fulltext-index-02">
+ <output-dir compare="Text">fulltext-index-02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="fulltext">
+ <compilation-unit name="fulltext-index-08">
+ <output-dir compare="Text">fulltext-index-08</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="fulltext">
+ <compilation-unit name="fulltext-index-09">
+ <output-dir compare="Text">fulltext-index-09</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="fulltext">
+ <compilation-unit name="stopwords-full-text-filter-1">
+ <output-dir compare="Text">stopwords-full-text-filter-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-join">
+ <compilation-unit name="btree-index-nested-loop-join">
+ <output-dir compare="Text">btree-index-nested-loop-join</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-join">
+ <compilation-unit name="btree-pidx-to-sidx-idxonly-equi-join_01">
+ <output-dir compare="Text">btree-pidx-to-sidx-idxonly-equi-join_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-join">
+ <compilation-unit name="btree-secondary-equi-join_02">
+ <output-dir compare="Text">btree-secondary-equi-join_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-join">
+ <compilation-unit name="btree-secondary-equi-join_03">
+ <output-dir compare="Text">btree-secondary-equi-join_03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-join">
+ <compilation-unit name="btree-secondary-equi-join_04">
+ <output-dir compare="Text">btree-secondary-equi-join_04</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-join">
+ <compilation-unit name="btree-sidx-idxonly-to-pidx-equi-join_01">
+ <output-dir compare="Text">btree-sidx-idxonly-to-pidx-equi-join_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-join">
+ <compilation-unit name="btree-sidx-idxonly-to-sidx-idxonly-equi-join_01">
+ <output-dir compare="Text">btree-sidx-idxonly-to-sidx-idxonly-equi-join_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-join">
+ <compilation-unit name="btree-sidx-non-idxonly-to-pidx-equi-join_01">
+ <output-dir compare="Text">btree-sidx-non-idxonly-to-pidx-equi-join_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-join">
+ <compilation-unit name="btree-sidx-non-idxonly-to-sidx-idxonly-equi-join_01">
+ <output-dir compare="Text">btree-sidx-non-idxonly-to-sidx-idxonly-equi-join_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-join" check-warnings="true">
+ <compilation-unit name="hints-indexnl-params">
+ <output-dir compare="Text">hints-indexnl-params</output-dir>
+ <expected-warn>ASX1132: Invalid specification for hint indexnl. ASX1001: Syntax error: In line 1 >>(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_05">
+ <output-dir compare="Text">rtree-spatial-intersect-point_05</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="btree-index-composite-key-03">
+ <output-dir compare="Text">btree-index-composite-key-03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="btree-index-composite-key-mixed-intervals">
+ <output-dir compare="Text">btree-index-composite-key-mixed-intervals</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="btree-index-rewrite-multiple">
+ <output-dir compare="Text">btree-index-rewrite-multiple</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="btree-sidx-composite-idxonly-01">
+ <output-dir compare="Text">btree-sidx-composite-idxonly-01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="btree-sidx-composite-idxonly-02">
+ <output-dir compare="Text">btree-sidx-composite-idxonly-02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="btree-sidx-composite-idxonly-03">
+ <output-dir compare="Text">btree-sidx-composite-idxonly-03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="btree-sidx-composite-idxonly-04">
+ <output-dir compare="Text">btree-sidx-composite-idxonly-04</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="btree-sidx-idxonly-01">
+ <output-dir compare="Text">btree-sidx-idxonly-01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="cust-index-age-nullable">
+ <output-dir compare="Text">cust-index-age-nullable</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="intersection-with-between">
+ <output-dir compare="Text">intersection-with-between</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="inverted-index-ngram-edit-distance-large-data">
+ <output-dir compare="Text">inverted-index-ngram-edit-distance-large-data</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="inverted-index-olist-edit-distance">
+ <output-dir compare="Text">inverted-index-olist-edit-distance</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="inverted-index-olist-jaccard">
+ <output-dir compare="Text">inverted-index-olist-jaccard</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="inverted-index-ulist-jaccard">
+ <output-dir compare="Text">inverted-index-ulist-jaccard</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="orders-index-custkey-conjunctive-open">
+ <output-dir compare="Text">orders-index-custkey-conjunctive-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="orders-index-custkey-conjunctive">
+ <output-dir compare="Text">orders-index-custkey-conjunctive</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="orders-index-custkey-open">
+ <output-dir compare="Text">orders-index-custkey-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="orders-index-custkey">
+ <output-dir compare="Text">orders-index-custkey</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="range-search-open">
+ <output-dir compare="Text">range-search-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="range-search">
+ <output-dir compare="Text">range-search</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="rtree-sidx-non-idxonly-02">
+ <output-dir compare="Text">rtree-sidx-non-idxonly-02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="dataset-with-meta">
+ <output-dir compare="Text">dataset-with-meta</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection" check-warnings="true">
+ <compilation-unit name="hints-skip-index">
+ <output-dir compare="Text">hints-skip-index</output-dir>
+ <expected-warn>ASX1132: Invalid specification for hint skip-index. ASX1001: Syntax error: In line 1 >>(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>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>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="verify">
+ <output-dir compare="Text">verify</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-enforced/index-join">
+ <compilation-unit name="ngram-jaccard">
+ <output-dir compare="Text">ngram-jaccard</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-enforced/index-join">
+ <compilation-unit name="ngram-jaccard-inline">
+ <output-dir compare="Text">ngram-jaccard-inline</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-enforced/index-selection">
+ <compilation-unit name="btree-index-composite-key-mixed-intervals">
+ <output-dir compare="Text">btree-index-composite-key-mixed-intervals</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-enforced/index-selection">
+ <compilation-unit name="btree-index-rewrite-multiple">
+ <output-dir compare="Text">btree-index-rewrite-multiple</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-enforced/index-selection">
+ <compilation-unit name="orders-index-custkey">
+ <output-dir compare="Text">orders-index-custkey</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-enforced/index-selection">
+ <compilation-unit name="orders-index-custkey-conjunctive">
+ <output-dir compare="Text">orders-index-custkey-conjunctive</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-enforced/index-selection">
+ <compilation-unit name="range-search">
+ <output-dir compare="Text">range-search</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-non-enforced/index-selection">
+ <compilation-unit name="btree-index-01">
+ <output-dir compare="Text">btree-index-01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-non-enforced/index-selection">
+ <compilation-unit name="btree-index-02">
+ <output-dir compare="Text">btree-index-02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-non-enforced/index-selection">
+ <compilation-unit name="btree-index-04">
+ <output-dir compare="Text">btree-index-04</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-non-enforced/correlated-index-selection">
+ <compilation-unit name="btree-index-01">
+ <output-dir compare="Text">btree-index-01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-non-enforced/index-join">
+ <compilation-unit name="btree-equi-join-01">
+ <output-dir compare="Text">btree-equi-join-01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-join">
+ <compilation-unit name="ngram-jaccard">
+ <output-dir compare="Text">ngram-jaccard</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-join">
+ <compilation-unit name="ngram-jaccard-inline">
+ <output-dir compare="Text">ngram-jaccard-inline</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-join">
+ <compilation-unit name="btree-secondary-non-enforced-equi-join">
+ <output-dir compare="Text">btree-secondary-non-enforced-equi-join</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-join">
+ <compilation-unit name="btree-secondary-non-enforced-equi-join-2">
+ <output-dir compare="Text">btree-secondary-non-enforced-equi-join-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-selection">
+ <compilation-unit name="btree-index-composite-key-mixed-intervals">
+ <output-dir compare="Text">btree-index-composite-key-mixed-intervals</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-selection">
+ <compilation-unit name="btree-index-rewrite-multiple">
+ <output-dir compare="Text">btree-index-rewrite-multiple</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-selection">
+ <compilation-unit name="orders-index-custkey">
+ <output-dir compare="Text">orders-index-custkey</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-selection">
+ <compilation-unit name="orders-index-custkey-conjunctive">
+ <output-dir compare="Text">orders-index-custkey-conjunctive</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-selection">
+ <compilation-unit name="range-search">
+ <output-dir compare="Text">range-search</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-selection">
+ <compilation-unit name="non-enforced-01">
+ <output-dir compare="Text">non-enforced-01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-selection">
+ <compilation-unit name="non-enforced-02">
+ <output-dir compare="Text">non-enforced-02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-selection">
+ <compilation-unit name="non-enforced-04">
+ <output-dir compare="Text">non-enforced-04</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="btree-index-composite-key-mixed-intervals">
+ <output-dir compare="Text">btree-index-composite-key-mixed-intervals</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="btree-index-rewrite-multiple">
+ <output-dir compare="Text">btree-index-rewrite-multiple</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="cust-index-age-nullable">
+ <output-dir compare="Text">cust-index-age-nullable</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="inverted-index-olist-edit-distance">
+ <output-dir compare="Text">inverted-index-olist-edit-distance</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="inverted-index-olist-jaccard">
+ <output-dir compare="Text">inverted-index-olist-jaccard</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="inverted-index-ulist-jaccard">
+ <output-dir compare="Text">inverted-index-ulist-jaccard</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="orders-index-custkey">
+ <output-dir compare="Text">orders-index-custkey</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="orders-index-custkey-conjunctive">
+ <output-dir compare="Text">orders-index-custkey-conjunctive</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="orders-index-custkey-conjunctive-open">
+ <output-dir compare="Text">orders-index-custkey-conjunctive-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="orders-index-custkey-open">
+ <output-dir compare="Text">orders-index-custkey-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="range-search">
+ <output-dir compare="Text">range-search</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="range-search-open">
+ <output-dir compare="Text">range-search-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="rtree-secondary-index-optional">
+ <output-dir compare="Text">rtree-secondary-index-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index-dml">
+ <compilation-unit name="compact-dataset-and-its-indexes">
+ <output-dir compare="Text">compact-dataset-and-its-indexes</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index-dml">
+ <compilation-unit name="delete-from-loaded-dataset-with-index">
+ <output-dir compare="Text">delete-from-loaded-dataset-with-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index-dml">
+ <compilation-unit name="insert-into-loaded-dataset-with-index_01">
+ <output-dir compare="Text">insert-into-loaded-dataset-with-index_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index-dml">
+ <compilation-unit name="insert-into-loaded-dataset-with-index_02">
+ <output-dir compare="Text">insert-into-loaded-dataset-with-index_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index-dml">
+ <compilation-unit name="load-with-ngram-index">
+ <output-dir compare="Text">load-with-ngram-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index-dml">
+ <compilation-unit name="load-with-rtree-index">
+ <output-dir compare="Text">load-with-rtree-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index-dml">
+ <compilation-unit name="load-with-word-index">
+ <output-dir compare="Text">load-with-word-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index-dml">
+ <compilation-unit name="scan-delete-btree-secondary-index-nullable">
+ <output-dir compare="Text">scan-delete-btree-secondary-index-nullable</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index-dml">
+ <compilation-unit name="scan-delete-rtree-secondary-index-nullable">
+ <output-dir compare="Text">scan-delete-rtree-secondary-index-nullable</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index-dml">
+ <compilation-unit name="scan-delete-rtree-secondary-index">
+ <output-dir compare="Text">scan-delete-rtree-secondary-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/closed">
+ <compilation-unit name="use-case-1">
+ <output-dir compare="Text">use-case-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/open">
+ <compilation-unit name="use-case-1">
+ <output-dir compare="Text">use-case-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/closed">
+ <compilation-unit name="use-case-2">
+ <output-dir compare="Text">use-case-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/open">
+ <compilation-unit name="use-case-2">
+ <output-dir compare="Text">use-case-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/closed">
+ <compilation-unit name="use-case-3">
+ <output-dir compare="Text">use-case-3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/open">
+ <compilation-unit name="use-case-3">
+ <output-dir compare="Text">use-case-3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/closed">
+ <compilation-unit name="use-case-4">
+ <output-dir compare="Text">use-case-4</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/open">
+ <compilation-unit name="use-case-4">
+ <output-dir compare="Text">use-case-4</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/closed">
+ <compilation-unit name="with-3-level-record-path">
+ <output-dir compare="Text">with-3-level-record-path</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/open">
+ <compilation-unit name="with-3-level-record-path">
+ <output-dir compare="Text">with-3-level-record-path</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/closed">
+ <compilation-unit name="with-composite-sk">
+ <output-dir compare="Text">with-composite-sk</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/closed">
+ <compilation-unit name="with-composite-pk">
+ <output-dir compare="Text">with-composite-pk</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/closed">
+ <compilation-unit name="with-filter-fields">
+ <output-dir compare="Text">with-filter-fields</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/open">
+ <compilation-unit name="using-feed-new-index">
+ <output-dir compare="Text">using-feed</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-quantified-queries">
+ <compilation-unit name="use-case-1">
+ <output-dir compare="Text">use-case-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-quantified-queries">
+ <compilation-unit name="use-case-2">
+ <output-dir compare="Text">use-case-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-quantified-queries">
+ <compilation-unit name="use-case-3">
+ <output-dir compare="Text">use-case-3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-quantified-queries">
+ <compilation-unit name="with-composite-pk">
+ <output-dir compare="Text">with-composite-pk</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-quantified-queries">
+ <compilation-unit name="with-open-index">
+ <output-dir compare="Text">with-open-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="spatial">
+ <compilation-unit name="cell-aggregation-with-filtering">
+ <output-dir compare="Text">cell-aggregation-with-filtering</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="sql-compat">
+ <compilation-unit name="outer_join_01">
+ <output-dir compare="Text">outer_join_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="statement-params">
+ <compilation-unit name="index_01">
+ <output-dir compare="Text">index_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q45">
+ <output-dir compare="Text">q45</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="q10_returned_item">
+ <output-dir compare="Text">q10_returned_item</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="q10_returned_item_int64">
+ <output-dir compare="Text">q10_returned_item_int64</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="q12_shipping">
+ <output-dir compare="Text">q12_shipping</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="q14_promotion_effect">
+ <output-dir compare="Text">q14_promotion_effect</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="q15_top_supplier">
+ <output-dir compare="Text">q15_top_supplier</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="q01_pricing_summary_report_nt">
+ <output-dir compare="Text">q01_pricing_summary_report_nt</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="q03_shipping_priority_nt">
+ <output-dir compare="Text">q03_shipping_priority_nt</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="q04_order_priority">
+ <output-dir compare="Text">q04_order_priority</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="q04_order_priority_with_nodegroup">
+ <output-dir compare="Text">q04_order_priority</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="q05_local_supplier_volume">
+ <output-dir compare="Text">q05_local_supplier_volume</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="q06_forecast_revenue_change">
+ <output-dir compare="Text">q06_forecast_revenue_change</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="q07_volume_shipping">
+ <output-dir compare="Text">q07_volume_shipping</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="q08_national_market_share">
+ <output-dir compare="Text">q08_national_market_share</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="query-issue810">
+ <output-dir compare="Text">query-issue810</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="query-issue810-2">
+ <output-dir compare="Text">query-issue810-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="query-issue810-3">
+ <output-dir compare="Text">query-issue810-3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="query-issue827-2">
+ <output-dir compare="Text">query-issue827-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-leftouterjoin">
+ <compilation-unit name="probe-pidx-with-join-btree-sidx1">
+ <output-dir compare="Text">probe-pidx-with-join-btree-sidx1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-leftouterjoin">
+ <compilation-unit name="probe-pidx-with-join-btree-sidx2">
+ <output-dir compare="Text">probe-pidx-with-join-btree-sidx2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-leftouterjoin">
+ <compilation-unit name="probe-pidx-join-btree-sidx3-idxonly">
+ <output-dir compare="Text">probe-pidx-join-btree-sidx3-idxonly</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-leftouterjoin">
+ <compilation-unit name="probe-pidx-with-join-rtree-sidx2">
+ <output-dir compare="Text">probe-pidx-with-join-rtree-sidx2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-leftouterjoin">
+ <compilation-unit name="probe-pidx-with-join-invidx-sidx1">
+ <output-dir compare="Text">probe-pidx-with-join-invidx-sidx1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-leftouterjoin">
+ <compilation-unit name="probe-pidx-with-join-invidx-sidx2">
+ <output-dir compare="Text">probe-pidx-with-join-invidx-sidx2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-leftouterjoin">
+ <compilation-unit name="probe-sidx-btree-idxonly-join-btree-pidx1">
+ <output-dir compare="Text">probe-sidx-btree-idxonly-join-btree-pidx1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-leftouterjoin">
+ <compilation-unit name="probe-sidx-btree-idxonly-join-btree-sidx1-idxonly">
+ <output-dir compare="Text">probe-sidx-btree-idxonly-join-btree-sidx1-idxonly</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-leftouterjoin">
+ <compilation-unit name="probe-sidx-btree-non-idxonly-join-btree-pidx1">
+ <output-dir compare="Text">probe-sidx-btree-non-idxonly-join-btree-pidx1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-leftouterjoin">
+ <compilation-unit name="probe-sidx-btree-non-idxonly-join-btree-sidx1-idxonly">
+ <output-dir compare="Text">probe-sidx-btree-non-idxonly-join-btree-sidx1-idxonly</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tinysocial">
+ <compilation-unit name="tinysocial-suite">
+ <output-dir compare="Text">tinysocial-suite</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tinysocial">
+ <compilation-unit name="tinysocial-suite-open">
+ <output-dir compare="Text">tinysocial-suite-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="filters">
+ <compilation-unit name="load-with-secondary-btree">
+ <output-dir compare="Text">load-with-secondary-btree</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="filters">
+ <compilation-unit name="load-with-secondary-btree-index-only">
+ <output-dir compare="Text">load-with-secondary-btree-index-only</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="filters">
+ <compilation-unit name="load-with-secondary-inverted-word">
+ <output-dir compare="Text">load-with-secondary-inverted-word</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="filters">
+ <compilation-unit name="load-with-secondary-rtree">
+ <output-dir compare="Text">load-with-secondary-rtree</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="filters">
+ <compilation-unit name="insert-with-correlated-secondary-btree">
+ <output-dir compare="Text">insert-with-secondary-btree</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="union">
+ <compilation-unit name="union_opt_1">
+ <output-dir compare="Text">union_opt_1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="upsert">
+ <compilation-unit name="primary-correlated-secondary-btree">
+ <output-dir compare="Text">primary-secondary-btree</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="limit">
+ <compilation-unit name="push-limit-to-primary-lookup">
+ <output-dir compare="Text">push-limit-to-primary-lookup</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="limit">
+ <compilation-unit name="push-limit-to-primary-lookup-select">
+ <output-dir compare="Text">push-limit-to-primary-lookup-select</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+</test-suite>
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_profiled.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_profiled.xml
new file mode 100644
index 0000000..c22d6d0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_profiled.xml
@@ -0,0 +1,15880 @@
+<!--
+ ! Licensed to the Apache Software Foundation (ASF) under one
+ ! or more contributor license agreements. See the NOTICE file
+ ! distributed with this work for additional information
+ ! regarding copyright ownership. The ASF licenses this file
+ ! to you under the Apache License, Version 2.0 (the
+ ! "License"); you may not use this file except in compliance
+ ! with the License. You may obtain a copy of the License at
+ !
+ ! http://www.apache.org/licenses/LICENSE-2.0
+ !
+ ! Unless required by applicable law or agreed to in writing,
+ ! software distributed under the License is distributed on an
+ ! "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ ! KIND, either express or implied. See the License for the
+ ! specific language governing permissions and limitations
+ ! under the License.
+ !-->
+<!DOCTYPE test-suite [
+ <!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">
+ <!ENTITY TemporalQueries SYSTEM "queries_sqlpp/temporal/TemporalQueries.xml">
+ ]>
+<test-suite xmlns="urn:xml.testframework.asterix.apache.org" ResultOffsetPath="results" QueryOffsetPath="queries_sqlpp" QueryFileExtension=".sqlpp" SourceLocation="true">
+ &ObjectsQueries;
+ &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-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 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/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="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-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", "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="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-suite>
diff --git a/asterixdb/asterix-column/pom.xml b/asterixdb/asterix-column/pom.xml
new file mode 100644
index 0000000..ef87c91
--- /dev/null
+++ b/asterixdb/asterix-column/pom.xml
@@ -0,0 +1,151 @@
+<!--
+ ! Licensed to the Apache Software Foundation (ASF) under one
+ ! or more contributor license agreements. See the NOTICE file
+ ! distributed with this work for additional information
+ ! regarding copyright ownership. The ASF licenses this file
+ ! to you under the Apache License, Version 2.0 (the
+ ! "License"); you may not use this file except in compliance
+ ! with the License. You may obtain a copy of the License at
+ !
+ ! http://www.apache.org/licenses/LICENSE-2.0
+ !
+ ! Unless required by applicable law or agreed to in writing,
+ ! software distributed under the License is distributed on an
+ ! "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ ! KIND, either express or implied. See the License for the
+ ! specific language governing permissions and limitations
+ ! under the License.
+ !-->
+<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/xsd/maven-4.0.0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+ <parent>
+ <artifactId>apache-asterixdb</artifactId>
+ <groupId>org.apache.asterix</groupId>
+ <version>0.9.9-SNAPSHOT</version>
+ </parent>
+ <artifactId>asterix-column</artifactId>
+
+ <licenses>
+ <license>
+ <name>Apache License, Version 2.0</name>
+ <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
+ <distribution>repo</distribution>
+ <comments>A business-friendly OSS license</comments>
+ </license>
+ </licenses>
+
+ <properties>
+ <root.dir>${basedir}/..</root.dir>
+ </properties>
+
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.rat</groupId>
+ <artifactId>apache-rat-plugin</artifactId>
+ <executions>
+ <execution>
+ <id>default</id>
+ <phase>validate</phase>
+ <goals>
+ <goal>check</goal>
+ </goals>
+ <configuration>
+ <licenses>
+ <license implementation="org.apache.rat.analysis.license.ApacheSoftwareLicense20"/>
+ </licenses>
+ <excludes combine.children="append">
+ <exclude>src/test/resources/result/**</exclude>
+ </excludes>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ </plugins>
+ </build>
+
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.asterix</groupId>
+ <artifactId>asterix-common</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.asterix</groupId>
+ <artifactId>asterix-om</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.asterix</groupId>
+ <artifactId>asterix-runtime</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.asterix</groupId>
+ <artifactId>asterix-external-data</artifactId>
+ <version>${project.version}</version>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hyracks</groupId>
+ <artifactId>hyracks-util</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hyracks</groupId>
+ <artifactId>hyracks-data-std</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hyracks</groupId>
+ <artifactId>hyracks-dataflow-common</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hyracks</groupId>
+ <artifactId>hyracks-storage-am-lsm-btree-column</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hyracks</groupId>
+ <artifactId>algebricks-data</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.commons</groupId>
+ <artifactId>commons-lang3</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hyracks</groupId>
+ <artifactId>hyracks-storage-am-common</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hyracks</groupId>
+ <artifactId>hyracks-api</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hyracks</groupId>
+ <artifactId>hyracks-storage-common</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>com.fasterxml.jackson.core</groupId>
+ <artifactId>jackson-databind</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>it.unimi.dsi</groupId>
+ <artifactId>fastutil</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.parquet</groupId>
+ <artifactId>parquet-column</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.parquet</groupId>
+ <artifactId>parquet-common</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.parquet</groupId>
+ <artifactId>parquet-encoding</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>junit</groupId>
+ <artifactId>junit</artifactId>
+ <scope>test</scope>
+ </dependency>
+ </dependencies>
+</project>
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/AbstractNestedValueAssembler.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/AbstractNestedValueAssembler.java
new file mode 100644
index 0000000..1a4c3ef
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/AbstractNestedValueAssembler.java
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.column.assembler;
+
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+
+abstract class AbstractNestedValueAssembler extends AbstractValueAssembler {
+ protected final ArrayBackedValueStorage storage;
+
+ AbstractNestedValueAssembler(int level, AssemblerInfo info) {
+ super(level, info);
+ storage = new ArrayBackedValueStorage();
+ }
+
+ /**
+ * @return whether the nested assembler was started or not
+ */
+ final boolean isStarted() {
+ return started;
+ }
+
+ /**
+ * Add a nested value
+ *
+ * @param value contains the value and its information
+ */
+ abstract void addValue(AbstractValueAssembler value) throws HyracksDataException;
+
+ /**
+ * Add a nested {@link ATypeTag#NULL}
+ *
+ * @param value contains the value's information
+ */
+ abstract void addNull(AbstractValueAssembler value) throws HyracksDataException;
+
+ /**
+ * Add a nested {@link ATypeTag#MISSING}
+ */
+ void addMissing() throws HyracksDataException {
+ //By default, we ignore missing
+ }
+
+ @Override
+ final void addNullToAncestor(int nullLevel) throws HyracksDataException {
+ AbstractNestedValueAssembler parent = getParent();
+ if (nullLevel + 1 == level) {
+ parent.start();
+ parent.addNull(this);
+ return;
+ }
+ parent.addNullToAncestor(nullLevel);
+ }
+
+ @Override
+ final void addMissingToAncestor(int missingLevel) throws HyracksDataException {
+ AbstractNestedValueAssembler parent = getParent();
+ if (missingLevel + 1 == level) {
+ parent.start();
+ parent.addMissing();
+ return;
+ }
+ parent.addMissingToAncestor(missingLevel);
+ }
+
+ /**
+ * Recursively start the path of this assembler by staring all un-started parents
+ */
+ public final void start() {
+ if (started) {
+ return;
+ }
+ started = true;
+ reset();
+ AbstractNestedValueAssembler parent = getParent();
+ if (parent != null && !parent.isStarted()) {
+ parent.start();
+ }
+ }
+
+ /**
+ * End the assembler and add this nested value to its parent
+ */
+ public final void end() throws HyracksDataException {
+ if (started) {
+ addValueToParent();
+ started = false;
+ }
+
+ if (isDelegate()) {
+ getParent().end();
+ }
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/AbstractPrimitiveValueAssembler.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/AbstractPrimitiveValueAssembler.java
new file mode 100644
index 0000000..9f1809d
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/AbstractPrimitiveValueAssembler.java
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.column.assembler;
+
+import org.apache.asterix.column.assembler.value.IValueGetter;
+import org.apache.asterix.column.bytes.stream.in.AbstractBytesInputStream;
+import org.apache.asterix.column.values.IColumnValuesReader;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+
+public abstract class AbstractPrimitiveValueAssembler extends AbstractValueAssembler {
+ /**
+ * An indicator to go to the next value
+ */
+ public static final int NEXT_ASSEMBLER = -1;
+ protected final IValueGetter primitiveValueGetter;
+ protected final IColumnValuesReader reader;
+
+ AbstractPrimitiveValueAssembler(int level, AssemblerInfo info, IColumnValuesReader reader,
+ IValueGetter primitiveValueGetter) {
+ super(level, info);
+ this.primitiveValueGetter = primitiveValueGetter;
+ this.reader = reader;
+ }
+
+ public final void reset(AbstractBytesInputStream in, int startIndex, int numberOfTuples)
+ throws HyracksDataException {
+ reader.reset(in, numberOfTuples);
+ reader.skip(startIndex);
+ }
+
+ @Override
+ public final IValueReference getValue() throws HyracksDataException {
+ return primitiveValueGetter.getValue(reader);
+ }
+
+ @Override
+ void addNullToAncestor(int nullLevel) throws HyracksDataException {
+ AbstractNestedValueAssembler parent = getParent();
+ if (nullLevel + 1 == level) {
+ parent.start();
+ parent.addNull(this);
+ return;
+ }
+ parent.addNullToAncestor(nullLevel);
+ }
+
+ @Override
+ void addMissingToAncestor(int missingLevel) throws HyracksDataException {
+ AbstractNestedValueAssembler parent = getParent();
+ if (missingLevel + 1 == level) {
+ parent.start();
+ parent.addMissing();
+ return;
+ }
+ parent.addMissingToAncestor(missingLevel);
+ }
+
+ @Override
+ final void addValueToParent() throws HyracksDataException {
+ AbstractNestedValueAssembler parent = getParent();
+ parent.start();
+ getParent().addValue(this);
+ }
+
+ public final int getColumnIndex() {
+ return reader.getColumnIndex();
+ }
+
+ public final void skip(int count) throws HyracksDataException {
+ reader.skip(count);
+ }
+
+ /**
+ * Move to the next primitive value assembler
+ *
+ * @return the index of the next value
+ */
+ public abstract int next() throws HyracksDataException;
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/AbstractValueAssembler.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/AbstractValueAssembler.java
new file mode 100644
index 0000000..0071917
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/AbstractValueAssembler.java
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.column.assembler;
+
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+
+public abstract class AbstractValueAssembler {
+ protected static final VoidPointable NULL;
+ protected static final VoidPointable MISSING;
+ private final AbstractNestedValueAssembler parent;
+ private final IValueReference fieldName;
+ private final int fieldIndex;
+ private final boolean delegate;
+ protected final int level;
+ protected boolean started;
+
+ static {
+ NULL = new VoidPointable();
+ NULL.set(new byte[] { ATypeTag.NULL.serialize() }, 0, 1);
+
+ MISSING = new VoidPointable();
+ MISSING.set(new byte[] { ATypeTag.MISSING.serialize() }, 0, 1);
+ }
+
+ protected AbstractValueAssembler(int level, AssemblerInfo info) {
+ this.parent = info.getParent();
+ this.fieldName = info.getFieldName();
+ this.fieldIndex = info.getFieldIndex();
+ this.delegate = info.isDelegate();
+ this.level = level;
+ }
+
+ /**
+ * Add {@link ATypeTag#NULL} value to the ancestor at {@code nullLevel}
+ *
+ * @param nullLevel at what level the null occurred
+ */
+ abstract void addNullToAncestor(int nullLevel) throws HyracksDataException;
+
+ /**
+ * Add {@link ATypeTag#MISSING} value to the ancestor at {@code missingLevel}
+ *
+ * @param missingLevel at what level the missing occurred
+ */
+ abstract void addMissingToAncestor(int missingLevel) throws HyracksDataException;
+
+ /**
+ * Add the value of this assembler to its parent
+ */
+ abstract void addValueToParent() throws HyracksDataException;
+
+ /**
+ * @return the assembled value
+ */
+ public abstract IValueReference getValue() throws HyracksDataException;
+
+ /**
+ * Reset assembler
+ */
+ void reset() {
+ //NoOp
+ }
+
+ /**
+ * @return whether this assembler is the delegate (or representative) of its siblings
+ */
+ final boolean isDelegate() {
+ return delegate;
+ }
+
+ /**
+ * @return parent of the assembler
+ */
+ final AbstractNestedValueAssembler getParent() {
+ return parent;
+ }
+
+ /**
+ * Return the field name of the value of this assembler
+ */
+ final IValueReference getFieldName() {
+ return fieldName;
+ }
+
+ /**
+ * Return the field index of the value of this assembler (for closed types)
+ */
+ final int getFieldIndex() {
+ return fieldIndex;
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/ArrayValueAssembler.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/ArrayValueAssembler.java
new file mode 100644
index 0000000..2352e7f
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/ArrayValueAssembler.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.column.assembler;
+
+import org.apache.asterix.builders.IAsterixListBuilder;
+import org.apache.asterix.builders.ListBuilderFactory;
+import org.apache.asterix.om.types.AbstractCollectionType;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+
+public class ArrayValueAssembler extends AbstractNestedValueAssembler {
+ private final IAsterixListBuilder listBuilder;
+ private final AbstractCollectionType collectionType;
+ private final int firstValueIndex;
+
+ ArrayValueAssembler(int level, AssemblerInfo info, int firstValueIndex) {
+ super(level, info);
+ this.firstValueIndex = firstValueIndex;
+ collectionType = (AbstractCollectionType) info.getDeclaredType();
+ listBuilder = new ListBuilderFactory().create(collectionType.getTypeTag());
+ }
+
+ final int getFirstValueIndex() {
+ return firstValueIndex;
+ }
+
+ @Override
+ void reset() {
+ listBuilder.reset(collectionType);
+ storage.reset();
+ }
+
+ @Override
+ void addValue(AbstractValueAssembler value) throws HyracksDataException {
+ listBuilder.addItem(value.getValue());
+ }
+
+ @Override
+ void addNull(AbstractValueAssembler value) throws HyracksDataException {
+ listBuilder.addItem(NULL);
+ }
+
+ @Override
+ void addMissing() throws HyracksDataException {
+ listBuilder.addItem(MISSING);
+ }
+
+ @Override
+ void addValueToParent() throws HyracksDataException {
+ storage.reset();
+ listBuilder.write(storage.getDataOutput(), true);
+ getParent().addValue(this);
+ }
+
+ @Override
+ public IValueReference getValue() {
+ return storage;
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/ArrayWithUnionValueAssembler.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/ArrayWithUnionValueAssembler.java
new file mode 100644
index 0000000..dcd240b
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/ArrayWithUnionValueAssembler.java
@@ -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.
+ */
+package org.apache.asterix.column.assembler;
+
+import org.apache.asterix.column.metadata.schema.AbstractSchemaNode;
+import org.apache.asterix.column.metadata.schema.UnionSchemaNode;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class ArrayWithUnionValueAssembler extends ArrayValueAssembler {
+ private final int numberOfUnionChildren;
+ private int numberOfAddedValues;
+ private boolean nonMissingValueAdded;
+
+ ArrayWithUnionValueAssembler(int level, AssemblerInfo info, int firstValueIndex, AbstractSchemaNode itemNode) {
+ super(level, info, firstValueIndex);
+ this.numberOfUnionChildren = ((UnionSchemaNode) itemNode).getChildren().size();
+ }
+
+ @Override
+ void reset() {
+ numberOfAddedValues = 0;
+ nonMissingValueAdded = false;
+ super.reset();
+ }
+
+ @Override
+ void addValue(AbstractValueAssembler value) throws HyracksDataException {
+ nonMissingValueAdded = true;
+ numberOfAddedValues++;
+ super.addValue(value);
+ }
+
+ @Override
+ void addNull(AbstractValueAssembler value) throws HyracksDataException {
+ nonMissingValueAdded = true;
+ numberOfAddedValues++;
+ super.addNull(value);
+ }
+
+ @Override
+ void addMissing() throws HyracksDataException {
+ numberOfAddedValues++;
+ if (nonMissingValueAdded && numberOfAddedValues >= numberOfUnionChildren) {
+ nonMissingValueAdded = false;
+ numberOfAddedValues = numberOfAddedValues % numberOfUnionChildren;
+ } else if (numberOfAddedValues == numberOfUnionChildren) {
+ super.addMissing();
+ numberOfAddedValues = 0;
+ }
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/AssemblerInfo.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/AssemblerInfo.java
new file mode 100644
index 0000000..712e65c
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/AssemblerInfo.java
@@ -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.
+ */
+package org.apache.asterix.column.assembler;
+
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+
+public class AssemblerInfo {
+ private final AbstractNestedValueAssembler parent;
+ private final IAType declaredType;
+ private final boolean delegate;
+ private final IValueReference fieldName;
+ private final int fieldIndex;
+
+ public AssemblerInfo() {
+ this(BuiltinType.ANY, null, false);
+ }
+
+ public AssemblerInfo(IAType declaredType, EmptyAssembler parent) {
+ this(declaredType, parent, false);
+ }
+
+ public AssemblerInfo(IAType declaredType, AbstractNestedValueAssembler parent, boolean delegate) {
+ this(declaredType, parent, delegate, null, -1);
+ }
+
+ public AssemblerInfo(IAType declaredType, AbstractNestedValueAssembler parent, boolean delegate,
+ IValueReference fieldName) {
+ this(declaredType, parent, delegate, fieldName, -1);
+ }
+
+ public AssemblerInfo(IAType declaredType, AbstractNestedValueAssembler parent, boolean delegate, int fieldIndex) {
+ this(declaredType, parent, delegate, null, fieldIndex);
+ }
+
+ public AssemblerInfo(IAType declaredType, AbstractNestedValueAssembler parent, boolean delegate,
+ IValueReference fieldName, int fieldIndex) {
+ this(declaredType, parent, delegate, fieldName, fieldIndex, false);
+ }
+
+ public AssemblerInfo(IAType declaredType, AbstractNestedValueAssembler parent, boolean delegate,
+ IValueReference fieldName, int fieldIndex, boolean fieldNameTagged) {
+ this.parent = parent;
+ this.declaredType = declaredType;
+ this.delegate = delegate;
+ this.fieldName = fieldNameTagged ? fieldName : createTaggedFieldName(fieldName);
+ this.fieldIndex = fieldIndex;
+ }
+
+ private IValueReference createTaggedFieldName(IValueReference fieldName) {
+ if (fieldName == null) {
+ return null;
+ }
+ byte[] storage = new byte[1 + fieldName.getLength()];
+ storage[0] = ATypeTag.STRING.serialize();
+ System.arraycopy(fieldName.getByteArray(), fieldName.getStartOffset(), storage, 1, fieldName.getLength());
+ VoidPointable taggedFieldName = new VoidPointable();
+ taggedFieldName.set(storage, 0, storage.length);
+ return taggedFieldName;
+ }
+
+ public AbstractNestedValueAssembler getParent() {
+ return parent;
+ }
+
+ public IAType getDeclaredType() {
+ return declaredType;
+ }
+
+ public boolean isDelegate() {
+ return delegate;
+ }
+
+ public IValueReference getFieldName() {
+ return fieldName;
+ }
+
+ public int getFieldIndex() {
+ return fieldIndex;
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/EmptyAssembler.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/EmptyAssembler.java
new file mode 100644
index 0000000..406a401
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/EmptyAssembler.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.column.assembler;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+
+public class EmptyAssembler extends AbstractNestedValueAssembler {
+
+ EmptyAssembler() {
+ super(-1, new AssemblerInfo());
+ }
+
+ @Override
+ void addValue(AbstractValueAssembler value) throws HyracksDataException {
+ //noOp
+ }
+
+ @Override
+ void addValueToParent() throws HyracksDataException {
+ //noOp
+ }
+
+ @Override
+ void addNull(AbstractValueAssembler value) throws HyracksDataException {
+ //noOp
+ }
+
+ @Override
+ public IValueReference getValue() throws HyracksDataException {
+ return null;
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/ObjectValueAssembler.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/ObjectValueAssembler.java
new file mode 100644
index 0000000..536ce02
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/ObjectValueAssembler.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.asterix.column.assembler;
+
+import org.apache.asterix.builders.RecordBuilder;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+
+public class ObjectValueAssembler extends AbstractNestedValueAssembler {
+ private final RecordBuilder recordBuilder;
+ private final ARecordType recordType;
+
+ ObjectValueAssembler(int level, AssemblerInfo info) {
+ super(level, info);
+ recordBuilder = new RecordBuilder();
+ recordType = (ARecordType) info.getDeclaredType();
+ }
+
+ @Override
+ void reset() {
+ recordBuilder.reset(recordType);
+ storage.reset();
+ }
+
+ @Override
+ void addValue(AbstractValueAssembler value) throws HyracksDataException {
+ int valueIndex = value.getFieldIndex();
+ if (valueIndex >= 0) {
+ recordBuilder.addField(valueIndex, value.getValue());
+ } else {
+ recordBuilder.addField(value.getFieldName(), value.getValue());
+ }
+ }
+
+ @Override
+ void addNull(AbstractValueAssembler value) throws HyracksDataException {
+ int valueIndex = value.getFieldIndex();
+ if (valueIndex >= 0) {
+ recordBuilder.addField(valueIndex, NULL);
+ } else {
+ recordBuilder.addField(value.getFieldName(), NULL);
+ }
+ }
+
+ @Override
+ void addValueToParent() throws HyracksDataException {
+ storage.reset();
+ recordBuilder.write(storage.getDataOutput(), true);
+ getParent().addValue(this);
+ }
+
+ @Override
+ public IValueReference getValue() {
+ return storage;
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/PrimitiveValueAssembler.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/PrimitiveValueAssembler.java
new file mode 100644
index 0000000..9592a12
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/PrimitiveValueAssembler.java
@@ -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.
+ */
+package org.apache.asterix.column.assembler;
+
+import org.apache.asterix.column.assembler.value.IValueGetter;
+import org.apache.asterix.column.values.IColumnValuesReader;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class PrimitiveValueAssembler extends AbstractPrimitiveValueAssembler {
+
+ PrimitiveValueAssembler(int level, AssemblerInfo info, IColumnValuesReader reader, IValueGetter primitiveValue) {
+ super(level, info, reader, primitiveValue);
+ }
+
+ @Override
+ public int next() throws HyracksDataException {
+ if (!reader.next()) {
+ throw new IllegalAccessError("no more values");
+ } else if (reader.isNull() && (isDelegate() || reader.getLevel() + 1 == level)) {
+ addNullToAncestor(reader.getLevel());
+ } else if (reader.isValue()) {
+ addValueToParent();
+ }
+
+ if (isDelegate()) {
+ getParent().end();
+ }
+ //Go to next value
+ return -1;
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/RepeatedPrimitiveValueAssembler.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/RepeatedPrimitiveValueAssembler.java
new file mode 100644
index 0000000..8fa228f
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/RepeatedPrimitiveValueAssembler.java
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.column.assembler;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.asterix.column.assembler.value.IValueGetter;
+import org.apache.asterix.column.values.IColumnValuesReader;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+class RepeatedPrimitiveValueAssembler extends AbstractPrimitiveValueAssembler {
+ private final List<ArrayValueAssembler> arrays;
+
+ RepeatedPrimitiveValueAssembler(int level, AssemblerInfo info, IColumnValuesReader reader,
+ IValueGetter primitiveValue) {
+ super(level, info, reader, primitiveValue);
+ this.arrays = new ArrayList<>();
+ }
+
+ public void addArray(ArrayValueAssembler assembler) {
+ arrays.add(assembler);
+ }
+
+ @Override
+ public int next() throws HyracksDataException {
+ if (!reader.next()) {
+ throw new IllegalStateException("No more values");
+ } else if (reader.isNull() && (!arrays.isEmpty() || reader.getLevel() + 1 == level)) {
+ /*
+ * There are two cases here for where the null belongs to:
+ * 1- If the null is an array item, then add it
+ * 2- If the null is an ancestor, then we only add null if this column is the array delegate
+ * (i.e., !arrays.isEmpty())
+ */
+ addNullToAncestor(reader.getLevel());
+ } else if (reader.isMissing() && reader.getLevel() + 1 == level) {
+ /*
+ * Add a missing item
+ */
+ addMissingToAncestor(reader.getLevel());
+ } else if (reader.isValue()) {
+ addValueToParent();
+ }
+
+ if (isDelegate()) {
+ getParent().end();
+ }
+
+ //Initially, go to the next primitive assembler
+ int nextIndex = NEXT_ASSEMBLER;
+ if (!arrays.isEmpty()) {
+ /*
+ * This assembler is a delegate of a repeated group
+ * The delimiter index tells us that this assembler is responsible for a finished group
+ */
+ int delimiterIndex = reader.getDelimiterIndex();
+ if (delimiterIndex < arrays.size() && reader.isDelimiter()) {
+ //Also finish the next group
+ delimiterIndex++;
+ }
+
+ int numberOfFinishedGroups = Math.min(delimiterIndex, arrays.size());
+ for (int i = 0; i < numberOfFinishedGroups; i++) {
+ //I'm the delegate for this group of repeated values and the group(s) is finished
+ ArrayValueAssembler assembler = arrays.get(i);
+ assembler.end();
+ }
+
+ //Is the repeated group (determined by the delimiter index) still unfinished?
+ if (delimiterIndex < arrays.size()) {
+ //Yes, go to the first value of the unfinished repeated group
+ nextIndex = arrays.get(delimiterIndex).getFirstValueIndex();
+ }
+ }
+
+ //Go to next value
+ return nextIndex;
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/value/AbstractFixedLengthValueGetter.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/value/AbstractFixedLengthValueGetter.java
new file mode 100644
index 0000000..aeef686
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/value/AbstractFixedLengthValueGetter.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.column.assembler.value;
+
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+
+public abstract class AbstractFixedLengthValueGetter implements IValueGetter {
+ protected final VoidPointable value;
+
+ AbstractFixedLengthValueGetter(ATypeTag typeTag, int nonTaggedLength) {
+ //+1 for the type tag
+ byte[] storage = new byte[1 + nonTaggedLength];
+ storage[0] = typeTag.serialize();
+ value = new VoidPointable();
+ value.set(storage, 0, storage.length);
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/value/BooleanValueGetter.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/value/BooleanValueGetter.java
new file mode 100644
index 0000000..4a776ab
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/value/BooleanValueGetter.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.column.assembler.value;
+
+import org.apache.asterix.column.values.IColumnValuesReader;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.data.std.primitive.BooleanPointable;
+
+class BooleanValueGetter extends AbstractFixedLengthValueGetter {
+ BooleanValueGetter() {
+ super(ATypeTag.BOOLEAN, 1);
+ }
+
+ @Override
+ public IValueReference getValue(IColumnValuesReader reader) {
+ BooleanPointable.setBoolean(value.getByteArray(), value.getStartOffset() + 1, reader.getBoolean());
+ return value;
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/value/DoubleValueGetter.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/value/DoubleValueGetter.java
new file mode 100644
index 0000000..2e88896
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/value/DoubleValueGetter.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.column.assembler.value;
+
+import org.apache.asterix.column.values.IColumnValuesReader;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.data.std.primitive.DoublePointable;
+
+class DoubleValueGetter extends AbstractFixedLengthValueGetter {
+ DoubleValueGetter() {
+ super(ATypeTag.DOUBLE, Double.BYTES);
+ }
+
+ @Override
+ public IValueReference getValue(IColumnValuesReader reader) {
+ DoublePointable.setDouble(value.getByteArray(), value.getStartOffset() + 1, reader.getDouble());
+ return value;
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/value/IValueGetter.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/value/IValueGetter.java
new file mode 100644
index 0000000..9e58ab8
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/value/IValueGetter.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.column.assembler.value;
+
+import org.apache.asterix.column.values.IColumnValuesReader;
+import org.apache.hyracks.data.std.api.IValueReference;
+
+@FunctionalInterface
+public interface IValueGetter {
+ IValueReference getValue(IColumnValuesReader reader);
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/value/IValueGetterFactory.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/value/IValueGetterFactory.java
new file mode 100644
index 0000000..0b58cfc4
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/value/IValueGetterFactory.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.asterix.column.assembler.value;
+
+import org.apache.asterix.om.types.ATypeTag;
+
+@FunctionalInterface
+public interface IValueGetterFactory {
+ IValueGetter createValueGetter(ATypeTag typeTag);
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/value/LongValueGetter.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/value/LongValueGetter.java
new file mode 100644
index 0000000..e76e3c9
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/value/LongValueGetter.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.column.assembler.value;
+
+import org.apache.asterix.column.values.IColumnValuesReader;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.data.std.primitive.LongPointable;
+
+class LongValueGetter extends AbstractFixedLengthValueGetter {
+ LongValueGetter() {
+ super(ATypeTag.BIGINT, Long.BYTES);
+ }
+
+ @Override
+ public IValueReference getValue(IColumnValuesReader reader) {
+ LongPointable.setLong(value.getByteArray(), value.getStartOffset() + 1, reader.getLong());
+ return value;
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/value/MissingValueGetter.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/value/MissingValueGetter.java
new file mode 100644
index 0000000..1ae84ee
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/value/MissingValueGetter.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.column.assembler.value;
+
+import org.apache.asterix.column.values.IColumnValuesReader;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+
+public class MissingValueGetter implements IValueGetter {
+ public static final IValueGetter INSTANCE = new MissingValueGetter();
+ private static final VoidPointable MISSING;
+
+ static {
+ MISSING = new VoidPointable();
+ MISSING.set(new byte[] { ATypeTag.MISSING.serialize() }, 0, 1);
+ }
+
+ private MissingValueGetter() {
+ }
+
+ @Override
+ public IValueReference getValue(IColumnValuesReader reader) {
+ return MISSING;
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/value/NullValueGetter.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/value/NullValueGetter.java
new file mode 100644
index 0000000..e050252
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/value/NullValueGetter.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.column.assembler.value;
+
+import org.apache.asterix.column.values.IColumnValuesReader;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+
+public class NullValueGetter implements IValueGetter {
+ public static final IValueGetter INSTANCE = new NullValueGetter();
+ private static final VoidPointable NULL;
+
+ static {
+ NULL = new VoidPointable();
+ NULL.set(new byte[] { ATypeTag.NULL.serialize() }, 0, 1);
+ }
+
+ private NullValueGetter() {
+ }
+
+ @Override
+ public IValueReference getValue(IColumnValuesReader reader) {
+ return NULL;
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/value/StringValueGetter.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/value/StringValueGetter.java
new file mode 100644
index 0000000..1dd1aa7
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/value/StringValueGetter.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.column.assembler.value;
+
+import org.apache.asterix.column.values.IColumnValuesReader;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+
+class StringValueGetter implements IValueGetter {
+ private final ArrayBackedValueStorage value;
+
+ public StringValueGetter() {
+ value = new ArrayBackedValueStorage();
+ }
+
+ @Override
+ public IValueReference getValue(IColumnValuesReader reader) {
+ IValueReference string = reader.getBytes();
+ value.setSize(1 + string.getLength());
+ byte[] bytes = value.getByteArray();
+ bytes[0] = ATypeTag.STRING.serialize();
+ System.arraycopy(string.getByteArray(), string.getStartOffset(), bytes, 1, string.getLength());
+ return value;
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/value/UUIDValueGetter.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/value/UUIDValueGetter.java
new file mode 100644
index 0000000..135ed85
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/value/UUIDValueGetter.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.column.assembler.value;
+
+import org.apache.asterix.column.values.IColumnValuesReader;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.data.std.api.IValueReference;
+
+class UUIDValueGetter extends AbstractFixedLengthValueGetter {
+ UUIDValueGetter() {
+ super(ATypeTag.UUID, 16);
+ }
+
+ @Override
+ public IValueReference getValue(IColumnValuesReader reader) {
+ IValueReference uuid = reader.getBytes();
+ System.arraycopy(uuid.getByteArray(), uuid.getStartOffset(), value.getByteArray(), value.getStartOffset() + 1,
+ uuid.getLength());
+ return value;
+ }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/value/ValueGetterFactory.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/value/ValueGetterFactory.java
new file mode 100644
index 0000000..5f7fd7e
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/value/ValueGetterFactory.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.column.assembler.value;
+
+import org.apache.asterix.om.types.ATypeTag;
+
+public class ValueGetterFactory implements IValueGetterFactory {
+ public static final IValueGetterFactory INSTANCE = new ValueGetterFactory();
+
+ private ValueGetterFactory() {
+ }
+
+ @Override
+ public IValueGetter createValueGetter(ATypeTag typeTag) {
+ switch (typeTag) {
+ case NULL:
+ return NullValueGetter.INSTANCE;
+ case MISSING:
+ return MissingValueGetter.INSTANCE;
+ case BOOLEAN:
+ return new BooleanValueGetter();
+ case BIGINT:
+ return new LongValueGetter();
+ case DOUBLE:
+ return new DoubleValueGetter();
+ case STRING:
+ return new StringValueGetter();
+ case UUID:
+ return new UUIDValueGetter();
+ default:
+ throw new UnsupportedOperationException(typeTag + " is not supported");
+ }
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/ParquetDeltaBinaryPackingConfig.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/ParquetDeltaBinaryPackingConfig.java
new file mode 100644
index 0000000..f591d57
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/ParquetDeltaBinaryPackingConfig.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.column.bytes;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.bytes.BytesUtils;
+
+/**
+ * Copy of {@link org.apache.parquet.column.values.delta.DeltaBinaryPackingConfig}
+ */
+public class ParquetDeltaBinaryPackingConfig {
+ private int blockSizeInValues;
+ private int miniBlockNumInABlock;
+ private int miniBlockSizeInValues;
+
+ public ParquetDeltaBinaryPackingConfig(int blockSizeInValues, int miniBlockNumInABlock) {
+ reset(blockSizeInValues, miniBlockNumInABlock);
+ }
+
+ private void reset(int blockSizeInValues, int miniBlockNumInABlock) {
+ this.blockSizeInValues = blockSizeInValues;
+ this.miniBlockNumInABlock = miniBlockNumInABlock;
+ double miniSize = (double) blockSizeInValues / miniBlockNumInABlock;
+ Preconditions.checkArgument(miniSize % 8 == 0, "miniBlockSize must be multiple of 8, but it's " + miniSize);
+ this.miniBlockSizeInValues = (int) miniSize;
+ }
+
+ public static ParquetDeltaBinaryPackingConfig readConfig(InputStream in, ParquetDeltaBinaryPackingConfig config)
+ throws IOException {
+ final int blockSizeInValues = BytesUtils.readUnsignedVarInt(in);
+ final int miniBlockNumInABlock = BytesUtils.readUnsignedVarInt(in);
+ if (config == null) {
+ return new ParquetDeltaBinaryPackingConfig(blockSizeInValues, miniBlockNumInABlock);
+ }
+ config.reset(blockSizeInValues, miniBlockNumInABlock);
+ return config;
+ }
+
+ public BytesInput toBytesInput() {
+ return BytesInput.concat(BytesInput.fromUnsignedVarInt(blockSizeInValues),
+ BytesInput.fromUnsignedVarInt(miniBlockNumInABlock));
+ }
+
+ public int getBlockSizeInValues() {
+ return blockSizeInValues;
+ }
+
+ public int getMiniBlockNumInABlock() {
+ return miniBlockNumInABlock;
+ }
+
+ public int getMiniBlockSizeInValues() {
+ return miniBlockSizeInValues;
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/decoder/AbstractParquetValuesReader.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/decoder/AbstractParquetValuesReader.java
new file mode 100644
index 0000000..5f5b88c
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/decoder/AbstractParquetValuesReader.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.column.bytes.decoder;
+
+import java.io.IOException;
+
+import org.apache.asterix.column.bytes.stream.in.AbstractBytesInputStream;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.parquet.column.values.ValuesReader;
+
+/**
+ * Replaces {@link ValuesReader}
+ */
+public abstract class AbstractParquetValuesReader {
+ public abstract void initFromPage(AbstractBytesInputStream stream) throws IOException;
+
+ public abstract void skip();
+
+ public int readInteger() {
+ throw new UnsupportedOperationException(getClass().getName());
+ }
+
+ public long readLong() {
+ throw new UnsupportedOperationException(getClass().getName());
+ }
+
+ public double readDouble() {
+ throw new UnsupportedOperationException(getClass().getName());
+ }
+
+ public IValueReference readBytes() {
+ throw new UnsupportedOperationException(getClass().getName());
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/decoder/ParquetDeltaBinaryPackingValuesReader.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/decoder/ParquetDeltaBinaryPackingValuesReader.java
new file mode 100644
index 0000000..9aafa0f
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/decoder/ParquetDeltaBinaryPackingValuesReader.java
@@ -0,0 +1,216 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.column.bytes.decoder;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+import org.apache.asterix.column.bytes.ParquetDeltaBinaryPackingConfig;
+import org.apache.asterix.column.bytes.stream.in.AbstractBytesInputStream;
+import org.apache.parquet.bytes.BytesUtils;
+import org.apache.parquet.column.values.bitpacking.BytePackerForLong;
+import org.apache.parquet.column.values.bitpacking.Packer;
+import org.apache.parquet.column.values.delta.DeltaBinaryPackingValuesReader;
+import org.apache.parquet.io.ParquetDecodingException;
+
+/**
+ * Re-implementation of {@link DeltaBinaryPackingValuesReader}
+ */
+public class ParquetDeltaBinaryPackingValuesReader extends AbstractParquetValuesReader {
+ private int totalValueCount;
+ /**
+ * values read by the caller
+ */
+ private int valuesBufferedRead;
+ private int valuesRead;
+
+ /**
+ * stores the decoded values including the first value which is written to the header
+ */
+ private long[] valuesBuffer;
+ /**
+ * values loaded to the buffer, it could be bigger than the totalValueCount
+ * when data is not aligned to mini block, which means padding 0s are in the buffer
+ */
+ private int valuesBuffered;
+ private AbstractBytesInputStream in;
+ private ParquetDeltaBinaryPackingConfig config;
+ private int[] bitWidths;
+ private ByteBuffer bitWidthBuffer;
+ private long lastElement;
+
+ /**
+ * Loads one block at a time instead of eagerly loading all blocks in {@link DeltaBinaryPackingValuesReader}.
+ * This is to fix the {@link #valuesBuffer} size
+ */
+ @Override
+ public void initFromPage(AbstractBytesInputStream stream) throws IOException {
+ this.in = stream;
+ this.config = ParquetDeltaBinaryPackingConfig.readConfig(in, this.config);
+ this.totalValueCount = BytesUtils.readUnsignedVarInt(in);
+ allocateValuesBuffer();
+ bitWidths = allocate(bitWidths, config.getMiniBlockNumInABlock());
+ valuesBuffered = 0;
+
+ valuesBufferedRead = 0;
+ valuesRead = 0;
+
+ //read first value from header
+ valuesBuffer[valuesBuffered++] = BytesUtils.readZigZagVarLong(in);
+ lastElement = valuesBuffer[0];
+
+ if (valuesBuffered < totalValueCount) {
+ loadNewBlockToBuffer();
+ }
+ }
+
+ /**
+ * the value buffer is allocated so that the size of it is multiple of mini block
+ * because when writing, data is flushed on a mini block basis
+ */
+ private void allocateValuesBuffer() {
+ //+ 1 because first value written to header is also stored in values buffer
+ final int bufferSize = config.getMiniBlockSizeInValues() * config.getMiniBlockNumInABlock() + 1;
+ if (valuesBuffer == null || valuesBuffer.length < bufferSize) {
+ valuesBuffer = new long[bufferSize];
+ } else {
+ Arrays.fill(valuesBuffer, 0);
+ }
+ }
+
+ private int[] allocate(int[] array, int size) {
+ if (array == null || array.length < size) {
+ return new int[size];
+ }
+ return array;
+ }
+
+ @Override
+ public void skip() {
+ checkRead();
+ valuesRead++;
+ }
+
+ @Override
+ public int readInteger() {
+ // TODO: probably implement it separately
+ return (int) readLong();
+ }
+
+ @Override
+ public long readLong() {
+ checkRead();
+ valuesRead++;
+ return valuesBuffer[valuesBufferedRead++];
+ }
+
+ private void checkRead() {
+ if (valuesRead >= totalValueCount) {
+ throw new ParquetDecodingException("no more value to read, total value count is " + totalValueCount);
+ }
+ if (valuesBufferedRead >= valuesBuffered) {
+ //Set the last value buffered as the first
+ lastElement = valuesBuffer[valuesBufferedRead - 1];
+ valuesBufferedRead = 0;
+ valuesBuffered = 0;
+ Arrays.fill(valuesBuffer, 0);
+ try {
+ loadNewBlockToBuffer();
+ } catch (IOException e) {
+ throw new ParquetDecodingException("can not load next block", e);
+ }
+
+ }
+ }
+
+ private void loadNewBlockToBuffer() throws IOException {
+ long minDeltaInCurrentBlock;
+ try {
+ minDeltaInCurrentBlock = BytesUtils.readZigZagVarLong(in);
+ } catch (IOException e) {
+ throw new ParquetDecodingException("can not read min delta in current block", e);
+ }
+
+ readBitWidthsForMiniBlocks();
+
+ // mini block is atomic for reading, we read a mini block when there are more values left
+ int i;
+ for (i = 0; i < config.getMiniBlockNumInABlock() && valuesRead + valuesBuffered < totalValueCount; i++) {
+ BytePackerForLong packer = Packer.LITTLE_ENDIAN.newBytePackerForLong(bitWidths[i]);
+ unpackMiniBlock(packer);
+ }
+
+ //calculate values from deltas unpacked for current block
+ int valueUnpacked = i * config.getMiniBlockSizeInValues();
+ long prev = lastElement;
+ for (int j = valuesBuffered - valueUnpacked; j < valuesBuffered; j++) {
+ valuesBuffer[j] += minDeltaInCurrentBlock + prev;
+ prev = valuesBuffer[j];
+ }
+ }
+
+ /**
+ * mini block has a size of 8*n, unpack 8 value each time
+ *
+ * @param packer the packer created from bitwidth of current mini block
+ */
+ private void unpackMiniBlock(BytePackerForLong packer) throws IOException {
+ for (int j = 0; j < config.getMiniBlockSizeInValues(); j += 8) {
+ unpack8Values(packer);
+ }
+ }
+
+ private void unpack8Values(BytePackerForLong packer) throws IOException {
+ // get a single buffer of 8 values. most of the time, this won't require a copy
+ ByteBuffer buffer = readBitWidth(packer.getBitWidth());
+ packer.unpack8Values(buffer, buffer.position(), valuesBuffer, valuesBuffered);
+ this.valuesBuffered += 8;
+ }
+
+ private void readBitWidthsForMiniBlocks() {
+ for (int i = 0; i < config.getMiniBlockNumInABlock(); i++) {
+ try {
+ bitWidths[i] = BytesUtils.readIntLittleEndianOnOneByte(in);
+ } catch (IOException e) {
+ throw new ParquetDecodingException("Can not decode bit width in block header", e);
+ }
+ }
+ }
+
+ private ByteBuffer prepareBitWidthBuffer(int length) {
+ if (bitWidthBuffer == null || bitWidthBuffer.capacity() < length) {
+ bitWidthBuffer = ByteBuffer.allocate(length);
+ }
+ bitWidthBuffer.clear();
+ bitWidthBuffer.limit(length);
+ return bitWidthBuffer;
+ }
+
+ private ByteBuffer readBitWidth(int length) throws IOException {
+ ByteBuffer buffer = prepareBitWidthBuffer(length);
+ int read = in.read(buffer);
+ if (read != length) {
+ throw new EOFException("Reached end of stream");
+ }
+ buffer.position(0);
+ return buffer;
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/decoder/ParquetDeltaByteArrayReader.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/decoder/ParquetDeltaByteArrayReader.java
new file mode 100644
index 0000000..70c25b8
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/decoder/ParquetDeltaByteArrayReader.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.column.bytes.decoder;
+
+import java.io.IOException;
+
+import org.apache.asterix.column.bytes.stream.in.AbstractBytesInputStream;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.util.string.UTF8StringUtil;
+import org.apache.parquet.bytes.BytesUtils;
+import org.apache.parquet.column.values.deltastrings.DeltaByteArrayReader;
+import org.apache.parquet.io.ParquetDecodingException;
+
+/**
+ * Re-implementation of {@link DeltaByteArrayReader}
+ */
+public class ParquetDeltaByteArrayReader extends AbstractParquetValuesReader {
+ private final AbstractParquetValuesReader prefixLengthReader;
+ private final ParquetDeltaLengthByteArrayValuesReader suffixReader;
+ private final byte[] lengthBytes;
+
+ private final ArrayBackedValueStorage temp;
+ private final ArrayBackedValueStorage previous;
+ boolean newPage;
+
+ public ParquetDeltaByteArrayReader(boolean containsLength) {
+ this.prefixLengthReader = new ParquetDeltaBinaryPackingValuesReader();
+ this.suffixReader = new ParquetDeltaLengthByteArrayValuesReader();
+ this.temp = new ArrayBackedValueStorage();
+ this.previous = new ArrayBackedValueStorage();
+ lengthBytes = containsLength ? new byte[4] : new byte[0];
+ }
+
+ @Override
+ public void initFromPage(AbstractBytesInputStream stream) throws IOException {
+ AbstractBytesInputStream prefixStream = stream.sliceStream(BytesUtils.readUnsignedVarInt(stream));
+ prefixLengthReader.initFromPage(prefixStream);
+ suffixReader.initFromPage(stream);
+ previous.reset();
+ temp.reset();
+ newPage = true;
+ }
+
+ @Override
+ public void skip() {
+ // read the next value to skip so that previous is correct.
+ this.readBytes();
+ }
+
+ @Override
+ public IValueReference readBytes() {
+ int prefixLength = prefixLengthReader.readInteger();
+ // This does not copy bytes
+ IValueReference suffix = suffixReader.readBytes();
+
+ // NOTE: due to PARQUET-246, it is important that we
+ // respect prefixLength which was read from prefixLengthReader,
+ // even for the *first* value of a page. Even though the first
+ // value of the page should have an empty prefix, it may not
+ // because of PARQUET-246.
+
+ // We have to do this to materialize the output
+ try {
+ int lengthSize;
+ if (prefixLength != 0) {
+ lengthSize = appendLength(prefixLength + suffix.getLength());
+ temp.append(previous.getByteArray(), previous.getStartOffset(), prefixLength);
+ } else {
+ lengthSize = appendLength(suffix.getLength());
+ }
+ temp.append(suffix);
+ /*
+ * Adding length after appending prefix and suffix is important as we do not overwrite the original
+ * previous bytes
+ * */
+ System.arraycopy(lengthBytes, 0, temp.getByteArray(), 0, lengthSize);
+ previous.set(temp.getByteArray(), temp.getStartOffset() + lengthSize, temp.getLength() - lengthSize);
+ } catch (IOException e) {
+ throw new ParquetDecodingException(e);
+ }
+ newPage = false;
+ return temp;
+ }
+
+ private int appendLength(int length) {
+ if (lengthBytes.length > 0) {
+ int numOfBytes = UTF8StringUtil.encodeUTF8Length(length, lengthBytes, 0);
+ temp.setSize(numOfBytes);
+ return numOfBytes;
+ }
+ temp.setSize(0);
+ return 0;
+ }
+
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/decoder/ParquetDeltaLengthByteArrayValuesReader.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/decoder/ParquetDeltaLengthByteArrayValuesReader.java
new file mode 100644
index 0000000..9913269
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/decoder/ParquetDeltaLengthByteArrayValuesReader.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.column.bytes.decoder;
+
+import java.io.IOException;
+
+import org.apache.asterix.column.bytes.stream.in.AbstractBytesInputStream;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.parquet.bytes.BytesUtils;
+import org.apache.parquet.io.ParquetDecodingException;
+
+public class ParquetDeltaLengthByteArrayValuesReader extends AbstractParquetValuesReader {
+
+ private final VoidPointable value;
+ private final AbstractParquetValuesReader lengthReader;
+ private AbstractBytesInputStream in;
+
+ public ParquetDeltaLengthByteArrayValuesReader() {
+ this.lengthReader = new ParquetDeltaBinaryPackingValuesReader();
+ value = new VoidPointable();
+ }
+
+ @Override
+ public void initFromPage(AbstractBytesInputStream stream) throws IOException {
+ AbstractBytesInputStream lengthStream = stream.sliceStream(BytesUtils.readUnsignedVarInt(stream));
+ lengthReader.initFromPage(lengthStream);
+ this.in = stream;
+ }
+
+ @Override
+ public void skip() {
+ int length = lengthReader.readInteger();
+ try {
+ in.skipFully(length);
+ } catch (IOException e) {
+ throw new ParquetDecodingException("Failed to skip " + length + " bytes");
+ }
+ }
+
+ @Override
+ public IValueReference readBytes() {
+ int length = lengthReader.readInteger();
+ try {
+ in.read(value, length);
+ return value;
+ } catch (IOException e) {
+ throw new ParquetDecodingException("Failed to read " + length + " bytes");
+ }
+ }
+
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/decoder/ParquetDoublePlainValuesReader.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/decoder/ParquetDoublePlainValuesReader.java
new file mode 100644
index 0000000..196bec2
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/decoder/ParquetDoublePlainValuesReader.java
@@ -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.
+ */
+package org.apache.asterix.column.bytes.decoder;
+
+import java.io.IOException;
+
+import org.apache.asterix.column.bytes.stream.in.AbstractBytesInputStream;
+import org.apache.parquet.bytes.LittleEndianDataInputStream;
+import org.apache.parquet.io.ParquetDecodingException;
+
+public class ParquetDoublePlainValuesReader extends AbstractParquetValuesReader {
+ private LittleEndianDataInputStream in;
+
+ @Override
+ public void initFromPage(AbstractBytesInputStream stream) throws IOException {
+ this.in = new LittleEndianDataInputStream(stream.remainingStream());
+ }
+
+ @Override
+ public void skip() {
+ try {
+ in.skipBytes(8);
+ } catch (IOException e) {
+ throw new ParquetDecodingException("could not skip double", e);
+ }
+ }
+
+ @Override
+ public double readDouble() {
+ try {
+ return in.readDouble();
+ } catch (IOException e) {
+ throw new ParquetDecodingException("could not read double", e);
+ }
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/decoder/ParquetRunLengthBitPackingHybridDecoder.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/decoder/ParquetRunLengthBitPackingHybridDecoder.java
new file mode 100644
index 0000000..4607dc2
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/decoder/ParquetRunLengthBitPackingHybridDecoder.java
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.column.bytes.decoder;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Arrays;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.bytes.BytesUtils;
+import org.apache.parquet.column.values.bitpacking.BytePacker;
+import org.apache.parquet.column.values.bitpacking.Packer;
+import org.apache.parquet.column.values.rle.RunLengthBitPackingHybridDecoder;
+import org.apache.parquet.io.ParquetDecodingException;
+
+/**
+ * Re-implementation of {@link RunLengthBitPackingHybridDecoder}
+ */
+public class ParquetRunLengthBitPackingHybridDecoder {
+ private enum MODE {
+ RLE,
+ PACKED
+ }
+
+ private final int bitWidth;
+ private final BytePacker packer;
+ private InputStream in;
+
+ private MODE mode;
+ private int currentCount;
+ private int currentValue;
+ private int currentBufferLength;
+ private int[] currentBuffer;
+ private byte[] bytes;
+
+ public ParquetRunLengthBitPackingHybridDecoder(int bitWidth) {
+ Preconditions.checkArgument(bitWidth >= 0 && bitWidth <= 32, "bitWidth must be >= 0 and <= 32");
+ this.bitWidth = bitWidth;
+ this.packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+ }
+
+ public void reset(InputStream in) {
+ this.in = in;
+ currentCount = 0;
+ currentBufferLength = 0;
+ }
+
+ public int readInt() throws HyracksDataException {
+ try {
+ return nextInt();
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ }
+
+ private int nextInt() throws IOException {
+ if (currentCount == 0) {
+ readNext();
+ }
+ --currentCount;
+ int result;
+ switch (mode) {
+ case RLE:
+ result = currentValue;
+ break;
+ case PACKED:
+ result = currentBuffer[currentBufferLength - 1 - currentCount];
+ break;
+ default:
+ throw new ParquetDecodingException("not a valid mode " + mode);
+ }
+ return result;
+ }
+
+ private void readNext() throws IOException {
+ Preconditions.checkArgument(in.available() > 0, "Reading past RLE/BitPacking stream.");
+ final int header = BytesUtils.readUnsignedVarInt(in);
+ mode = (header & 1) == 0 ? MODE.RLE : MODE.PACKED;
+ switch (mode) {
+ case RLE:
+ currentCount = header >>> 1;
+ currentValue = BytesUtils.readIntLittleEndianPaddedOnBitWidth(in, bitWidth);
+ break;
+ case PACKED:
+ int numGroups = header >>> 1;
+ currentCount = numGroups * 8;
+ allocateBuffers(currentCount, numGroups * bitWidth);
+ // At the end of the file RLE data though, there might not be that many bytes left.
+ int bytesToRead = (int) Math.ceil(currentCount * bitWidth / 8.0);
+ bytesToRead = Math.min(bytesToRead, in.available());
+ readFully(bytes, bytesToRead);
+ for (int valueIndex = 0, byteIndex = 0; valueIndex < currentCount; valueIndex += 8, byteIndex +=
+ bitWidth) {
+ packer.unpack8Values(bytes, byteIndex, currentBuffer, valueIndex);
+ }
+ break;
+ default:
+ throw new ParquetDecodingException("not a valid mode " + mode);
+ }
+ }
+
+ private void allocateBuffers(int intBufferLength, int byteBufferLength) {
+ if (currentBuffer == null || currentBuffer.length < intBufferLength) {
+ currentBuffer = new int[intBufferLength];
+ } else {
+ Arrays.fill(currentBuffer, 0);
+ }
+ currentBufferLength = intBufferLength;
+
+ if (bytes == null || bytes.length < byteBufferLength) {
+ bytes = new byte[byteBufferLength];
+ } else {
+ Arrays.fill(bytes, (byte) 0);
+ }
+ }
+
+ private void readFully(byte[] b, int len) throws IOException {
+ if (len < 0)
+ throw new IndexOutOfBoundsException();
+ int n = 0;
+ while (n < len) {
+ int count = in.read(b, n, len - n);
+ if (count < 0)
+ throw new EOFException();
+ n += count;
+ }
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/encoder/AbstractParquetDeltaBinaryPackingValuesWriter.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/encoder/AbstractParquetDeltaBinaryPackingValuesWriter.java
new file mode 100644
index 0000000..3102063
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/encoder/AbstractParquetDeltaBinaryPackingValuesWriter.java
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.column.bytes.encoder;
+
+import java.io.IOException;
+
+import org.apache.asterix.column.bytes.ParquetDeltaBinaryPackingConfig;
+import org.apache.asterix.column.bytes.stream.out.MultiTemporaryBufferBytesOutputStream;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnWriteMultiPageOp;
+import org.apache.parquet.bytes.BytesUtils;
+import org.apache.parquet.column.values.delta.DeltaBinaryPackingValuesWriter;
+import org.apache.parquet.io.ParquetEncodingException;
+
+/**
+ * Re-implementation of {@link DeltaBinaryPackingValuesWriter}
+ */
+public abstract class AbstractParquetDeltaBinaryPackingValuesWriter extends AbstractParquetValuesWriter {
+
+ public static final int DEFAULT_NUM_BLOCK_VALUES = 128;
+
+ public static final int DEFAULT_NUM_MINIBLOCKS = 4;
+
+ protected final MultiTemporaryBufferBytesOutputStream outputStream;
+
+ /**
+ * stores blockSizeInValues, miniBlockNumInABlock and miniBlockSizeInValues
+ */
+ protected final ParquetDeltaBinaryPackingConfig config;
+
+ /**
+ * bit width for each mini block, reused between flushes
+ */
+ protected final int[] bitWidths;
+
+ protected int totalValueCount = 0;
+
+ /**
+ * a pointer to deltaBlockBuffer indicating the end of deltaBlockBuffer
+ * the number of values in the deltaBlockBuffer that haven't flushed to baos
+ * it will be reset after each flush
+ */
+ protected int deltaValuesToFlush = 0;
+
+ /**
+ * bytes buffer for a mini block, it is reused for each mini block.
+ * Therefore the size of biggest miniblock with bitwith of MAX_BITWITH is allocated
+ */
+ protected byte[] miniBlockByteBuffer;
+
+ /**
+ * Estimated element size after encoding
+ */
+ protected int estimatedElementSize = 0;
+ /**
+ * Estimated size for all non-flushed elements
+ */
+ protected int estimatedSize = 0;
+
+ protected AbstractParquetDeltaBinaryPackingValuesWriter(int blockSizeInValues, int miniBlockNum,
+ Mutable<IColumnWriteMultiPageOp> multiPageOpRef) {
+ this.config = new ParquetDeltaBinaryPackingConfig(blockSizeInValues, miniBlockNum);
+ bitWidths = new int[config.getMiniBlockNumInABlock()];
+ outputStream = new MultiTemporaryBufferBytesOutputStream(multiPageOpRef);
+ }
+
+ protected void writeBitWidthForMiniBlock(int i) {
+ try {
+ BytesUtils.writeIntLittleEndianOnOneByte(outputStream, bitWidths[i]);
+ } catch (IOException e) {
+ throw new ParquetEncodingException("can not write bit width for mini-block", e);
+ }
+ }
+
+ protected int getMiniBlockCountToFlush(double numberCount) {
+ return (int) Math.ceil(numberCount / config.getMiniBlockSizeInValues());
+ }
+
+ @Override
+ public void reset() throws HyracksDataException {
+ this.totalValueCount = 0;
+ this.outputStream.reset();
+ this.deltaValuesToFlush = 0;
+ }
+
+ @Override
+ public void close() {
+ this.totalValueCount = 0;
+ this.deltaValuesToFlush = 0;
+ outputStream.finish();
+ }
+
+ @Override
+ public int getEstimatedSize() {
+ return outputStream.size() + estimatedSize;
+ }
+
+ @Override
+ public int getAllocatedSize() {
+ return outputStream.capacity();
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/encoder/AbstractParquetValuesWriter.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/encoder/AbstractParquetValuesWriter.java
new file mode 100644
index 0000000..b53ded2
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/encoder/AbstractParquetValuesWriter.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.column.bytes.encoder;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.column.values.ValuesWriter;
+
+/**
+ * Replaces {@link ValuesWriter}
+ */
+public abstract class AbstractParquetValuesWriter {
+
+ public abstract BytesInput getBytes();
+
+ /**
+ * called after getBytes() to reset the current buffer and start writing the next page
+ */
+ public abstract void reset() throws HyracksDataException;
+
+ /**
+ * Called to close the values writer. Any output stream is closed and can no longer be used.
+ * All resources are released.
+ */
+ public abstract void close();
+
+ public abstract int getEstimatedSize();
+
+ /**
+ * @return the allocated size of the buffer
+ */
+ public abstract int getAllocatedSize();
+
+ /**
+ * @param v the value to encode
+ */
+ public void writeBoolean(boolean v) {
+ throw new UnsupportedOperationException(getClass().getName());
+ }
+
+ /**
+ * @param v the value to encode
+ * @param skipLengthBytes whether to skip the length bytes of {@link UTF8StringPointable} or not
+ */
+ public void writeBytes(IValueReference v, boolean skipLengthBytes) {
+ throw new UnsupportedOperationException(getClass().getName());
+ }
+
+ /**
+ * @param v the value to encode
+ */
+ public void writeInteger(int v) {
+ throw new UnsupportedOperationException(getClass().getName());
+ }
+
+ /**
+ * @param v the value to encode
+ */
+ public void writeLong(long v) {
+ throw new UnsupportedOperationException(getClass().getName());
+ }
+
+ /**
+ * @param v the value to encode
+ */
+ public void writeDouble(double v) {
+ throw new UnsupportedOperationException(getClass().getName());
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/encoder/ParquetDeltaBinaryPackingValuesWriterForInteger.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/encoder/ParquetDeltaBinaryPackingValuesWriterForInteger.java
new file mode 100644
index 0000000..1c474fc
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/encoder/ParquetDeltaBinaryPackingValuesWriterForInteger.java
@@ -0,0 +1,233 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.column.bytes.encoder;
+
+import java.io.IOException;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnWriteMultiPageOp;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.bytes.BytesUtils;
+import org.apache.parquet.column.values.bitpacking.BytePacker;
+import org.apache.parquet.column.values.bitpacking.Packer;
+import org.apache.parquet.column.values.delta.DeltaBinaryPackingValuesWriterForInteger;
+import org.apache.parquet.io.ParquetEncodingException;
+
+/**
+ * Re-implementation of {@link DeltaBinaryPackingValuesWriterForInteger}
+ */
+public class ParquetDeltaBinaryPackingValuesWriterForInteger extends AbstractParquetDeltaBinaryPackingValuesWriter {
+ /**
+ * max bitwidth for a mini block, it is used to allocate miniBlockByteBuffer which is
+ * reused between flushes.
+ */
+ private static final int MAX_BITWIDTH = 32;
+
+ private final int blockSizeInValues;
+ private final int miniBlockNumInABlock;
+ private final int miniBlockSizeInValues;
+
+ /**
+ * stores delta values starting from the 2nd value written(1st value is stored in header).
+ * It's reused between flushes
+ */
+ private final int[] deltaBlockBuffer;
+
+ /**
+ * firstValue is written to the header of the page
+ */
+ private int firstValue = 0;
+
+ /**
+ * cache previous written value for calculating delta
+ */
+ private int previousValue = 0;
+
+ /**
+ * min delta is written to the beginning of each block.
+ * it's zig-zag encoded. The deltas stored in each block is actually the difference to min delta,
+ * therefore are all positive
+ * it will be reset after each flush
+ */
+ private int minDeltaInCurrentBlock = Integer.MAX_VALUE;
+ private int maxDeltaInCurrentBlock = Integer.MIN_VALUE;
+ private int estimatedSize = 0;
+
+ public ParquetDeltaBinaryPackingValuesWriterForInteger(Mutable<IColumnWriteMultiPageOp> multiPageOpRef) {
+ this(DEFAULT_NUM_BLOCK_VALUES, DEFAULT_NUM_MINIBLOCKS, multiPageOpRef);
+ }
+
+ public ParquetDeltaBinaryPackingValuesWriterForInteger(int blockSizeInValues, int miniBlockNum,
+ Mutable<IColumnWriteMultiPageOp> multiPageOpRef) {
+ super(blockSizeInValues, miniBlockNum, multiPageOpRef);
+ this.blockSizeInValues = blockSizeInValues;
+ this.miniBlockNumInABlock = miniBlockNum;
+ double miniSize = (double) blockSizeInValues / miniBlockNumInABlock;
+ Preconditions.checkArgument(miniSize % 8 == 0, "miniBlockSize must be multiple of 8, but it's " + miniSize);
+ this.miniBlockSizeInValues = (int) miniSize;
+
+ deltaBlockBuffer = new int[blockSizeInValues];
+ miniBlockByteBuffer = new byte[miniBlockSizeInValues * MAX_BITWIDTH];
+ }
+
+ @Override
+ public void writeInteger(int v) {
+ totalValueCount++;
+
+ if (totalValueCount == 1) {
+ firstValue = v;
+ previousValue = firstValue;
+ return;
+ }
+
+ // Calculate delta. The possible overflow is accounted for. The algorithm is correct because
+ // Java int is working as a modalar ring with base 2^32 and because of the plus and minus
+ // properties of a ring. http://en.wikipedia.org/wiki/Modular_arithmetic#Integers_modulo_n
+ int delta = v - previousValue;
+ previousValue = v;
+
+ deltaBlockBuffer[deltaValuesToFlush++] = delta;
+
+ if (delta < minDeltaInCurrentBlock) {
+ minDeltaInCurrentBlock = delta;
+ }
+
+ if (blockSizeInValues == deltaValuesToFlush) {
+ flushBlockBuffer();
+ } else {
+ //Recalibrate the estimated size
+ if (delta > maxDeltaInCurrentBlock) {
+ maxDeltaInCurrentBlock = delta;
+ estimatedElementSize =
+ (64 - Long.numberOfLeadingZeros(maxDeltaInCurrentBlock - minDeltaInCurrentBlock));
+ estimatedSize = estimatedElementSize * deltaValuesToFlush;
+ } else {
+ estimatedSize += estimatedElementSize;
+ }
+ }
+ }
+
+ private void flushBlockBuffer() {
+ // since we store the min delta, the deltas will be converted to be the difference to min delta
+ // and all positive
+ for (int i = 0; i < deltaValuesToFlush; i++) {
+ deltaBlockBuffer[i] = deltaBlockBuffer[i] - minDeltaInCurrentBlock;
+ }
+
+ writeMinDelta();
+ int miniBlocksToFlush = getMiniBlockCountToFlush(deltaValuesToFlush);
+
+ calculateBitWidthsForDeltaBlockBuffer(miniBlocksToFlush);
+ for (int i = 0; i < miniBlockNumInABlock; i++) {
+ writeBitWidthForMiniBlock(i);
+ }
+
+ for (int i = 0; i < miniBlocksToFlush; i++) {
+ // writing i th miniblock
+ int currentBitWidth = bitWidths[i];
+ int blockOffset = 0;
+ BytePacker packer = Packer.LITTLE_ENDIAN.newBytePacker(currentBitWidth);
+ int miniBlockStart = i * miniBlockSizeInValues;
+ for (int j = miniBlockStart; j < (i + 1) * miniBlockSizeInValues; j += 8) {//8 values per pack
+ // mini block is atomic in terms of flushing
+ // This may write more values when reach to the end of data writing to last mini block,
+ // since it may not be aligned to miniblock,
+ // but doesn't matter. The reader uses total count to see if reached the end.
+ packer.pack8Values(deltaBlockBuffer, j, miniBlockByteBuffer, blockOffset);
+ blockOffset += currentBitWidth;
+ }
+ try {
+ outputStream.write(miniBlockByteBuffer, 0, blockOffset);
+ } catch (IOException e) {
+ throw new ParquetEncodingException(e);
+ }
+ }
+
+ minDeltaInCurrentBlock = Integer.MAX_VALUE;
+ deltaValuesToFlush = 0;
+ estimatedSize = 0;
+ maxDeltaInCurrentBlock = Integer.MIN_VALUE;
+ }
+
+ private void writeMinDelta() {
+ try {
+ BytesUtils.writeZigZagVarInt(minDeltaInCurrentBlock, outputStream);
+ } catch (IOException e) {
+ throw new ParquetEncodingException("can not write min delta for block", e);
+ }
+ }
+
+ /**
+ * iterate through values in each mini block and calculate the bitWidths of max values.
+ *
+ * @param miniBlocksToFlush number of miniblocks
+ */
+ private void calculateBitWidthsForDeltaBlockBuffer(int miniBlocksToFlush) {
+ for (int miniBlockIndex = 0; miniBlockIndex < miniBlocksToFlush; miniBlockIndex++) {
+ int mask = 0;
+ int miniStart = miniBlockIndex * miniBlockSizeInValues;
+
+ /*
+ * The end of current mini block could be the end of current block(deltaValuesToFlush) buffer
+ * when data is not aligned to mini block
+ */
+ int miniEnd = Math.min((miniBlockIndex + 1) * miniBlockSizeInValues, deltaValuesToFlush);
+
+ for (int i = miniStart; i < miniEnd; i++) {
+ mask |= deltaBlockBuffer[i];
+ }
+ bitWidths[miniBlockIndex] = 32 - Integer.numberOfLeadingZeros(mask);
+ }
+ }
+
+ /**
+ * getBytes will trigger flushing block buffer, DO NOT write after getBytes() is called without calling reset()
+ *
+ * @return a BytesInput that contains the encoded page data
+ */
+ @Override
+ public BytesInput getBytes() {
+ // The Page Header should include: blockSizeInValues, numberOfMiniBlocks, totalValueCount
+ if (deltaValuesToFlush != 0) {
+ flushBlockBuffer();
+ }
+ BytesInput configBytes = BytesInput.concat(BytesInput.fromUnsignedVarInt(blockSizeInValues),
+ BytesInput.fromUnsignedVarInt(miniBlockNumInABlock));
+ return BytesInput.concat(configBytes, BytesInput.fromUnsignedVarInt(totalValueCount),
+ BytesInput.fromZigZagVarInt(firstValue), outputStream.asBytesInput());
+ }
+
+ @Override
+ public void reset() throws HyracksDataException {
+ super.reset();
+ this.minDeltaInCurrentBlock = Integer.MAX_VALUE;
+ estimatedSize = 0;
+ maxDeltaInCurrentBlock = Integer.MIN_VALUE;
+ }
+
+ @Override
+ public void close() {
+ super.close();
+ this.minDeltaInCurrentBlock = Integer.MAX_VALUE;
+ estimatedSize = 0;
+ maxDeltaInCurrentBlock = Integer.MIN_VALUE;
+ }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/encoder/ParquetDeltaBinaryPackingValuesWriterForLong.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/encoder/ParquetDeltaBinaryPackingValuesWriterForLong.java
new file mode 100644
index 0000000..6ba40c1
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/encoder/ParquetDeltaBinaryPackingValuesWriterForLong.java
@@ -0,0 +1,234 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.column.bytes.encoder;
+
+import java.io.IOException;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnWriteMultiPageOp;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.bytes.BytesUtils;
+import org.apache.parquet.column.values.bitpacking.BytePackerForLong;
+import org.apache.parquet.column.values.bitpacking.Packer;
+import org.apache.parquet.column.values.delta.DeltaBinaryPackingValuesWriterForLong;
+import org.apache.parquet.io.ParquetEncodingException;
+
+/**
+ * Re-implementation of {@link DeltaBinaryPackingValuesWriterForLong}
+ */
+public class ParquetDeltaBinaryPackingValuesWriterForLong extends AbstractParquetDeltaBinaryPackingValuesWriter {
+ /**
+ * max bitwidth for a mini block, it is used to allocate miniBlockByteBuffer which is
+ * reused between flushes.
+ */
+ private static final int MAX_BITWIDTH = 64;
+
+ private final int blockSizeInValues;
+ private final int miniBlockNumInABlock;
+ private final int miniBlockSizeInValues;
+
+ /**
+ * stores delta values starting from the 2nd value written(1st value is stored in header).
+ * It's reused between flushes
+ */
+ private final long[] deltaBlockBuffer;
+
+ /**
+ * firstValue is written to the header of the page
+ */
+ private long firstValue = 0;
+
+ /**
+ * cache previous written value for calculating delta
+ */
+ private long previousValue = 0;
+
+ /**
+ * min delta is written to the beginning of each block.
+ * it's zig-zag encoded. The deltas stored in each block is actually the difference to min delta,
+ * therefore are all positive
+ * it will be reset after each flush
+ */
+ private long minDeltaInCurrentBlock = Long.MAX_VALUE;
+ private long maxDeltaInCurrentBlock = Long.MIN_VALUE;
+
+ public ParquetDeltaBinaryPackingValuesWriterForLong(Mutable<IColumnWriteMultiPageOp> multiPageOpRef) {
+ this(DEFAULT_NUM_BLOCK_VALUES, DEFAULT_NUM_MINIBLOCKS, multiPageOpRef);
+ }
+
+ public ParquetDeltaBinaryPackingValuesWriterForLong(int blockSizeInValues, int miniBlockNum,
+ Mutable<IColumnWriteMultiPageOp> multiPageOpRef) {
+ super(blockSizeInValues, miniBlockNum, multiPageOpRef);
+ this.blockSizeInValues = blockSizeInValues;
+ this.miniBlockNumInABlock = miniBlockNum;
+ double miniSize = (double) blockSizeInValues / miniBlockNumInABlock;
+ Preconditions.checkArgument(miniSize % 8 == 0, "miniBlockSize must be multiple of 8, but it's " + miniSize);
+ this.miniBlockSizeInValues = (int) miniSize;
+ deltaBlockBuffer = new long[blockSizeInValues];
+ miniBlockByteBuffer = new byte[miniBlockSizeInValues * MAX_BITWIDTH];
+ }
+
+ @Override
+ public void writeLong(long v) {
+ totalValueCount++;
+
+ if (totalValueCount == 1) {
+ firstValue = v;
+ previousValue = firstValue;
+ return;
+ }
+
+ // Calculate delta. The possible overflow is accounted for. The algorithm is correct because
+ // Java long is working as a modalar ring with base 2^64 and because of the plus and minus
+ // properties of a ring. http://en.wikipedia.org/wiki/Modular_arithmetic#Integers_modulo_n
+ long delta = v - previousValue;
+ previousValue = v;
+
+ deltaBlockBuffer[deltaValuesToFlush++] = delta;
+
+ if (delta < minDeltaInCurrentBlock) {
+ minDeltaInCurrentBlock = delta;
+ }
+
+ if (blockSizeInValues == deltaValuesToFlush) {
+ flushBlockBuffer();
+ } else {
+ //Recalibrate the estimated size
+ if (delta > maxDeltaInCurrentBlock) {
+ maxDeltaInCurrentBlock = delta;
+ estimatedElementSize =
+ (64 - Long.numberOfLeadingZeros(maxDeltaInCurrentBlock - minDeltaInCurrentBlock));
+ estimatedSize = estimatedElementSize * deltaValuesToFlush;
+ } else {
+ estimatedSize += estimatedElementSize;
+ }
+ }
+ }
+
+ private void flushBlockBuffer() {
+ // since we store the min delta, the deltas will be converted to be the difference to min delta
+ // and all positive
+ for (int i = 0; i < deltaValuesToFlush; i++) {
+ deltaBlockBuffer[i] = deltaBlockBuffer[i] - minDeltaInCurrentBlock;
+ }
+
+ writeMinDelta();
+ int miniBlocksToFlush = getMiniBlockCountToFlush(deltaValuesToFlush);
+
+ calculateBitWidthsForDeltaBlockBuffer(miniBlocksToFlush);
+ int minBitWidth = Integer.MAX_VALUE;
+ for (int i = 0; i < miniBlockNumInABlock; i++) {
+ writeBitWidthForMiniBlock(i);
+ minBitWidth = Math.min(bitWidths[i], minBitWidth);
+ }
+
+ for (int i = 0; i < miniBlocksToFlush; i++) {
+ // writing i th miniblock
+ int currentBitWidth = bitWidths[i];
+ int blockOffset = 0;
+ // TODO: should this cache the packer?
+ BytePackerForLong packer = Packer.LITTLE_ENDIAN.newBytePackerForLong(currentBitWidth);
+ int miniBlockStart = i * miniBlockSizeInValues;
+ // pack values into the miniblock buffer, 8 at a time to get exactly currentBitWidth bytes
+ for (int j = miniBlockStart; j < (i + 1) * miniBlockSizeInValues; j += 8) {
+ // mini block is atomic in terms of flushing
+ // This may write more values when reach to the end of data writing to last mini block,
+ // since it may not be aligned to miniblock,
+ // but doesn't matter. The reader uses total count to see if reached the end.
+ packer.pack8Values(deltaBlockBuffer, j, miniBlockByteBuffer, blockOffset);
+ blockOffset += currentBitWidth;
+ }
+ try {
+ outputStream.write(miniBlockByteBuffer, 0, blockOffset);
+ } catch (IOException e) {
+ throw new ParquetEncodingException(e);
+ }
+ }
+
+ minDeltaInCurrentBlock = Long.MAX_VALUE;
+ maxDeltaInCurrentBlock = Long.MIN_VALUE;
+ deltaValuesToFlush = 0;
+ estimatedElementSize = 0;
+ estimatedSize = 0;
+ }
+
+ private void writeMinDelta() {
+ try {
+ BytesUtils.writeZigZagVarLong(minDeltaInCurrentBlock, outputStream);
+ } catch (IOException e) {
+ throw new ParquetEncodingException("can not write min delta for block", e);
+ }
+ }
+
+ /**
+ * iterate through values in each mini block and calculate the bitWidths of max values.
+ *
+ * @param miniBlocksToFlush number of miniblocks
+ */
+ private void calculateBitWidthsForDeltaBlockBuffer(int miniBlocksToFlush) {
+ for (int miniBlockIndex = 0; miniBlockIndex < miniBlocksToFlush; miniBlockIndex++) {
+ long mask = 0;
+ int miniStart = miniBlockIndex * miniBlockSizeInValues;
+
+ //The end of current mini block could be the end of current block(deltaValuesToFlush) buffer
+ //when data is not aligned to mini block
+ int miniEnd = Math.min((miniBlockIndex + 1) * miniBlockSizeInValues, deltaValuesToFlush);
+
+ for (int i = miniStart; i < miniEnd; i++) {
+ mask |= deltaBlockBuffer[i];
+ }
+ bitWidths[miniBlockIndex] = 64 - Long.numberOfLeadingZeros(mask);
+ }
+ }
+
+ /**
+ * getBytes will trigger flushing block buffer, DO NOT write after getBytes() is called without calling reset()
+ *
+ * @return a BytesInput that contains the encoded page data
+ */
+ @Override
+ public BytesInput getBytes() {
+ // The Page Header should include: blockSizeInValues, numberOfMiniBlocks, totalValueCount
+ if (deltaValuesToFlush != 0) {
+ flushBlockBuffer();
+ }
+ BytesInput configBytes = BytesInput.concat(BytesInput.fromUnsignedVarInt(blockSizeInValues),
+ BytesInput.fromUnsignedVarInt(miniBlockNumInABlock));
+ return BytesInput.concat(configBytes, BytesInput.fromUnsignedVarInt(totalValueCount),
+ BytesInput.fromZigZagVarLong(firstValue), outputStream.asBytesInput());
+ }
+
+ @Override
+ public void reset() throws HyracksDataException {
+ super.reset();
+ this.minDeltaInCurrentBlock = Long.MAX_VALUE;
+ this.maxDeltaInCurrentBlock = Long.MIN_VALUE;
+ previousValue = 0;
+ estimatedElementSize = 0;
+ estimatedSize = 0;
+ }
+
+ @Override
+ public void close() {
+ super.close();
+ this.minDeltaInCurrentBlock = Long.MAX_VALUE;
+ }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/encoder/ParquetDeltaByteArrayWriter.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/encoder/ParquetDeltaByteArrayWriter.java
new file mode 100644
index 0000000..1b46116
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/encoder/ParquetDeltaByteArrayWriter.java
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.column.bytes.encoder;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnWriteMultiPageOp;
+import org.apache.hyracks.util.string.UTF8StringUtil;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.column.values.deltastrings.DeltaByteArrayWriter;
+
+/**
+ * Re-implementation of {@link DeltaByteArrayWriter}
+ */
+public class ParquetDeltaByteArrayWriter extends AbstractParquetValuesWriter {
+ private static final IValueReference EMPTY_VALUE;
+ private final ParquetDeltaBinaryPackingValuesWriterForInteger prefixLengthWriter;
+ private final ParquetDeltaLengthByteArrayValuesWriter suffixWriter;
+ private final VoidPointable suffix;
+ private final ArrayBackedValueStorage previous = new ArrayBackedValueStorage();
+
+ static {
+ VoidPointable emptyPointable = new VoidPointable();
+ emptyPointable.set(new byte[0], 0, 0);
+ EMPTY_VALUE = emptyPointable;
+ }
+
+ public ParquetDeltaByteArrayWriter(Mutable<IColumnWriteMultiPageOp> multiPageOpRef) {
+ this.prefixLengthWriter = new ParquetDeltaBinaryPackingValuesWriterForInteger(multiPageOpRef);
+ this.suffixWriter = new ParquetDeltaLengthByteArrayValuesWriter(multiPageOpRef);
+ suffix = new VoidPointable();
+ suffix.set(EMPTY_VALUE);
+ }
+
+ @Override
+ public BytesInput getBytes() {
+ BytesInput prefixBytes = prefixLengthWriter.getBytes();
+ BytesInput prefixLength = BytesInput.fromUnsignedVarInt((int) prefixBytes.size());
+ BytesInput suffixBytes = suffixWriter.getBytes();
+ return BytesInput.concat(prefixLength, prefixBytes, suffixBytes);
+ }
+
+ @Override
+ public void reset() throws HyracksDataException {
+ prefixLengthWriter.reset();
+ suffixWriter.reset();
+ previous.reset();
+ suffix.set(EMPTY_VALUE);
+ }
+
+ @Override
+ public void close() {
+ prefixLengthWriter.close();
+ suffixWriter.close();
+ previous.reset();
+ suffix.set(EMPTY_VALUE);
+ }
+
+ @Override
+ public int getEstimatedSize() {
+ return prefixLengthWriter.getEstimatedSize() + suffixWriter.getEstimatedSize();
+ }
+
+ @Override
+ public int getAllocatedSize() {
+ return prefixLengthWriter.getAllocatedSize() + suffixWriter.getAllocatedSize();
+ }
+
+ @Override
+ public void writeBytes(IValueReference value, boolean skipLengthBytes) {
+ byte[] bytes = value.getByteArray();
+ int start = value.getStartOffset();
+ int length = value.getLength();
+ if (skipLengthBytes) {
+ int lengthBytes = UTF8StringUtil.getNumBytesToStoreLength(bytes, start);
+ start += lengthBytes;
+ length -= lengthBytes;
+ }
+ writeBytes(bytes, start, length);
+ }
+
+ private void writeBytes(byte[] bytes, int offset, int length) {
+ final byte[] prevBytes = previous.getByteArray();
+ final int prevOffset = previous.getStartOffset();
+ final int minLength = Math.min(length, previous.getLength());
+ // find the number of matching prefix bytes between this value and the previous one
+ int i;
+ for (i = 0; (i < minLength) && (bytes[i + offset] == prevBytes[i + prevOffset]); i++);
+ prefixLengthWriter.writeInteger(i);
+ suffix.set(bytes, offset + i, length - i);
+ suffixWriter.writeBytes(suffix, false);
+ // We store as bytes could be evicted from the buffer cache
+ previous.set(bytes, offset, length);
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/encoder/ParquetDeltaLengthByteArrayValuesWriter.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/encoder/ParquetDeltaLengthByteArrayValuesWriter.java
new file mode 100644
index 0000000..afab48eb
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/encoder/ParquetDeltaLengthByteArrayValuesWriter.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.column.bytes.encoder;
+
+import java.io.IOException;
+
+import org.apache.asterix.column.bytes.stream.out.MultiTemporaryBufferBytesOutputStream;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnWriteMultiPageOp;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.bytes.LittleEndianDataOutputStream;
+import org.apache.parquet.column.values.deltalengthbytearray.DeltaLengthByteArrayValuesWriter;
+import org.apache.parquet.io.ParquetEncodingException;
+
+/**
+ * Re-implementation of {@link DeltaLengthByteArrayValuesWriter}
+ */
+public class ParquetDeltaLengthByteArrayValuesWriter extends AbstractParquetValuesWriter {
+ private final ParquetDeltaBinaryPackingValuesWriterForInteger lengthWriter;
+ private final MultiTemporaryBufferBytesOutputStream outputStream;
+ private final LittleEndianDataOutputStream out;
+
+ public ParquetDeltaLengthByteArrayValuesWriter(Mutable<IColumnWriteMultiPageOp> multiPageOpRef) {
+ outputStream = new MultiTemporaryBufferBytesOutputStream(multiPageOpRef);
+ out = new LittleEndianDataOutputStream(outputStream);
+ lengthWriter = new ParquetDeltaBinaryPackingValuesWriterForInteger(multiPageOpRef);
+ }
+
+ @Override
+ public void writeBytes(IValueReference value, boolean skipLengthBytes) {
+ try {
+ lengthWriter.writeInteger(value.getLength());
+ out.write(value.getByteArray(), value.getStartOffset(), value.getLength());
+ } catch (IOException e) {
+ throw new ParquetEncodingException("could not write bytes", e);
+ }
+ }
+
+ @Override
+ public BytesInput getBytes() {
+ try {
+ out.flush();
+ } catch (IOException e) {
+ throw new ParquetEncodingException("could not write page", e);
+ }
+ BytesInput lengthBytes = lengthWriter.getBytes();
+ BytesInput lengthSize = BytesInput.fromUnsignedVarInt((int) lengthBytes.size());
+ BytesInput arrayBytes = outputStream.asBytesInput();
+ return BytesInput.concat(lengthSize, lengthBytes, arrayBytes);
+ }
+
+ @Override
+ public void reset() throws HyracksDataException {
+ lengthWriter.reset();
+ outputStream.reset();
+ }
+
+ @Override
+ public void close() {
+ lengthWriter.close();
+ outputStream.finish();
+ }
+
+ @Override
+ public int getEstimatedSize() {
+ return lengthWriter.getEstimatedSize() + outputStream.size();
+ }
+
+ @Override
+ public int getAllocatedSize() {
+ return lengthWriter.getAllocatedSize() + outputStream.capacity();
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/encoder/ParquetPlainValuesWriter.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/encoder/ParquetPlainValuesWriter.java
new file mode 100644
index 0000000..0298e59
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/encoder/ParquetPlainValuesWriter.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.column.bytes.encoder;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+
+import org.apache.asterix.column.bytes.stream.out.AbstractBytesOutputStream;
+import org.apache.asterix.column.bytes.stream.out.MultiTemporaryBufferBytesOutputStream;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnWriteMultiPageOp;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.bytes.LittleEndianDataOutputStream;
+import org.apache.parquet.column.values.plain.PlainValuesWriter;
+import org.apache.parquet.io.ParquetEncodingException;
+
+/**
+ * Re-implementation of {@link PlainValuesWriter}
+ */
+public class ParquetPlainValuesWriter extends AbstractParquetValuesWriter {
+ public static final Charset CHARSET = StandardCharsets.UTF_8;
+
+ private final AbstractBytesOutputStream outputStream;
+ private final LittleEndianDataOutputStream out;
+
+ public ParquetPlainValuesWriter(Mutable<IColumnWriteMultiPageOp> multiPageOpRef) {
+ outputStream = new MultiTemporaryBufferBytesOutputStream(multiPageOpRef);
+ out = new LittleEndianDataOutputStream(outputStream);
+ }
+
+ @Override
+ public final void writeDouble(double v) {
+ try {
+ out.writeDouble(v);
+ } catch (IOException e) {
+ throw new ParquetEncodingException("could not write double", e);
+ }
+ }
+
+ @Override
+ public BytesInput getBytes() {
+ try {
+ out.flush();
+ } catch (IOException e) {
+ throw new ParquetEncodingException("could not write page", e);
+ }
+ return outputStream.asBytesInput();
+ }
+
+ @Override
+ public void reset() throws HyracksDataException {
+ outputStream.reset();
+ }
+
+ @Override
+ public void close() {
+ outputStream.finish();
+ }
+
+ @Override
+ public int getEstimatedSize() {
+ return outputStream.size();
+ }
+
+ @Override
+ public int getAllocatedSize() {
+ return outputStream.capacity();
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/encoder/ParquetRunLengthBitPackingHybridEncoder.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/encoder/ParquetRunLengthBitPackingHybridEncoder.java
new file mode 100644
index 0000000..671e0a1
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/encoder/ParquetRunLengthBitPackingHybridEncoder.java
@@ -0,0 +1,263 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.column.bytes.encoder;
+
+import java.io.IOException;
+
+import org.apache.asterix.column.bytes.stream.out.GrowableBytesOutputStream;
+import org.apache.asterix.column.bytes.stream.out.pointer.IReservedPointer;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.bytes.BytesUtils;
+import org.apache.parquet.column.values.bitpacking.BytePacker;
+import org.apache.parquet.column.values.bitpacking.Packer;
+import org.apache.parquet.column.values.rle.RunLengthBitPackingHybridEncoder;
+
+/**
+ * Re-implementation of {@link RunLengthBitPackingHybridEncoder}
+ */
+public class ParquetRunLengthBitPackingHybridEncoder {
+ private final BytePacker packer;
+
+ private final GrowableBytesOutputStream outputStream;
+
+ /**
+ * The bit width used for bit-packing and for writing
+ * the repeated-value
+ */
+ private final int bitWidth;
+
+ /**
+ * Values that are bit packed 8 at at a time are packed into this
+ * buffer, which is then written to baos
+ */
+ private final byte[] packBuffer;
+
+ /**
+ * Previous value written, used to detect repeated values
+ */
+ private int previousValue;
+
+ /**
+ * We buffer 8 values at a time, and either bit pack them
+ * or discard them after writing a rle-run
+ */
+ private final int[] bufferedValues;
+ private int numBufferedValues;
+
+ /**
+ * How many times a value has been repeated
+ */
+ private int repeatCount;
+
+ /**
+ * How many groups of 8 values have been written
+ * to the current bit-packed-run
+ */
+ private int bitPackedGroupCount;
+
+ /**
+ * A "pointer" to a single byte in baos,
+ * which we use as our bit-packed-header. It's really
+ * the logical index of the byte in baos.
+ * <p>
+ * We are only using one byte for this header,
+ * which limits us to writing 504 values per bit-packed-run.
+ * <p>
+ * MSB must be 0 for varint encoding, LSB must be 1 to signify
+ * that this is a bit-packed-header leaves 6 bits to write the
+ * number of 8-groups -> (2^6 - 1) * 8 = 504
+ */
+ private final IReservedPointer bitPackedRunHeaderPointer;
+
+ private boolean toBytesCalled;
+
+ public ParquetRunLengthBitPackingHybridEncoder(int bitWidth) {
+
+ Preconditions.checkArgument(bitWidth >= 0 && bitWidth <= 32, "bitWidth must be >= 0 and <= 32");
+
+ this.bitWidth = bitWidth;
+ this.outputStream = new GrowableBytesOutputStream();
+ this.bitPackedRunHeaderPointer = outputStream.createPointer();
+ this.packBuffer = new byte[bitWidth];
+ this.bufferedValues = new int[8];
+ this.packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+ reset(false);
+ }
+
+ private void reset(boolean resetBaos) {
+ if (resetBaos) {
+ this.outputStream.reset();
+ }
+ this.previousValue = 0;
+ this.numBufferedValues = 0;
+ this.repeatCount = 0;
+ this.bitPackedGroupCount = 0;
+ this.bitPackedRunHeaderPointer.reset();
+ this.toBytesCalled = false;
+ }
+
+ public void writeInt(int value) throws IOException {
+ if (value == previousValue) {
+ // keep track of how many times we've seen this value
+ // consecutively
+ ++repeatCount;
+
+ if (repeatCount >= 8) {
+ // we've seen this at least 8 times, we're
+ // certainly going to write an rle-run,
+ // so just keep on counting repeats for now
+ return;
+ }
+ } else {
+ // This is a new value, check if it signals the end of
+ // an rle-run
+ if (repeatCount >= 8) {
+ // it does! write an rle-run
+ writeRleRun();
+ }
+
+ // this is a new value so we've only seen it once
+ repeatCount = 1;
+ // start tracking this value for repeats
+ previousValue = value;
+ }
+
+ // We have not seen enough repeats to justify an rle-run yet,
+ // so buffer this value in case we decide to write a bit-packed-run
+ bufferedValues[numBufferedValues] = value;
+ ++numBufferedValues;
+
+ if (numBufferedValues == 8) {
+ // we've encountered less than 8 repeated values, so
+ // either start a new bit-packed-run or append to the
+ // current bit-packed-run
+ writeOrAppendBitPackedRun();
+ }
+ }
+
+ private void writeOrAppendBitPackedRun() throws IOException {
+ if (bitPackedGroupCount >= 63) {
+ // we've packed as many values as we can for this run,
+ // end it and start a new one
+ endPreviousBitPackedRun();
+ }
+
+ if (!bitPackedRunHeaderPointer.isSet()) {
+ // this is a new bit-packed-run, allocate a byte for the header
+ // and keep a "pointer" to it so that it can be mutated later
+ outputStream.reserveByte(bitPackedRunHeaderPointer);
+ }
+
+ packer.pack8Values(bufferedValues, 0, packBuffer, 0);
+ outputStream.write(packBuffer);
+
+ // empty the buffer, they've all been written
+ numBufferedValues = 0;
+
+ // clear the repeat count, as some repeated values
+ // may have just been bit packed into this run
+ repeatCount = 0;
+
+ ++bitPackedGroupCount;
+ }
+
+ /**
+ * If we are currently writing a bit-packed-run, update the
+ * bit-packed-header and consider this run to be over
+ * <p>
+ * does nothing if we're not currently writing a bit-packed run
+ */
+ private void endPreviousBitPackedRun() {
+ if (!bitPackedRunHeaderPointer.isSet()) {
+ // we're not currently in a bit-packed-run
+ return;
+ }
+
+ // create bit-packed-header, which needs to fit in 1 byte
+ byte bitPackHeader = (byte) ((bitPackedGroupCount << 1) | 1);
+
+ // update this byte
+ bitPackedRunHeaderPointer.setByte(bitPackHeader);
+
+ // mark that this run is over
+ bitPackedRunHeaderPointer.reset();
+
+ // reset the number of groups
+ bitPackedGroupCount = 0;
+ }
+
+ private void writeRleRun() throws IOException {
+ // we may have been working on a bit-packed-run
+ // so close that run if it exists before writing this
+ // rle-run
+ endPreviousBitPackedRun();
+
+ // write the rle-header (lsb of 0 signifies a rle run)
+ BytesUtils.writeUnsignedVarInt(repeatCount << 1, outputStream);
+ // write the repeated-value
+ BytesUtils.writeIntLittleEndianPaddedOnBitWidth(outputStream, previousValue, bitWidth);
+
+ // reset the repeat count
+ repeatCount = 0;
+
+ // throw away all the buffered values, they were just repeats and they've been written
+ numBufferedValues = 0;
+ }
+
+ public BytesInput toBytes() throws IOException {
+ Preconditions.checkArgument(!toBytesCalled, "You cannot call toBytes() more than once without calling reset()");
+
+ // write anything that is buffered / queued up for an rle-run
+ if (repeatCount >= 8) {
+ writeRleRun();
+ } else if (numBufferedValues > 0) {
+ for (int i = numBufferedValues; i < 8; i++) {
+ bufferedValues[i] = 0;
+ }
+ writeOrAppendBitPackedRun();
+ endPreviousBitPackedRun();
+ } else {
+ endPreviousBitPackedRun();
+ }
+
+ toBytesCalled = true;
+ return outputStream.asBytesInput();
+ }
+
+ /**
+ * Reset this encoder for re-use
+ */
+ public void reset() {
+ reset(true);
+ }
+
+ public void close() {
+ reset(false);
+ outputStream.finish();
+ }
+
+ public int getEstimatedSize() {
+ return outputStream.size() + repeatCount * bitWidth;
+ }
+
+ public int getAllocatedSize() {
+ return outputStream.capacity();
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/stream/in/AbstractBytesInputStream.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/stream/in/AbstractBytesInputStream.java
new file mode 100644
index 0000000..b50143b
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/stream/in/AbstractBytesInputStream.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.column.bytes.stream.in;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnBufferProvider;
+
+public abstract class AbstractBytesInputStream extends InputStream {
+
+ public abstract void resetAt(int bytesToSkip, AbstractBytesInputStream stream) throws IOException;
+
+ protected abstract void addBuffer(ByteBuffer buffer);
+
+ public abstract void read(IPointable pointable, int length) throws EOFException;
+
+ @Override
+ public abstract int read() throws IOException;
+
+ @Override
+ public abstract int read(byte[] bytes, int offset, int length) throws IOException;
+
+ @Override
+ public abstract long skip(long n);
+
+ public abstract int read(ByteBuffer out);
+
+ public abstract AbstractBytesInputStream remainingStream() throws EOFException;
+
+ public abstract AbstractBytesInputStream sliceStream(int length) throws EOFException;
+
+ @Override
+ public abstract void mark(int readLimit);
+
+ @Override
+ public abstract void reset() throws IOException;
+
+ public abstract void reset(IColumnBufferProvider bufferProvider) throws HyracksDataException;
+
+ @Override
+ public abstract int available();
+
+ public final void skipFully(long n) throws IOException {
+ long skipped = skip(n);
+ if (skipped < n) {
+ throw new EOFException("Not enough bytes to skip: " + skipped + " < " + n);
+ }
+ }
+
+ @Override
+ public final boolean markSupported() {
+ return true;
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/stream/in/ByteBufferInputStream.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/stream/in/ByteBufferInputStream.java
new file mode 100644
index 0000000..833765c
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/stream/in/ByteBufferInputStream.java
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.column.bytes.stream.in;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnBufferProvider;
+
+public final class ByteBufferInputStream extends AbstractBytesInputStream {
+ private ByteBuffer buffer;
+ private int mark = -1;
+
+ @Override
+ public void reset(IColumnBufferProvider bufferProvider) {
+ addBuffer(bufferProvider.getBuffer());
+ }
+
+ @Override
+ protected void addBuffer(ByteBuffer buffer) {
+ this.buffer = buffer;
+ mark = -1;
+ }
+
+ @Override
+ public void resetAt(int bytesToSkip, AbstractBytesInputStream stream) throws IOException {
+ ByteBufferInputStream in = (ByteBufferInputStream) stream;
+ buffer = in.buffer.duplicate();
+ buffer.position(buffer.position() + bytesToSkip);
+ mark = -1;
+ }
+
+ @Override
+ public void read(IPointable pointable, int length) throws EOFException {
+ if (buffer.remaining() < length) {
+ throw new EOFException();
+ }
+
+ pointable.set(buffer.array(), buffer.position(), length);
+ buffer.position(buffer.position() + length);
+ }
+
+ @Override
+ public int read() throws IOException {
+ if (!buffer.hasRemaining()) {
+ throw new EOFException();
+ }
+ return buffer.get() & 0xFF; // as unsigned
+ }
+
+ @Override
+ public int read(byte[] bytes, int offset, int length) throws IOException {
+ if (length == 0) {
+ return 0;
+ }
+
+ int remaining = buffer.remaining();
+ if (remaining <= 0) {
+ return -1;
+ }
+
+ int bytesToRead = Math.min(remaining, length);
+ buffer.get(bytes, offset, bytesToRead);
+
+ return bytesToRead;
+ }
+
+ @Override
+ public long skip(long n) {
+ if (n == 0) {
+ return 0;
+ }
+
+ if (!buffer.hasRemaining()) {
+ return -1;
+ }
+
+ // buffer.remaining is an int, so this will always fit in an int
+ int bytesToSkip = (int) Math.min(buffer.remaining(), n);
+ buffer.position(buffer.position() + bytesToSkip);
+
+ return bytesToSkip;
+ }
+
+ @Override
+ public int read(ByteBuffer out) {
+ int bytesToCopy;
+ ByteBuffer copyBuffer;
+ if (buffer.remaining() <= out.remaining()) {
+ // copy the entire buffer
+ bytesToCopy = buffer.remaining();
+ copyBuffer = buffer;
+ } else {
+ // copy a slice of the current buffer
+ bytesToCopy = out.remaining();
+ copyBuffer = buffer.duplicate();
+ copyBuffer.position(buffer.position());
+ copyBuffer.limit(buffer.position() + bytesToCopy);
+ buffer.position(buffer.position() + bytesToCopy);
+ }
+
+ out.put(copyBuffer);
+ out.flip();
+
+ return bytesToCopy;
+ }
+
+ @Override
+ public AbstractBytesInputStream sliceStream(int length) throws EOFException {
+ if (buffer.remaining() < length) {
+ throw new EOFException();
+ }
+ ByteBuffer copy = buffer.duplicate();
+ copy.position(buffer.position());
+ copy.limit(buffer.position() + length);
+ ByteBufferInputStream in = new ByteBufferInputStream();
+ in.addBuffer(copy);
+ buffer.position(buffer.position() + length);
+ return in;
+ }
+
+ @Override
+ public AbstractBytesInputStream remainingStream() {
+ ByteBuffer remaining = buffer.duplicate();
+ remaining.position(buffer.position());
+ buffer.position(buffer.limit());
+ ByteBufferInputStream in = new ByteBufferInputStream();
+ in.addBuffer(remaining);
+ return in;
+ }
+
+ @Override
+ public void mark(int readlimit) {
+ this.mark = buffer.position();
+ }
+
+ @Override
+ public void reset() throws IOException {
+ if (mark >= 0) {
+ buffer.position(mark);
+ this.mark = -1;
+ } else {
+ throw new IOException("No mark defined");
+ }
+ }
+
+ @Override
+ public int available() {
+ return buffer.remaining();
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/stream/in/MultiByteBufferInputStream.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/stream/in/MultiByteBufferInputStream.java
new file mode 100644
index 0000000..31f8179
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/stream/in/MultiByteBufferInputStream.java
@@ -0,0 +1,303 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.column.bytes.stream.in;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Queue;
+
+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.storage.am.lsm.btree.column.api.IColumnBufferProvider;
+
+public final class MultiByteBufferInputStream extends AbstractBytesInputStream {
+ private static final ByteBuffer EMPTY;
+
+ static {
+ EMPTY = ByteBuffer.allocate(0);
+ EMPTY.limit(0);
+ }
+
+ private final Queue<ByteBuffer> buffers;
+ private final ArrayBackedValueStorage tempPointableStorage;
+ private int length;
+
+ private ByteBuffer current;
+ private int position;
+
+ public MultiByteBufferInputStream() {
+ this.buffers = new ArrayDeque<>();
+ tempPointableStorage = new ArrayBackedValueStorage();
+ this.current = EMPTY;
+ this.position = 0;
+ this.length = 0;
+
+ }
+
+ private MultiByteBufferInputStream(MultiByteBufferInputStream original, int len) throws EOFException {
+ buffers = new ArrayDeque<>();
+ tempPointableStorage = new ArrayBackedValueStorage();
+ position = original.position;
+ length = original.length;
+ buffers.addAll(original.sliceBuffers(len));
+ nextBuffer();
+ }
+
+ @Override
+ public void reset(IColumnBufferProvider bufferProvider) throws HyracksDataException {
+ reset();
+ length = bufferProvider.getLength();
+ if (length > 0) {
+ bufferProvider.readAll(buffers);
+ nextBuffer();
+ }
+ }
+
+ @Override
+ protected void addBuffer(ByteBuffer buffer) {
+ buffers.add(buffer);
+ length += buffer.remaining();
+ }
+
+ @Override
+ public void resetAt(int bytesToSkip, AbstractBytesInputStream stream) throws IOException {
+ MultiByteBufferInputStream original = (MultiByteBufferInputStream) stream;
+ buffers.clear();
+ position = original.position;
+ length = original.length;
+ current = original.current.duplicate();
+ for (ByteBuffer buffer : original.buffers) {
+ buffers.add(buffer.duplicate());
+ }
+
+ if (skip(bytesToSkip) != bytesToSkip) {
+ throw new EOFException();
+ }
+ }
+
+ @Override
+ public long skip(long n) {
+ if (n <= 0) {
+ return 0;
+ }
+
+ if (current == null) {
+ return -1;
+ }
+
+ long bytesSkipped = 0;
+ while (bytesSkipped < n) {
+ if (current.remaining() > 0) {
+ long bytesToSkip = Math.min(n - bytesSkipped, current.remaining());
+ current.position(current.position() + (int) bytesToSkip);
+ bytesSkipped += bytesToSkip;
+ this.position += bytesToSkip;
+ } else if (!nextBuffer()) {
+ // there are no more buffers
+ return bytesSkipped > 0 ? bytesSkipped : -1;
+ }
+ }
+
+ return bytesSkipped;
+ }
+
+ @Override
+ public int read(ByteBuffer out) {
+ int len = out.remaining();
+ if (len <= 0) {
+ return 0;
+ }
+
+ if (current == null) {
+ return -1;
+ }
+
+ int bytesCopied = 0;
+ while (bytesCopied < len) {
+ if (current.remaining() > 0) {
+ int bytesToCopy;
+ ByteBuffer copyBuffer;
+ if (current.remaining() <= out.remaining()) {
+ // copy all the current buffer
+ bytesToCopy = current.remaining();
+ copyBuffer = current;
+ } else {
+ // copy a slice of the current buffer
+ bytesToCopy = out.remaining();
+ copyBuffer = current.duplicate();
+ copyBuffer.limit(copyBuffer.position() + bytesToCopy);
+ current.position(copyBuffer.position() + bytesToCopy);
+ }
+
+ out.put(copyBuffer);
+ bytesCopied += bytesToCopy;
+ this.position += bytesToCopy;
+
+ } else if (!nextBuffer()) {
+ // there are no more buffers
+ return bytesCopied > 0 ? bytesCopied : -1;
+ }
+ }
+
+ return bytesCopied;
+ }
+
+ @Override
+ public AbstractBytesInputStream sliceStream(int length) throws EOFException {
+ return new MultiByteBufferInputStream(this, length);
+ }
+
+ @Override
+ public AbstractBytesInputStream remainingStream() throws EOFException {
+ return new MultiByteBufferInputStream(this, length - position);
+ }
+
+ @Override
+ public int read(byte[] bytes, int off, int len) {
+ if (len <= 0) {
+ if (len < 0) {
+ throw new IndexOutOfBoundsException("Read length must be greater than 0: " + len);
+ }
+ return 0;
+ }
+
+ if (current == null) {
+ return -1;
+ }
+
+ int bytesRead = 0;
+ while (bytesRead < len) {
+ if (current.remaining() > 0) {
+ int bytesToRead = Math.min(len - bytesRead, current.remaining());
+ current.get(bytes, off + bytesRead, bytesToRead);
+ bytesRead += bytesToRead;
+ this.position += bytesToRead;
+ } else if (!nextBuffer()) {
+ // there are no more buffers
+ return bytesRead > 0 ? bytesRead : -1;
+ }
+ }
+
+ return bytesRead;
+ }
+
+ @Override
+ public int read(byte[] bytes) {
+ return read(bytes, 0, bytes.length);
+ }
+
+ @Override
+ public int read() throws IOException {
+ if (current == null) {
+ throw new EOFException();
+ }
+
+ while (true) {
+ if (current.remaining() > 0) {
+ this.position += 1;
+ return current.get() & 0xFF; // as unsigned
+ } else if (!nextBuffer()) {
+ // there are no more buffers
+ throw new EOFException();
+ }
+ }
+ }
+
+ @Override
+ public void read(IPointable pointable, int length) throws EOFException {
+ if (current.remaining() >= length) {
+ pointable.set(current.array(), current.position(), length);
+ current.position(current.position() + length);
+ position += length;
+ } else {
+ tempPointableStorage.setSize(length);
+ //Read first half part from the current buffer
+ int bytesRead = read(tempPointableStorage.getByteArray(), 0, length);
+ if (bytesRead != length) {
+ throw new EOFException();
+ }
+ pointable.set(tempPointableStorage);
+ }
+ }
+
+ @Override
+ public int available() {
+ return length - position;
+ }
+
+ @Override
+ public void mark(int readLimit) {
+ throw new UnsupportedOperationException("reset() is not supported");
+ }
+
+ @Override
+ public void reset() {
+ buffers.clear();
+ this.current = EMPTY;
+ this.position = 0;
+ this.length = 0;
+ }
+
+ private List<ByteBuffer> sliceBuffers(long length) throws EOFException {
+ if (length <= 0) {
+ return Collections.emptyList();
+ }
+
+ if (current == null) {
+ throw new EOFException();
+ }
+
+ List<ByteBuffer> sliceBuffers = new ArrayList<>();
+ long bytesAccumulated = 0;
+ while (bytesAccumulated < length) {
+ if (current.remaining() > 0) {
+ // get a slice of the current buffer to return
+ // always fits in an int because remaining returns an int that is >= 0
+ int bufLen = (int) Math.min(length - bytesAccumulated, current.remaining());
+ ByteBuffer slice = current.duplicate();
+ slice.limit(slice.position() + bufLen);
+ sliceBuffers.add(slice);
+ bytesAccumulated += bufLen;
+
+ // update state; the bytes are considered read
+ current.position(current.position() + bufLen);
+ this.position += bufLen;
+ } else if (!nextBuffer()) {
+ // there are no more buffers
+ throw new EOFException();
+ }
+ }
+
+ return sliceBuffers;
+ }
+
+ private boolean nextBuffer() {
+ if (buffers.isEmpty()) {
+ return false;
+ }
+ current = buffers.poll();
+ return true;
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/stream/out/AbstractBytesOutputStream.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/stream/out/AbstractBytesOutputStream.java
new file mode 100644
index 0000000..698eac4
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/stream/out/AbstractBytesOutputStream.java
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.column.bytes.stream.out;
+
+import java.io.IOException;
+import java.io.OutputStream;
+
+import org.apache.asterix.column.bytes.stream.out.pointer.IReservedPointer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.column.values.ValuesWriter;
+
+/**
+ * Extends {@link OutputStream} to include methods needed by {@link ValuesWriter}
+ */
+public abstract class AbstractBytesOutputStream extends OutputStream {
+ private final ParquetBytesInput bytesInput;
+
+ protected AbstractBytesOutputStream() {
+ bytesInput = new ParquetBytesInput(this);
+ }
+
+ @Override
+ public abstract void write(int b) throws IOException;
+
+ @Override
+ public final void write(byte[] b) throws IOException {
+ write(b, 0, b.length);
+ }
+
+ @Override
+ public abstract void write(byte[] b, int off, int len) throws IOException;
+
+ public final void write(IValueReference value) throws IOException {
+ write(value.getByteArray(), value.getStartOffset(), value.getLength());
+ }
+
+ public final BytesInput asBytesInput() {
+ return bytesInput;
+ }
+
+ public abstract void finish();
+
+ /**
+ * Reset output stream
+ */
+ public abstract void reset() throws HyracksDataException;
+
+ /**
+ * Reserve a byte at the current position of the stream
+ *
+ * @param pointer pointer that references the current position
+ */
+ public abstract void reserveByte(IReservedPointer pointer) throws IOException;
+
+ /**
+ * Reserve an integer at the current position of the stream
+ *
+ * @param pointer pointer that references the current position
+ */
+ public abstract void reserveInteger(IReservedPointer pointer) throws IOException;
+
+ /**
+ * @return a reusable instance of {@link IReservedPointer}
+ */
+ public abstract IReservedPointer createPointer();
+
+ /**
+ * @return Size of written value
+ */
+ public abstract int size();
+
+ /**
+ * @return Allocated buffer size
+ */
+ public abstract int capacity();
+
+ /**
+ * Write the content to another output stream
+ *
+ * @param outputStream output stream to write to
+ */
+ public abstract void writeTo(OutputStream outputStream) throws IOException;
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/stream/out/AbstractMultiBufferBytesOutputStream.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/stream/out/AbstractMultiBufferBytesOutputStream.java
new file mode 100644
index 0000000..4b7c835
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/stream/out/AbstractMultiBufferBytesOutputStream.java
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.column.bytes.stream.out;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.asterix.column.bytes.stream.out.pointer.ByteBufferReservedPointer;
+import org.apache.asterix.column.bytes.stream.out.pointer.IReservedPointer;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnWriteMultiPageOp;
+
+abstract class AbstractMultiBufferBytesOutputStream extends AbstractBytesOutputStream {
+ protected final Mutable<IColumnWriteMultiPageOp> multiPageOpRef;
+ protected final List<ByteBuffer> buffers;
+ protected int currentBufferIndex;
+ protected int allocatedBytes;
+ protected int position;
+ protected ByteBuffer currentBuf;
+
+ AbstractMultiBufferBytesOutputStream(Mutable<IColumnWriteMultiPageOp> multiPageOpRef) {
+ this.multiPageOpRef = multiPageOpRef;
+ buffers = new ArrayList<>();
+ }
+
+ protected abstract ByteBuffer confiscateNewBuffer() throws HyracksDataException;
+
+ protected abstract void preReset() throws HyracksDataException;
+
+ @Override
+ public final void reset() throws HyracksDataException {
+ preReset();
+ position = 0;
+ currentBufferIndex = 0;
+ if (allocatedBytes == 0) {
+ allocateBuffer();
+ }
+ currentBufferIndex = 0;
+ currentBuf = buffers.get(0);
+ currentBuf.clear();
+ }
+
+ @Override
+ public final void write(int b) throws IOException {
+ ensureCapacity(1);
+ currentBuf.put((byte) b);
+ position++;
+ }
+
+ @Override
+ public final void write(byte[] b, int off, int len) throws IOException {
+ ensureCapacity(len);
+ int remaining = len;
+ int offset = off;
+ while (remaining > 0) {
+ setNextBufferIfNeeded();
+ int writeLength = Math.min(remaining, currentBuf.remaining());
+ currentBuf.put(b, offset, writeLength);
+ position += writeLength;
+ offset += writeLength;
+ remaining -= writeLength;
+ }
+ }
+
+ @Override
+ public void reserveByte(IReservedPointer pointer) throws IOException {
+ ensureCapacity(Byte.BYTES);
+ int offset = getCurrentBufferPosition();
+ currentBuf.put((byte) 0);
+ position += 1;
+ ((ByteBufferReservedPointer) pointer).setPointer(currentBuf, offset);
+ }
+
+ @Override
+ public final void reserveInteger(IReservedPointer pointer) throws HyracksDataException {
+ ensureCapacity(Integer.BYTES);
+ int offset = getCurrentBufferPosition();
+ currentBuf.putInt(0);
+ position += Integer.BYTES;
+ ((ByteBufferReservedPointer) pointer).setPointer(currentBuf, offset);
+ }
+
+ @Override
+ public final IReservedPointer createPointer() {
+ return new ByteBufferReservedPointer();
+ }
+
+ public final int getCurrentBufferPosition() {
+ return currentBuf.position();
+ }
+
+ @Override
+ public final int size() {
+ return position;
+ }
+
+ @Override
+ public final int capacity() {
+ return allocatedBytes;
+ }
+
+ @Override
+ public final void finish() {
+ currentBuf = null;
+ buffers.clear();
+ allocatedBytes = 0;
+ }
+
+ /* *************************************************
+ * Helper methods
+ * *************************************************
+ */
+
+ private void ensureCapacity(int length) throws HyracksDataException {
+ if (position + length > allocatedBytes) {
+ allocateMoreBuffers(length);
+ } else if (length > 0) {
+ setNextBufferIfNeeded();
+ }
+ }
+
+ private void allocateMoreBuffers(int length) throws HyracksDataException {
+ int neededSpace = length - currentBuf.remaining();
+ while (neededSpace > 0) {
+ neededSpace -= allocateBuffer();
+ }
+ setNextBufferIfNeeded();
+ }
+
+ private void setNextBufferIfNeeded() {
+ if (currentBuf.remaining() == 0) {
+ currentBuf = buffers.get(++currentBufferIndex);
+ currentBuf.clear();
+ }
+ }
+
+ private int allocateBuffer() throws HyracksDataException {
+ ByteBuffer buffer = confiscateNewBuffer();
+ buffers.add(buffer);
+ buffer.clear();
+ int size = buffer.capacity();
+ allocatedBytes += size;
+ return size;
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/stream/out/ByteBufferOutputStream.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/stream/out/ByteBufferOutputStream.java
new file mode 100644
index 0000000..8817ae6
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/stream/out/ByteBufferOutputStream.java
@@ -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.
+ */
+package org.apache.asterix.column.bytes.stream.out;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+
+public final class ByteBufferOutputStream extends OutputStream {
+ private ByteBuffer buffer;
+ private int startOffset;
+
+ public void reset(ByteBuffer buffer) {
+ this.buffer = buffer;
+ startOffset = buffer.position();
+ }
+
+ public int size() {
+ return buffer.position() - startOffset;
+ }
+
+ @Override
+ public void write(int b) throws IOException {
+ buffer.put((byte) b);
+ }
+
+ @Override
+ public void write(byte[] b, int off, int len) throws IOException {
+ buffer.put(b, off, len);
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/stream/out/GrowableBytesOutputStream.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/stream/out/GrowableBytesOutputStream.java
new file mode 100644
index 0000000..20daf7d
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/stream/out/GrowableBytesOutputStream.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.column.bytes.stream.out;
+
+import java.io.IOException;
+import java.io.OutputStream;
+
+import org.apache.asterix.column.bytes.stream.out.pointer.GrowableBytesPointer;
+import org.apache.asterix.column.bytes.stream.out.pointer.IReservedPointer;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+
+public final class GrowableBytesOutputStream extends AbstractBytesOutputStream {
+ private final ArrayBackedValueStorage storage;
+
+ public GrowableBytesOutputStream() {
+ storage = new ArrayBackedValueStorage(128);
+ }
+
+ @Override
+ public void write(int b) throws IOException {
+ storage.getDataOutput().write(b);
+ }
+
+ @Override
+ public void write(byte[] b, int off, int len) throws IOException {
+ storage.getDataOutput().write(b, off, len);
+ }
+
+ @Override
+ public void finish() {
+ reset();
+ }
+
+ @Override
+ public void reset() {
+ storage.reset();
+ }
+
+ @Override
+ public void reserveByte(IReservedPointer pointer) throws IOException {
+ ((GrowableBytesPointer) pointer).setPointer(storage.getLength());
+ storage.getDataOutput().write(0);
+ }
+
+ @Override
+ public void reserveInteger(IReservedPointer pointer) throws IOException {
+ ((GrowableBytesPointer) pointer).setPointer(storage.getLength());
+ storage.getDataOutput().writeInt(0);
+ }
+
+ @Override
+ public IReservedPointer createPointer() {
+ return new GrowableBytesPointer(storage);
+ }
+
+ @Override
+ public int size() {
+ return storage.getLength();
+ }
+
+ @Override
+ public int capacity() {
+ return storage.getByteArray().length;
+ }
+
+ @Override
+ public void writeTo(OutputStream outputStream) throws IOException {
+ outputStream.write(storage.getByteArray(), storage.getStartOffset(), storage.getLength());
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/stream/out/MultiPersistentBufferBytesOutputStream.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/stream/out/MultiPersistentBufferBytesOutputStream.java
new file mode 100644
index 0000000..c910131
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/stream/out/MultiPersistentBufferBytesOutputStream.java
@@ -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.
+ */
+package org.apache.asterix.column.bytes.stream.out;
+
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnWriteMultiPageOp;
+
+public final class MultiPersistentBufferBytesOutputStream extends AbstractMultiBufferBytesOutputStream {
+ public MultiPersistentBufferBytesOutputStream(Mutable<IColumnWriteMultiPageOp> multiPageOpRef) {
+ super(multiPageOpRef);
+ }
+
+ @Override
+ protected ByteBuffer confiscateNewBuffer() throws HyracksDataException {
+ return multiPageOpRef.getValue().confiscatePersistent();
+ }
+
+ @Override
+ protected void preReset() throws HyracksDataException {
+ if (allocatedBytes > 0) {
+ //Persist all buffers before resetting the stream
+ multiPageOpRef.getValue().persist();
+ allocatedBytes = 0;
+ buffers.clear();
+ }
+ }
+
+ @Override
+ public void writeTo(OutputStream outputStream) {
+ throw new IllegalAccessError("Persistent stream cannot be written to other stream");
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/stream/out/MultiTemporaryBufferBytesOutputStream.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/stream/out/MultiTemporaryBufferBytesOutputStream.java
new file mode 100644
index 0000000..cf2808e
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/stream/out/MultiTemporaryBufferBytesOutputStream.java
@@ -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.
+ */
+package org.apache.asterix.column.bytes.stream.out;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnWriteMultiPageOp;
+
+public final class MultiTemporaryBufferBytesOutputStream extends AbstractMultiBufferBytesOutputStream {
+ public MultiTemporaryBufferBytesOutputStream(Mutable<IColumnWriteMultiPageOp> multiPageOpRef) {
+ super(multiPageOpRef);
+ }
+
+ @Override
+ protected void preReset() {
+ //NoOp
+ }
+
+ @Override
+ protected ByteBuffer confiscateNewBuffer() throws HyracksDataException {
+ return multiPageOpRef.getValue().confiscateTemporary();
+ }
+
+ @Override
+ public void writeTo(OutputStream outputStream) throws IOException {
+ int writtenSize = 0;
+ for (int i = 0; i < currentBufferIndex + 1; i++) {
+ ByteBuffer buffer = buffers.get(i);
+ outputStream.write(buffer.array(), 0, buffer.position());
+ writtenSize += buffer.position();
+ }
+ if (writtenSize != position) {
+ //Sanity check
+ throw new IllegalStateException("Size is different");
+ }
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/stream/out/ParquetBytesInput.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/stream/out/ParquetBytesInput.java
new file mode 100644
index 0000000..c5ad38e
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/stream/out/ParquetBytesInput.java
@@ -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.
+ */
+package org.apache.asterix.column.bytes.stream.out;
+
+import java.io.IOException;
+import java.io.OutputStream;
+
+import org.apache.asterix.column.bytes.encoder.ParquetDeltaBinaryPackingValuesWriterForLong;
+import org.apache.parquet.bytes.BytesInput;
+
+/**
+ * A wrapper for {@link BytesInput} which is used to concatenate multiple {@link AbstractBytesOutputStream}
+ *
+ * @see ParquetDeltaBinaryPackingValuesWriterForLong#getBytes() as an example
+ */
+class ParquetBytesInput extends BytesInput {
+ private final AbstractBytesOutputStream outputStream;
+
+ ParquetBytesInput(AbstractBytesOutputStream outputStream) {
+ this.outputStream = outputStream;
+ }
+
+ @Override
+ public final void writeAllTo(OutputStream outputStream) throws IOException {
+ this.outputStream.writeTo(outputStream);
+ }
+
+ @Override
+ public final long size() {
+ return outputStream.size();
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/stream/out/pointer/ByteBufferReservedPointer.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/stream/out/pointer/ByteBufferReservedPointer.java
new file mode 100644
index 0000000..8773a31
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/stream/out/pointer/ByteBufferReservedPointer.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.column.bytes.stream.out.pointer;
+
+import java.nio.ByteBuffer;
+
+public class ByteBufferReservedPointer implements IReservedPointer {
+ private ByteBuffer buffer;
+ private int offset;
+
+ public void setPointer(ByteBuffer buffer, int offset) {
+ this.buffer = buffer;
+ this.offset = offset;
+ }
+
+ @Override
+ public void setByte(byte value) {
+ buffer.put(offset, value);
+ }
+
+ @Override
+ public void setInteger(int value) {
+ buffer.putInt(offset, value);
+ }
+
+ @Override
+ public void reset() {
+ buffer = null;
+ }
+
+ @Override
+ public boolean isSet() {
+ return buffer != null;
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/stream/out/pointer/GrowableBytesPointer.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/stream/out/pointer/GrowableBytesPointer.java
new file mode 100644
index 0000000..0863c72
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/stream/out/pointer/GrowableBytesPointer.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.column.bytes.stream.out.pointer;
+
+import org.apache.hyracks.data.std.primitive.IntegerPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+
+public class GrowableBytesPointer implements IReservedPointer {
+ private final ArrayBackedValueStorage storage;
+ private int offset;
+
+ public GrowableBytesPointer(ArrayBackedValueStorage storage) {
+ this.storage = storage;
+ }
+
+ public void setPointer(int offset) {
+ this.offset = offset;
+ }
+
+ @Override
+ public void setByte(byte value) {
+ storage.getByteArray()[offset] = value;
+ }
+
+ @Override
+ public void setInteger(int value) {
+ IntegerPointable.setInteger(storage.getByteArray(), offset, value);
+ }
+
+ @Override
+ public void reset() {
+ offset = -1;
+ }
+
+ @Override
+ public boolean isSet() {
+ return offset >= 0;
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/stream/out/pointer/IReservedPointer.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/stream/out/pointer/IReservedPointer.java
new file mode 100644
index 0000000..46c4d36
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/bytes/stream/out/pointer/IReservedPointer.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.column.bytes.stream.out.pointer;
+
+import org.apache.asterix.column.bytes.stream.out.AbstractBytesOutputStream;
+
+/**
+ * Pointer that reference a position in {@link AbstractBytesOutputStream}
+ */
+public interface IReservedPointer {
+ /**
+ * Set a byte value at the pointer's position
+ *
+ * @param value byte value to be set
+ */
+ void setByte(byte value);
+
+ /**
+ * Set an integer value at the pointer's position
+ *
+ * @param value integer value to be set
+ */
+ void setInteger(int value);
+
+ /**
+ * Reset the pointer
+ */
+ void reset();
+
+ /**
+ * @return whether the pointer is set or not
+ */
+ boolean isSet();
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/AbstractColumnImmutableMetadata.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/AbstractColumnImmutableMetadata.java
new file mode 100644
index 0000000..c7b4651
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/AbstractColumnImmutableMetadata.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.column.metadata;
+
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+
+public abstract class AbstractColumnImmutableMetadata extends AbstractColumnMetadata {
+ protected final IValueReference serializedMetadata;
+ protected final int numberOfColumns;
+
+ protected AbstractColumnImmutableMetadata(ARecordType datasetType, ARecordType metaType, int numberOfPrimaryKeys,
+ IValueReference serializedMetadata, int numberOfColumns) {
+ super(datasetType, metaType, numberOfPrimaryKeys);
+ this.serializedMetadata = serializedMetadata;
+ this.numberOfColumns = numberOfColumns;
+ }
+
+ @Override
+ public final IValueReference serializeColumnsMetadata() {
+ return serializedMetadata;
+ }
+
+ @Override
+ public final void abort() throws HyracksDataException {
+ //NoOp as the metadata is immutable
+ }
+
+ @Override
+ public int getNumberOfColumns() {
+ return numberOfColumns;
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/AbstractColumnImmutableReadMetadata.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/AbstractColumnImmutableReadMetadata.java
new file mode 100644
index 0000000..5ac38d7
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/AbstractColumnImmutableReadMetadata.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.column.metadata;
+
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.AbstractColumnTupleReader;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.projection.IColumnProjectionInfo;
+
+public abstract class AbstractColumnImmutableReadMetadata extends AbstractColumnImmutableMetadata
+ implements IColumnProjectionInfo {
+ protected AbstractColumnImmutableReadMetadata(ARecordType datasetType, ARecordType metaType,
+ int numberOfPrimaryKeys, IValueReference serializedMetadata, int numberOfColumns) {
+ super(datasetType, metaType, numberOfPrimaryKeys, serializedMetadata, numberOfColumns);
+ }
+
+ /**
+ * @return the corresponding reader (merge reader or query reader) given <code>this</code> metadata
+ */
+ public abstract AbstractColumnTupleReader createTupleReader();
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/AbstractColumnMetadata.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/AbstractColumnMetadata.java
new file mode 100644
index 0000000..4e19cbc
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/AbstractColumnMetadata.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.column.metadata;
+
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnMetadata;
+
+public abstract class AbstractColumnMetadata implements IColumnMetadata {
+ protected static final int WRITERS_POINTER = 0;
+ protected static final int FIELD_NAMES_POINTER = WRITERS_POINTER + Integer.BYTES;
+ protected static final int SCHEMA_POINTER = FIELD_NAMES_POINTER + Integer.BYTES;
+ protected static final int META_SCHEMA_POINTER = SCHEMA_POINTER + Integer.BYTES;
+ protected static final int PATH_INFO_POINTER = META_SCHEMA_POINTER + Integer.BYTES;
+ protected static final int OFFSETS_SIZE = PATH_INFO_POINTER + Integer.BYTES;
+ private final ARecordType datasetType;
+ private final ARecordType metaType;
+
+ private final int numberOfPrimaryKeys;
+ private final int recordFieldIndex;
+
+ protected AbstractColumnMetadata(ARecordType datasetType, ARecordType metaType, int numberOfPrimaryKeys) {
+ this.datasetType = datasetType;
+ this.metaType = metaType;
+ this.numberOfPrimaryKeys = numberOfPrimaryKeys;
+ this.recordFieldIndex = numberOfPrimaryKeys;
+ }
+
+ public final ARecordType getDatasetType() {
+ return datasetType;
+ }
+
+ public final ARecordType getMetaType() {
+ return metaType;
+ }
+
+ public final int getNumberOfPrimaryKeys() {
+ return numberOfPrimaryKeys;
+ }
+
+ public final int getRecordFieldIndex() {
+ return recordFieldIndex;
+ }
+
+ public final int getMetaRecordFieldIndex() {
+ return recordFieldIndex + 1;
+ }
+
+ public abstract int getNumberOfColumns();
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/FieldNamesDictionary.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/FieldNamesDictionary.java
new file mode 100644
index 0000000..aa2e194
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/FieldNamesDictionary.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.column.metadata;
+
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.AStringSerializerDeserializer;
+import org.apache.asterix.om.base.AMutableString;
+import org.apache.hyracks.api.dataflow.value.IBinaryHashFunction;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.util.string.UTF8StringReader;
+import org.apache.hyracks.util.string.UTF8StringWriter;
+
+import it.unimi.dsi.fastutil.ints.Int2IntMap;
+import it.unimi.dsi.fastutil.ints.Int2IntOpenHashMap;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap;
+
+public class FieldNamesDictionary {
+ //For both declared and inferred fields
+ private final List<IValueReference> fieldNames;
+ private final Object2IntMap<String> declaredFieldNamesToIndexMap;
+ private final Int2IntMap hashToFieldNameIndexMap;
+ private final IBinaryHashFunction fieldNameHashFunction;
+
+ //For declared fields
+ private final AMutableString mutableString;
+ private final AStringSerializerDeserializer stringSerDer;
+
+ //For lookups
+ private final ArrayBackedValueStorage lookupStorage;
+
+ public FieldNamesDictionary() {
+ this(new ArrayList<>(), new Object2IntOpenHashMap<>(), new Int2IntOpenHashMap());
+ }
+
+ private FieldNamesDictionary(List<IValueReference> fieldNames, Object2IntMap<String> declaredFieldNamesToIndexMap,
+ Int2IntMap hashToFieldNameIndexMap) {
+ this.fieldNames = fieldNames;
+ this.declaredFieldNamesToIndexMap = declaredFieldNamesToIndexMap;
+ this.hashToFieldNameIndexMap = hashToFieldNameIndexMap;
+
+ mutableString = new AMutableString("");
+ stringSerDer = new AStringSerializerDeserializer(new UTF8StringWriter(), new UTF8StringReader());
+ fieldNameHashFunction =
+ new PointableBinaryHashFunctionFactory(UTF8StringPointable.FACTORY).createBinaryHashFunction();
+ lookupStorage = new ArrayBackedValueStorage();
+ }
+
+ public List<IValueReference> getFieldNames() {
+ return fieldNames;
+ }
+
+ //TODO solve collision (they're so rare that I haven't seen any)
+ public int getOrCreateFieldNameIndex(IValueReference fieldName) throws HyracksDataException {
+ int hash = getHash(fieldName);
+ if (!hashToFieldNameIndexMap.containsKey(hash)) {
+ int index = addFieldName(creatFieldName(fieldName), hash);
+ hashToFieldNameIndexMap.put(hash, index);
+ return index;
+ }
+ return hashToFieldNameIndexMap.get(hash);
+ }
+
+ public int getOrCreateFieldNameIndex(String fieldName) throws HyracksDataException {
+ if (!declaredFieldNamesToIndexMap.containsKey(fieldName)) {
+ IValueReference serializedFieldName = creatFieldName(fieldName);
+ int hash = getHash(serializedFieldName);
+ int index = addFieldName(serializedFieldName, hash);
+ declaredFieldNamesToIndexMap.put(fieldName, index);
+ return index;
+ }
+ return declaredFieldNamesToIndexMap.getInt(fieldName);
+ }
+
+ public int getFieldNameIndex(String fieldName) throws HyracksDataException {
+ lookupStorage.reset();
+ serializeFieldName(fieldName, lookupStorage);
+ return hashToFieldNameIndexMap.getOrDefault(getHash(lookupStorage), -1);
+ }
+
+ private ArrayBackedValueStorage creatFieldName(IValueReference fieldName) throws HyracksDataException {
+ ArrayBackedValueStorage copy = new ArrayBackedValueStorage(fieldName.getLength());
+ copy.append(fieldName);
+ return copy;
+ }
+
+ private ArrayBackedValueStorage creatFieldName(String fieldName) throws HyracksDataException {
+ ArrayBackedValueStorage serializedFieldName = new ArrayBackedValueStorage();
+ serializeFieldName(fieldName, serializedFieldName);
+ return serializedFieldName;
+ }
+
+ private void serializeFieldName(String fieldName, ArrayBackedValueStorage storage) throws HyracksDataException {
+ mutableString.setValue(fieldName);
+ stringSerDer.serialize(mutableString, storage.getDataOutput());
+ }
+
+ private int getHash(IValueReference fieldName) throws HyracksDataException {
+ byte[] object = fieldName.getByteArray();
+ int start = fieldName.getStartOffset();
+ int length = fieldName.getLength();
+
+ return fieldNameHashFunction.hash(object, start, length);
+ }
+
+ private int addFieldName(IValueReference fieldName, int hash) {
+ int index = fieldNames.size();
+ hashToFieldNameIndexMap.put(hash, index);
+ fieldNames.add(fieldName);
+ return index;
+ }
+
+ public IValueReference getFieldName(int index) {
+ return fieldNames.get(index);
+ }
+
+ public void serialize(DataOutput output) throws IOException {
+ output.writeInt(fieldNames.size());
+ for (IValueReference fieldName : fieldNames) {
+ output.writeInt(fieldName.getLength());
+ output.write(fieldName.getByteArray(), fieldName.getStartOffset(), fieldName.getLength());
+ }
+
+ output.writeInt(declaredFieldNamesToIndexMap.size());
+ for (Object2IntMap.Entry<String> declaredFieldIndex : declaredFieldNamesToIndexMap.object2IntEntrySet()) {
+ output.writeUTF(declaredFieldIndex.getKey());
+ output.writeInt(declaredFieldIndex.getIntValue());
+ }
+
+ for (Int2IntMap.Entry hashIndex : hashToFieldNameIndexMap.int2IntEntrySet()) {
+ output.writeInt(hashIndex.getIntKey());
+ output.writeInt(hashIndex.getIntValue());
+ }
+ }
+
+ public static FieldNamesDictionary deserialize(DataInput input) throws IOException {
+ int numberOfFieldNames = input.readInt();
+
+ List<IValueReference> fieldNames = new ArrayList<>();
+ deserializeFieldNames(input, fieldNames, numberOfFieldNames);
+
+ Object2IntMap<String> declaredFieldNamesToIndexMap = new Object2IntOpenHashMap<>();
+ deserializeDeclaredFieldNames(input, declaredFieldNamesToIndexMap);
+
+ Int2IntMap hashToFieldNameIndexMap = new Int2IntOpenHashMap();
+ deserializeHashToFieldNameIndex(input, hashToFieldNameIndexMap, numberOfFieldNames);
+
+ return new FieldNamesDictionary(fieldNames, declaredFieldNamesToIndexMap, hashToFieldNameIndexMap);
+ }
+
+ public void abort(DataInputStream input) throws IOException {
+ int numberOfFieldNames = input.readInt();
+
+ fieldNames.clear();
+ deserializeFieldNames(input, fieldNames, numberOfFieldNames);
+
+ declaredFieldNamesToIndexMap.clear();
+ deserializeDeclaredFieldNames(input, declaredFieldNamesToIndexMap);
+
+ hashToFieldNameIndexMap.clear();
+ deserializeHashToFieldNameIndex(input, hashToFieldNameIndexMap, numberOfFieldNames);
+ }
+
+ private static void deserializeFieldNames(DataInput input, List<IValueReference> fieldNames, int numberOfFieldNames)
+ throws IOException {
+
+ for (int i = 0; i < numberOfFieldNames; i++) {
+ int length = input.readInt();
+ ArrayBackedValueStorage fieldName = new ArrayBackedValueStorage(length);
+ fieldName.setSize(length);
+ input.readFully(fieldName.getByteArray(), 0, length);
+ fieldNames.add(fieldName);
+ }
+ }
+
+ private static void deserializeDeclaredFieldNames(DataInput input,
+ Object2IntMap<String> declaredFieldNamesToIndexMap) throws IOException {
+ int numberOfDeclaredFieldNames = input.readInt();
+ for (int i = 0; i < numberOfDeclaredFieldNames; i++) {
+ String fieldName = input.readUTF();
+ int fieldNameIndex = input.readInt();
+ declaredFieldNamesToIndexMap.put(fieldName, fieldNameIndex);
+ }
+ }
+
+ private static void deserializeHashToFieldNameIndex(DataInput input, Int2IntMap hashToFieldNameIndexMap,
+ int numberOfFieldNames) throws IOException {
+ for (int i = 0; i < numberOfFieldNames; i++) {
+ int hash = input.readInt();
+ int fieldNameIndex = input.readInt();
+ hashToFieldNameIndexMap.put(hash, fieldNameIndex);
+ }
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/PathInfoSerializer.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/PathInfoSerializer.java
new file mode 100644
index 0000000..f72b77b
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/PathInfoSerializer.java
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.column.metadata;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.asterix.column.metadata.schema.AbstractSchemaNestedNode;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+
+import it.unimi.dsi.fastutil.ints.IntArrayList;
+import it.unimi.dsi.fastutil.ints.IntList;
+
+public class PathInfoSerializer {
+ private final ArrayBackedValueStorage primaryKeyOutputPathStorage;
+ private final ArrayBackedValueStorage pathOutputStorage;
+ private final IntList delimiters;
+ private int level;
+
+ public PathInfoSerializer() {
+ primaryKeyOutputPathStorage = new ArrayBackedValueStorage();
+ pathOutputStorage = new ArrayBackedValueStorage();
+ delimiters = new IntArrayList();
+ level = 0;
+ }
+
+ public void reset() {
+ primaryKeyOutputPathStorage.reset();
+ pathOutputStorage.reset();
+ }
+
+ public void enter(AbstractSchemaNestedNode nestedNode) {
+ if (nestedNode.isCollection()) {
+ delimiters.add(0, level - 1);
+ }
+ if (nestedNode.isObjectOrCollection()) {
+ level++;
+ }
+ }
+
+ public void exit(AbstractSchemaNestedNode nestedNode) {
+ if (nestedNode.isCollection()) {
+ delimiters.removeInt(0);
+ }
+ if (nestedNode.isObjectOrCollection()) {
+ level--;
+ }
+ }
+
+ public void writePathInfo(ATypeTag typeTag, int columnIndex, boolean primaryKey) throws IOException {
+ DataOutput output =
+ primaryKey ? primaryKeyOutputPathStorage.getDataOutput() : pathOutputStorage.getDataOutput();
+ //type tag
+ output.write(typeTag.serialize());
+ //columnIndex
+ output.writeInt(columnIndex);
+ //maxLevel
+ output.writeInt(level);
+ //is primary key
+ output.writeBoolean(primaryKey);
+ //Is collection
+ boolean collection = !delimiters.isEmpty();
+ output.writeBoolean(collection);
+ if (collection) {
+ output.writeInt(delimiters.size());
+ for (int i = 0; i < delimiters.size(); i++) {
+ output.writeInt(delimiters.getInt(i));
+ }
+ }
+ }
+
+ public void serialize(DataOutput output, int numberOfColumns) throws IOException {
+ output.writeInt(numberOfColumns);
+ output.write(primaryKeyOutputPathStorage.getByteArray(), 0, primaryKeyOutputPathStorage.getLength());
+ output.write(pathOutputStorage.getByteArray(), 0, pathOutputStorage.getLength());
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/schema/AbstractSchemaNestedNode.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/schema/AbstractSchemaNestedNode.java
new file mode 100644
index 0000000..187e460
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/schema/AbstractSchemaNestedNode.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.column.metadata.schema;
+
+public abstract class AbstractSchemaNestedNode extends AbstractSchemaNode {
+
+ @Override
+ public final boolean isNested() {
+ return true;
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/schema/AbstractSchemaNode.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/schema/AbstractSchemaNode.java
new file mode 100644
index 0000000..c9d8635
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/schema/AbstractSchemaNode.java
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.column.metadata.schema;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.util.Map;
+
+import org.apache.asterix.column.metadata.PathInfoSerializer;
+import org.apache.asterix.column.metadata.schema.collection.ArraySchemaNode;
+import org.apache.asterix.column.metadata.schema.collection.MultisetSchemaNode;
+import org.apache.asterix.column.metadata.schema.primitive.MissingFieldSchemaNode;
+import org.apache.asterix.column.metadata.schema.primitive.PrimitiveSchemaNode;
+import org.apache.asterix.column.util.RunLengthIntArray;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public abstract class AbstractSchemaNode {
+ private int counter;
+
+ public abstract ATypeTag getTypeTag();
+
+ public abstract boolean isNested();
+
+ public abstract boolean isObjectOrCollection();
+
+ public abstract boolean isCollection();
+
+ public final void incrementCounter() {
+ counter++;
+ }
+
+ public final void setCounter(int counter) {
+ this.counter = counter;
+ }
+
+ public final int getCounter() {
+ return counter;
+ }
+
+ public abstract <R, T> R accept(ISchemaNodeVisitor<R, T> visitor, T arg) throws HyracksDataException;
+
+ public abstract void serialize(DataOutput output, PathInfoSerializer pathInfoSerializer) throws IOException;
+
+ public static AbstractSchemaNode deserialize(DataInput input,
+ Map<AbstractSchemaNestedNode, RunLengthIntArray> definitionLevels) throws IOException {
+ ATypeTag typeTag = ATypeTag.VALUE_TYPE_MAPPING[input.readByte()];
+ switch (typeTag) {
+ case SYSTEM_NULL:
+ return MissingFieldSchemaNode.INSTANCE;
+ case OBJECT:
+ return new ObjectSchemaNode(input, definitionLevels);
+ case ARRAY:
+ return new ArraySchemaNode(input, definitionLevels);
+ case MULTISET:
+ return new MultisetSchemaNode(input, definitionLevels);
+ case UNION:
+ return new UnionSchemaNode(input, definitionLevels);
+ case NULL:
+ case MISSING:
+ case BOOLEAN:
+ case BIGINT:
+ case DOUBLE:
+ case STRING:
+ case UUID:
+ return new PrimitiveSchemaNode(typeTag, input);
+ default:
+ throw new UnsupportedEncodingException(typeTag + " is not supported");
+ }
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/schema/ISchemaNodeVisitor.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/schema/ISchemaNodeVisitor.java
new file mode 100644
index 0000000..4d38156
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/schema/ISchemaNodeVisitor.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.asterix.column.metadata.schema;
+
+import org.apache.asterix.column.metadata.schema.collection.AbstractCollectionSchemaNode;
+import org.apache.asterix.column.metadata.schema.primitive.PrimitiveSchemaNode;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public interface ISchemaNodeVisitor<R, T> {
+ R visit(ObjectSchemaNode objectNode, T arg) throws HyracksDataException;
+
+ R visit(AbstractCollectionSchemaNode collectionNode, T arg) throws HyracksDataException;
+
+ R visit(UnionSchemaNode unionNode, T arg) throws HyracksDataException;
+
+ R visit(PrimitiveSchemaNode primitiveNode, T arg) throws HyracksDataException;
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/schema/ObjectSchemaNode.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/schema/ObjectSchemaNode.java
new file mode 100644
index 0000000..a230e86
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/schema/ObjectSchemaNode.java
@@ -0,0 +1,182 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.column.metadata.schema;
+
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.asterix.column.metadata.PathInfoSerializer;
+import org.apache.asterix.column.metadata.schema.primitive.MissingFieldSchemaNode;
+import org.apache.asterix.column.operation.lsm.flush.FlushColumnMetadata;
+import org.apache.asterix.column.util.RunLengthIntArray;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.util.annotations.CriticalPath;
+
+import it.unimi.dsi.fastutil.ints.Int2IntMap;
+import it.unimi.dsi.fastutil.ints.Int2IntMap.Entry;
+import it.unimi.dsi.fastutil.ints.Int2IntOpenHashMap;
+import it.unimi.dsi.fastutil.ints.IntImmutableList;
+import it.unimi.dsi.fastutil.ints.IntList;
+
+public final class ObjectSchemaNode extends AbstractSchemaNestedNode {
+ private final Int2IntMap fieldNameIndexToChildIndexMap;
+ private final List<AbstractSchemaNode> children;
+
+ public ObjectSchemaNode() {
+ fieldNameIndexToChildIndexMap = new Int2IntOpenHashMap();
+ children = new ArrayList<>();
+ }
+
+ ObjectSchemaNode(DataInput input, Map<AbstractSchemaNestedNode, RunLengthIntArray> definitionLevels)
+ throws IOException {
+ if (definitionLevels != null) {
+ definitionLevels.put(this, new RunLengthIntArray());
+ }
+ int numberOfChildren = input.readInt();
+
+ fieldNameIndexToChildIndexMap = new Int2IntOpenHashMap();
+ deserializeFieldNameIndexToChildIndex(input, fieldNameIndexToChildIndexMap, numberOfChildren);
+
+ children = new ArrayList<>();
+ deserializeChildren(input, children, numberOfChildren, definitionLevels);
+ }
+
+ public AbstractSchemaNode getOrCreateChild(IValueReference fieldName, ATypeTag childTypeTag,
+ FlushColumnMetadata columnMetadata) throws HyracksDataException {
+ int numberOfChildren = children.size();
+ int fieldNameIndex = columnMetadata.getFieldNamesDictionary().getOrCreateFieldNameIndex(fieldName);
+ int childIndex = fieldNameIndexToChildIndexMap.getOrDefault(fieldNameIndex, numberOfChildren);
+ AbstractSchemaNode currentChild = childIndex == numberOfChildren ? null : children.get(childIndex);
+ AbstractSchemaNode newChild = columnMetadata.getOrCreateChild(currentChild, childTypeTag);
+ if (currentChild == null) {
+ children.add(childIndex, newChild);
+ fieldNameIndexToChildIndexMap.put(fieldNameIndex, childIndex);
+ } else if (currentChild != newChild) {
+ children.set(childIndex, newChild);
+ }
+
+ return newChild;
+ }
+
+ public void addChild(int fieldNameIndex, AbstractSchemaNode child) {
+ int childIndex = children.size();
+ fieldNameIndexToChildIndexMap.put(fieldNameIndex, childIndex);
+ children.add(child);
+ }
+
+ public AbstractSchemaNode getChild(int fieldNameIndex) {
+ if (fieldNameIndexToChildIndexMap.containsKey(fieldNameIndex)) {
+ return children.get(fieldNameIndexToChildIndexMap.get(fieldNameIndex));
+ }
+ return MissingFieldSchemaNode.INSTANCE;
+ }
+
+ public void removeChild(int fieldNameIndex) {
+ int childIndex = fieldNameIndexToChildIndexMap.remove(fieldNameIndex);
+ children.remove(childIndex);
+ }
+
+ public List<AbstractSchemaNode> getChildren() {
+ return children;
+ }
+
+ /**
+ * Should not be used in a {@link CriticalPath}
+ */
+ public IntList getChildrenFieldNameIndexes() {
+ return IntImmutableList.toList(fieldNameIndexToChildIndexMap.int2IntEntrySet().stream()
+ .sorted(Comparator.comparingInt(Entry::getIntValue)).mapToInt(Entry::getIntKey));
+ }
+
+ public boolean containsField(int fieldNameIndex) {
+ return fieldNameIndexToChildIndexMap.containsKey(fieldNameIndex);
+ }
+
+ @Override
+ public ATypeTag getTypeTag() {
+ return ATypeTag.OBJECT;
+ }
+
+ @Override
+ public boolean isObjectOrCollection() {
+ return true;
+ }
+
+ @Override
+ public boolean isCollection() {
+ return false;
+ }
+
+ @Override
+ public <R, T> R accept(ISchemaNodeVisitor<R, T> visitor, T arg) throws HyracksDataException {
+ return visitor.visit(this, arg);
+ }
+
+ @Override
+ public void serialize(DataOutput output, PathInfoSerializer pathInfoSerializer) throws IOException {
+ output.write(ATypeTag.OBJECT.serialize());
+ output.writeInt(children.size());
+ for (Int2IntMap.Entry fieldNameIndexChildIndex : fieldNameIndexToChildIndexMap.int2IntEntrySet()) {
+ output.writeInt(fieldNameIndexChildIndex.getIntKey());
+ output.writeInt(fieldNameIndexChildIndex.getIntValue());
+ }
+ pathInfoSerializer.enter(this);
+ for (AbstractSchemaNode child : children) {
+ child.serialize(output, pathInfoSerializer);
+ }
+ pathInfoSerializer.exit(this);
+ }
+
+ public void abort(DataInputStream input, Map<AbstractSchemaNestedNode, RunLengthIntArray> definitionLevels)
+ throws IOException {
+ definitionLevels.put(this, new RunLengthIntArray());
+
+ int numberOfChildren = input.readInt();
+
+ fieldNameIndexToChildIndexMap.clear();
+ deserializeFieldNameIndexToChildIndex(input, fieldNameIndexToChildIndexMap, numberOfChildren);
+
+ children.clear();
+ deserializeChildren(input, children, numberOfChildren, definitionLevels);
+ }
+
+ private static void deserializeFieldNameIndexToChildIndex(DataInput input, Int2IntMap fieldNameIndexToChildIndexMap,
+ int numberOfChildren) throws IOException {
+ for (int i = 0; i < numberOfChildren; i++) {
+ int fieldNameIndex = input.readInt();
+ int childIndex = input.readInt();
+ fieldNameIndexToChildIndexMap.put(fieldNameIndex, childIndex);
+ }
+ }
+
+ private static void deserializeChildren(DataInput input, List<AbstractSchemaNode> children, int numberOfChildren,
+ Map<AbstractSchemaNestedNode, RunLengthIntArray> definitionLevels) throws IOException {
+ for (int i = 0; i < numberOfChildren; i++) {
+ children.add(AbstractSchemaNode.deserialize(input, definitionLevels));
+ }
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/schema/UnionSchemaNode.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/schema/UnionSchemaNode.java
new file mode 100644
index 0000000..eba5ac0
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/schema/UnionSchemaNode.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.column.metadata.schema;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.EnumMap;
+import java.util.Map;
+
+import org.apache.asterix.column.metadata.PathInfoSerializer;
+import org.apache.asterix.column.metadata.schema.primitive.MissingFieldSchemaNode;
+import org.apache.asterix.column.metadata.schema.visitor.SchemaClipperVisitor;
+import org.apache.asterix.column.operation.lsm.flush.FlushColumnMetadata;
+import org.apache.asterix.column.util.RunLengthIntArray;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public final class UnionSchemaNode extends AbstractSchemaNestedNode {
+ private final AbstractSchemaNode originalType;
+ private final Map<ATypeTag, AbstractSchemaNode> children;
+
+ public UnionSchemaNode(AbstractSchemaNode child1, AbstractSchemaNode child2) {
+ children = new EnumMap<>(ATypeTag.class);
+ originalType = child1;
+ putChild(child1);
+ putChild(child2);
+ }
+
+ UnionSchemaNode(DataInput input, Map<AbstractSchemaNestedNode, RunLengthIntArray> definitionLevels)
+ throws IOException {
+ if (definitionLevels != null) {
+ definitionLevels.put(this, new RunLengthIntArray());
+ }
+ ATypeTag originalTypeTag = ATypeTag.VALUE_TYPE_MAPPING[input.readByte()];
+ int numberOfChildren = input.readInt();
+ children = new EnumMap<>(ATypeTag.class);
+ for (int i = 0; i < numberOfChildren; i++) {
+ AbstractSchemaNode child = AbstractSchemaNode.deserialize(input, definitionLevels);
+ children.put(child.getTypeTag(), child);
+ }
+ originalType = children.get(originalTypeTag);
+ }
+
+ private void putChild(AbstractSchemaNode child) {
+ children.put(child.getTypeTag(), child);
+ }
+
+ public AbstractSchemaNode getOriginalType() {
+ return originalType;
+ }
+
+ public AbstractSchemaNode getOrCreateChild(ATypeTag childTypeTag, FlushColumnMetadata columnMetadata)
+ throws HyracksDataException {
+ ATypeTag normalizedTypeTag = FlushColumnMetadata.getNormalizedTypeTag(childTypeTag);
+ AbstractSchemaNode currentChild = children.get(normalizedTypeTag);
+ //The parent of a union child should be the actual parent
+ AbstractSchemaNode newChild = columnMetadata.getOrCreateChild(currentChild, normalizedTypeTag);
+ if (currentChild != newChild) {
+ putChild(newChild);
+ }
+ return newChild;
+ }
+
+ public AbstractSchemaNode getChild(ATypeTag typeTag) {
+ return children.getOrDefault(typeTag, MissingFieldSchemaNode.INSTANCE);
+ }
+
+ public Map<ATypeTag, AbstractSchemaNode> getChildren() {
+ return children;
+ }
+
+ @Override
+ public boolean isObjectOrCollection() {
+ return false;
+ }
+
+ @Override
+ public boolean isCollection() {
+ return false;
+ }
+
+ @Override
+ public ATypeTag getTypeTag() {
+ return ATypeTag.UNION;
+ }
+
+ @Override
+ public <R, T> R accept(ISchemaNodeVisitor<R, T> visitor, T arg) throws HyracksDataException {
+ return visitor.visit(this, arg);
+ }
+
+ @Override
+ public void serialize(DataOutput output, PathInfoSerializer pathInfoSerializer) throws IOException {
+ output.write(ATypeTag.UNION.serialize());
+ output.writeByte(originalType.getTypeTag().serialize());
+ output.writeInt(children.size());
+ pathInfoSerializer.enter(this);
+ for (AbstractSchemaNode child : children.values()) {
+ child.serialize(output, pathInfoSerializer);
+ }
+ pathInfoSerializer.exit(this);
+ }
+
+ /**
+ * This would return any numeric node
+ *
+ * @return first numeric node or missing node
+ * @see SchemaClipperVisitor
+ */
+ public AbstractSchemaNode getNumericChildOrMissing() {
+ for (AbstractSchemaNode node : children.values()) {
+ if (ATypeHierarchy.getTypeDomain(node.getTypeTag()) == ATypeHierarchy.Domain.NUMERIC) {
+ return node;
+ }
+ }
+ return MissingFieldSchemaNode.INSTANCE;
+ }
+
+ public int getNumberOfNumericChildren() {
+ int counter = 0;
+ for (AbstractSchemaNode node : children.values()) {
+ if (ATypeHierarchy.getTypeDomain(node.getTypeTag()) == ATypeHierarchy.Domain.NUMERIC) {
+ counter++;
+ }
+ }
+
+ return counter;
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/schema/collection/AbstractCollectionSchemaNode.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/schema/collection/AbstractCollectionSchemaNode.java
new file mode 100644
index 0000000..8455864
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/schema/collection/AbstractCollectionSchemaNode.java
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.column.metadata.schema.collection;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.asterix.column.metadata.PathInfoSerializer;
+import org.apache.asterix.column.metadata.schema.AbstractSchemaNestedNode;
+import org.apache.asterix.column.metadata.schema.AbstractSchemaNode;
+import org.apache.asterix.column.metadata.schema.ISchemaNodeVisitor;
+import org.apache.asterix.column.operation.lsm.flush.FlushColumnMetadata;
+import org.apache.asterix.column.util.RunLengthIntArray;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public abstract class AbstractCollectionSchemaNode extends AbstractSchemaNestedNode {
+ private AbstractSchemaNode item;
+
+ AbstractCollectionSchemaNode() {
+ item = null;
+ }
+
+ AbstractCollectionSchemaNode(DataInput input, Map<AbstractSchemaNestedNode, RunLengthIntArray> definitionLevels)
+ throws IOException {
+ if (definitionLevels != null) {
+ definitionLevels.put(this, new RunLengthIntArray());
+ }
+ item = AbstractSchemaNode.deserialize(input, definitionLevels);
+ }
+
+ public final AbstractSchemaNode getOrCreateItem(ATypeTag childTypeTag, FlushColumnMetadata columnMetadata)
+ throws HyracksDataException {
+ AbstractSchemaNode newItem = columnMetadata.getOrCreateChild(item, childTypeTag);
+ if (newItem != item) {
+ item = newItem;
+ }
+ return item;
+ }
+
+ public final AbstractSchemaNode getItemNode() {
+ return item;
+ }
+
+ public final void setItemNode(AbstractSchemaNode item) {
+ this.item = item;
+ }
+
+ @Override
+ public final <R, T> R accept(ISchemaNodeVisitor<R, T> visitor, T arg) throws HyracksDataException {
+ return visitor.visit(this, arg);
+ }
+
+ @Override
+ public final boolean isObjectOrCollection() {
+ return true;
+ }
+
+ @Override
+ public final boolean isCollection() {
+ return true;
+ }
+
+ @Override
+ public final void serialize(DataOutput output, PathInfoSerializer pathInfoSerializer) throws IOException {
+ output.write(getTypeTag().serialize());
+ pathInfoSerializer.enter(this);
+ item.serialize(output, pathInfoSerializer);
+ pathInfoSerializer.exit(this);
+ }
+
+ public static AbstractCollectionSchemaNode create(ATypeTag typeTag) {
+ if (typeTag == ATypeTag.ARRAY) {
+ return new ArraySchemaNode();
+ }
+
+ return new MultisetSchemaNode();
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/schema/collection/ArraySchemaNode.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/schema/collection/ArraySchemaNode.java
new file mode 100644
index 0000000..084a434
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/schema/collection/ArraySchemaNode.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.column.metadata.schema.collection;
+
+import java.io.DataInput;
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.asterix.column.metadata.schema.AbstractSchemaNestedNode;
+import org.apache.asterix.column.util.RunLengthIntArray;
+import org.apache.asterix.om.types.ATypeTag;
+
+public final class ArraySchemaNode extends AbstractCollectionSchemaNode {
+
+ public ArraySchemaNode() {
+ super();
+ }
+
+ public ArraySchemaNode(DataInput input, Map<AbstractSchemaNestedNode, RunLengthIntArray> definitionLevels)
+ throws IOException {
+ super(input, definitionLevels);
+ }
+
+ @Override
+ public ATypeTag getTypeTag() {
+ return ATypeTag.ARRAY;
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/schema/collection/MultisetSchemaNode.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/schema/collection/MultisetSchemaNode.java
new file mode 100644
index 0000000..af27a5a
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/schema/collection/MultisetSchemaNode.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.asterix.column.metadata.schema.collection;
+
+import java.io.DataInput;
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.asterix.column.metadata.schema.AbstractSchemaNestedNode;
+import org.apache.asterix.column.util.RunLengthIntArray;
+import org.apache.asterix.om.types.ATypeTag;
+
+public final class MultisetSchemaNode extends AbstractCollectionSchemaNode {
+ public MultisetSchemaNode() {
+ super();
+ }
+
+ public MultisetSchemaNode(DataInput input, Map<AbstractSchemaNestedNode, RunLengthIntArray> definitionLevels)
+ throws IOException {
+ super(input, definitionLevels);
+ }
+
+ @Override
+ public ATypeTag getTypeTag() {
+ return ATypeTag.MULTISET;
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/schema/primitive/MissingFieldSchemaNode.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/schema/primitive/MissingFieldSchemaNode.java
new file mode 100644
index 0000000..98f408e
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/schema/primitive/MissingFieldSchemaNode.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.asterix.column.metadata.schema.primitive;
+
+import org.apache.asterix.column.metadata.schema.AbstractSchemaNode;
+import org.apache.asterix.om.types.ATypeTag;
+
+/**
+ * A special schema node a non-existing object or union field
+ */
+public final class MissingFieldSchemaNode extends PrimitiveSchemaNode {
+ public static final AbstractSchemaNode INSTANCE = new MissingFieldSchemaNode();
+
+ private MissingFieldSchemaNode() {
+ super(-1, ATypeTag.MISSING, false);
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/schema/primitive/PrimitiveSchemaNode.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/schema/primitive/PrimitiveSchemaNode.java
new file mode 100644
index 0000000..28d379d
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/schema/primitive/PrimitiveSchemaNode.java
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.column.metadata.schema.primitive;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.asterix.column.metadata.PathInfoSerializer;
+import org.apache.asterix.column.metadata.schema.AbstractSchemaNode;
+import org.apache.asterix.column.metadata.schema.ISchemaNodeVisitor;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class PrimitiveSchemaNode extends AbstractSchemaNode {
+ private final int columnIndex;
+ private final ATypeTag typeTag;
+ private final boolean primaryKey;
+
+ public PrimitiveSchemaNode(int columnIndex, ATypeTag typeTag, boolean primaryKey) {
+ this.columnIndex = columnIndex;
+ this.typeTag = typeTag;
+ this.primaryKey = primaryKey;
+ }
+
+ public PrimitiveSchemaNode(ATypeTag typeTag, DataInput input) throws IOException {
+ this.typeTag = typeTag;
+ columnIndex = input.readInt();
+ primaryKey = input.readBoolean();
+ }
+
+ public final int getColumnIndex() {
+ return columnIndex;
+ }
+
+ @Override
+ public final ATypeTag getTypeTag() {
+ return typeTag;
+ }
+
+ @Override
+ public final boolean isNested() {
+ return false;
+ }
+
+ @Override
+ public final boolean isObjectOrCollection() {
+ return false;
+ }
+
+ @Override
+ public final boolean isCollection() {
+ return false;
+ }
+
+ public final boolean isPrimaryKey() {
+ return primaryKey;
+ }
+
+ @Override
+ public final <R, T> R accept(ISchemaNodeVisitor<R, T> visitor, T arg) throws HyracksDataException {
+ return visitor.visit(this, arg);
+ }
+
+ @Override
+ public void serialize(DataOutput output, PathInfoSerializer pathInfoSerializer) throws IOException {
+ output.write(typeTag.serialize());
+ output.writeInt(columnIndex);
+ output.writeBoolean(primaryKey);
+ pathInfoSerializer.writePathInfo(typeTag, columnIndex, primaryKey);
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/schema/visitor/PathExtractorVisitor.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/schema/visitor/PathExtractorVisitor.java
new file mode 100644
index 0000000..2917074
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/schema/visitor/PathExtractorVisitor.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.column.metadata.schema.visitor;
+
+import org.apache.asterix.column.metadata.schema.AbstractSchemaNode;
+import org.apache.asterix.column.metadata.schema.ISchemaNodeVisitor;
+import org.apache.asterix.column.metadata.schema.ObjectSchemaNode;
+import org.apache.asterix.column.metadata.schema.UnionSchemaNode;
+import org.apache.asterix.column.metadata.schema.collection.AbstractCollectionSchemaNode;
+import org.apache.asterix.column.metadata.schema.primitive.MissingFieldSchemaNode;
+import org.apache.asterix.column.metadata.schema.primitive.PrimitiveSchemaNode;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class PathExtractorVisitor implements ISchemaNodeVisitor<AbstractSchemaNode, Void> {
+ @Override
+ public AbstractSchemaNode visit(ObjectSchemaNode objectNode, Void arg) throws HyracksDataException {
+ int fieldNameIndex = objectNode.getChildrenFieldNameIndexes().getInt(0);
+ if (fieldNameIndex < 0) {
+ return MissingFieldSchemaNode.INSTANCE;
+ }
+ return objectNode.getChild(fieldNameIndex).accept(this, null);
+ }
+
+ @Override
+ public AbstractSchemaNode visit(AbstractCollectionSchemaNode collectionNode, Void arg) throws HyracksDataException {
+ AbstractSchemaNode itemNode = collectionNode.getItemNode();
+ if (itemNode == null) {
+ return MissingFieldSchemaNode.INSTANCE;
+ }
+ return collectionNode.getItemNode().accept(this, null);
+ }
+
+ @Override
+ public AbstractSchemaNode visit(UnionSchemaNode unionNode, Void arg) throws HyracksDataException {
+ for (AbstractSchemaNode node : unionNode.getChildren().values()) {
+ // Using 'for-loop' is the only get the child out of a collection
+ return node.accept(this, null);
+ }
+ return MissingFieldSchemaNode.INSTANCE;
+ }
+
+ @Override
+ public AbstractSchemaNode visit(PrimitiveSchemaNode primitiveNode, Void arg) throws HyracksDataException {
+ //Missing column index is -1
+ return primitiveNode;
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/schema/visitor/SchemaBuilderFromIATypeVisitor.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/schema/visitor/SchemaBuilderFromIATypeVisitor.java
new file mode 100644
index 0000000..fb098fa
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/schema/visitor/SchemaBuilderFromIATypeVisitor.java
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.column.metadata.schema.visitor;
+
+import java.util.List;
+
+import org.apache.asterix.column.metadata.FieldNamesDictionary;
+import org.apache.asterix.column.metadata.schema.AbstractSchemaNode;
+import org.apache.asterix.column.metadata.schema.ObjectSchemaNode;
+import org.apache.asterix.column.metadata.schema.collection.ArraySchemaNode;
+import org.apache.asterix.column.operation.lsm.flush.FlushColumnMetadata;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.AbstractCollectionType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.types.IATypeVisitor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+
+public class SchemaBuilderFromIATypeVisitor implements IATypeVisitor<Void, AbstractSchemaNode> {
+ private final FlushColumnMetadata columnMetadata;
+ private final List<List<String>> primaryKeys;
+ private List<String> currentPrimaryKeyPath;
+ private int processedPrimaryKeys;
+ private int currentPathIndex;
+
+ public SchemaBuilderFromIATypeVisitor(FlushColumnMetadata columnMetadata, List<List<String>> primaryKeys) {
+ this.columnMetadata = columnMetadata;
+ this.primaryKeys = primaryKeys;
+ processedPrimaryKeys = 0;
+ }
+
+ @Override
+ public Void visit(ARecordType recordType, AbstractSchemaNode arg) {
+ ObjectSchemaNode objectNode = (ObjectSchemaNode) arg;
+ columnMetadata.enterLevel(objectNode);
+ try {
+ if (processedPrimaryKeys < primaryKeys.size()) {
+ processPrimaryKeys(recordType, objectNode);
+ }
+ for (int i = 0; i < recordType.getFieldTypes().length; i++) {
+ processField(i, recordType, objectNode);
+ }
+ } catch (HyracksDataException e) {
+ throw new IllegalStateException(e);
+ }
+ columnMetadata.exitLevel(objectNode);
+ return null;
+ }
+
+ @Override
+ public Void visit(AbstractCollectionType collectionType, AbstractSchemaNode arg) {
+ ArraySchemaNode collectionNode = (ArraySchemaNode) arg;
+ IAType itemType = collectionType.getItemType();
+ columnMetadata.enterLevel(collectionNode);
+ try {
+ AbstractSchemaNode itemNode = collectionNode.getOrCreateItem(itemType.getTypeTag(), columnMetadata);
+ itemType.accept(this, itemNode);
+ } catch (HyracksDataException e) {
+ throw new IllegalStateException(e);
+ }
+ columnMetadata.exitLevel(collectionNode);
+ return null;
+ }
+
+ @Override
+ public Void visit(AUnionType unionType, AbstractSchemaNode arg) {
+ throw new IllegalStateException(unionType.getTypeTag() + " is not a declared type");
+ }
+
+ @Override
+ public Void visitFlat(IAType flatType, AbstractSchemaNode arg) {
+ if (processedPrimaryKeys < primaryKeys.size()) {
+ processedPrimaryKeys++;
+ }
+ return null;
+ }
+
+ /*
+ * **************************************************************
+ * Handling primary keys and record fields conversion
+ * **************************************************************
+ */
+ private void processPrimaryKeys(ARecordType recordType, ObjectSchemaNode objectNode) throws HyracksDataException {
+ if (objectNode == columnMetadata.getRoot() || objectNode == columnMetadata.getMetaRoot()) {
+ while (processedPrimaryKeys < primaryKeys.size()) {
+ currentPrimaryKeyPath = primaryKeys.get(processedPrimaryKeys);
+ currentPathIndex = 0;
+ processPrimaryKeyPath(recordType, objectNode);
+ }
+ } else {
+ currentPathIndex++;
+ processPrimaryKeyPath(recordType, objectNode);
+ }
+ }
+
+ private void processPrimaryKeyPath(ARecordType recordType, ObjectSchemaNode objectNode)
+ throws HyracksDataException {
+ int fieldIndex = recordType.getFieldIndex(currentPrimaryKeyPath.get(currentPathIndex));
+ processField(fieldIndex, recordType, objectNode);
+ }
+
+ private void processField(int fieldIndex, ARecordType recordType, ObjectSchemaNode objectNode)
+ throws HyracksDataException {
+ IAType[] fieldTypes = recordType.getFieldTypes();
+ String[] fieldNames = recordType.getFieldNames();
+ FieldNamesDictionary dictionary = columnMetadata.getFieldNamesDictionary();
+
+ int fieldNameIndex = dictionary.getOrCreateFieldNameIndex(fieldNames[fieldIndex]);
+ IValueReference fieldName = dictionary.getFieldName(fieldNameIndex);
+
+ IAType fieldType = fieldTypes[fieldIndex];
+ AbstractSchemaNode child = objectNode.getOrCreateChild(fieldName, fieldType.getTypeTag(), columnMetadata);
+
+ fieldType.accept(this, child);
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/schema/visitor/SchemaClipperVisitor.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/schema/visitor/SchemaClipperVisitor.java
new file mode 100644
index 0000000..0771ccb
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/schema/visitor/SchemaClipperVisitor.java
@@ -0,0 +1,171 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.column.metadata.schema.visitor;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.asterix.column.metadata.FieldNamesDictionary;
+import org.apache.asterix.column.metadata.schema.AbstractSchemaNode;
+import org.apache.asterix.column.metadata.schema.ObjectSchemaNode;
+import org.apache.asterix.column.metadata.schema.UnionSchemaNode;
+import org.apache.asterix.column.metadata.schema.collection.AbstractCollectionSchemaNode;
+import org.apache.asterix.column.metadata.schema.primitive.MissingFieldSchemaNode;
+import org.apache.asterix.column.metadata.schema.primitive.PrimitiveSchemaNode;
+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.AbstractCollectionType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.types.IATypeVisitor;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
+import org.apache.asterix.runtime.projection.FunctionCallInformation;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.Warning;
+
+public class SchemaClipperVisitor implements IATypeVisitor<AbstractSchemaNode, AbstractSchemaNode> {
+ private final FieldNamesDictionary fieldNamesDictionary;
+ private final IWarningCollector warningCollector;
+ private final Map<String, FunctionCallInformation> functionCallInfoMap;
+
+ public SchemaClipperVisitor(FieldNamesDictionary fieldNamesDictionary,
+ Map<String, FunctionCallInformation> functionCallInfoMap, IWarningCollector warningCollector) {
+ this.fieldNamesDictionary = fieldNamesDictionary;
+ this.functionCallInfoMap = functionCallInfoMap;
+ this.warningCollector = warningCollector;
+ }
+
+ @Override
+ public AbstractSchemaNode visit(ARecordType recordType, AbstractSchemaNode arg) {
+ if (isNotCompatible(recordType, arg)) {
+ return MissingFieldSchemaNode.INSTANCE;
+ }
+
+ String[] fieldNames = recordType.getFieldNames();
+ IAType[] fieldTypes = recordType.getFieldTypes();
+ ObjectSchemaNode objectNode = getActualNode(arg, ATypeTag.OBJECT, ObjectSchemaNode.class);
+
+ ObjectSchemaNode clippedObjectNode = new ObjectSchemaNode();
+ try {
+ for (int i = 0; i < fieldNames.length; i++) {
+ int fieldNameIndex = fieldNamesDictionary.getFieldNameIndex(fieldNames[i]);
+ AbstractSchemaNode child = objectNode.getChild(fieldNameIndex);
+ clippedObjectNode.addChild(fieldNameIndex, fieldTypes[i].accept(this, child));
+ }
+ } catch (IOException e) {
+ throw new IllegalStateException(e);
+ }
+
+ return clippedObjectNode;
+ }
+
+ @Override
+ public AbstractSchemaNode visit(AbstractCollectionType collectionType, AbstractSchemaNode arg) {
+ if (isNotCompatible(collectionType, arg)) {
+ return MissingFieldSchemaNode.INSTANCE;
+ }
+ AbstractCollectionSchemaNode collectionNode =
+ getActualNode(arg, collectionType.getTypeTag(), AbstractCollectionSchemaNode.class);
+ AbstractSchemaNode newItemNode = collectionType.getItemType().accept(this, collectionNode.getItemNode());
+ AbstractCollectionSchemaNode clippedCollectionNode =
+ AbstractCollectionSchemaNode.create(collectionType.getTypeTag());
+ clippedCollectionNode.setItemNode(newItemNode);
+ return clippedCollectionNode;
+ }
+
+ @Override
+ public AbstractSchemaNode visit(AUnionType unionType, AbstractSchemaNode arg) {
+ return arg;
+ }
+
+ @Override
+ public AbstractSchemaNode visitFlat(IAType flatType, AbstractSchemaNode arg) {
+ if (flatType.getTypeTag() == ATypeTag.ANY) {
+ return arg;
+ } else if (isNotCompatible(flatType, arg)) {
+ return getNonCompatibleNumericNodeIfAny(flatType, arg);
+ }
+ return getActualNode(arg, flatType.getTypeTag(), PrimitiveSchemaNode.class);
+ }
+
+ private AbstractSchemaNode getNonCompatibleNumericNodeIfAny(IAType flatType, AbstractSchemaNode arg) {
+ ATypeHierarchy.Domain requestedDomain = ATypeHierarchy.getTypeDomain(flatType.getTypeTag());
+ ATypeHierarchy.Domain nodeDomain = ATypeHierarchy.getTypeDomain(arg.getTypeTag());
+ if (nodeDomain == requestedDomain && nodeDomain == ATypeHierarchy.Domain.NUMERIC) {
+ // This will be reconciled by the filter accessor
+ return arg;
+ } else if (arg.getTypeTag() == ATypeTag.UNION) {
+ UnionSchemaNode unionNode = (UnionSchemaNode) arg;
+ return unionNode.getNumericChildOrMissing();
+ }
+
+ return MissingFieldSchemaNode.INSTANCE;
+ }
+
+ private <T extends AbstractSchemaNode> T getActualNode(AbstractSchemaNode node, ATypeTag typeTag, Class<T> clazz) {
+ if (node.getTypeTag() == typeTag) {
+ return clazz.cast(node);
+ } else {
+ //Then it is a union (as we check for incompatibility before we call this method)
+ UnionSchemaNode unionNode = (UnionSchemaNode) node;
+ return clazz.cast(unionNode.getChild(typeTag));
+ }
+ }
+
+ private boolean isNotCompatible(IAType requestedType, AbstractSchemaNode schemaNode) {
+ if (requestedType.getTypeTag() != schemaNode.getTypeTag()) {
+ if (schemaNode.getTypeTag() != ATypeTag.UNION) {
+ warn(requestedType, schemaNode);
+ return true;
+ }
+ // Handle union
+ UnionSchemaNode unionNode = (UnionSchemaNode) schemaNode;
+ return notInUnion(requestedType, unionNode);
+ }
+ return unionContainsMultipleNumeric(schemaNode);
+ }
+
+ private boolean notInUnion(IAType requestedType, UnionSchemaNode unionNode) {
+ for (AbstractSchemaNode unionChildNode : unionNode.getChildren().values()) {
+ warn(requestedType, unionChildNode);
+ }
+ return !unionNode.getChildren().containsKey(requestedType.getTypeTag());
+ }
+
+ private void warn(IAType requestedType, AbstractSchemaNode schemaNode) {
+ if (ATypeHierarchy.isCompatible(requestedType.getTypeTag(), schemaNode.getTypeTag())) {
+ return;
+ }
+ if (warningCollector.shouldWarn()) {
+ Warning warning = functionCallInfoMap.get(requestedType.getTypeName())
+ .createWarning(requestedType.getTypeTag(), schemaNode.getTypeTag());
+ if (warning != null) {
+ warningCollector.warn(warning);
+ }
+ }
+ }
+
+ private boolean unionContainsMultipleNumeric(AbstractSchemaNode schemaNode) {
+ if (schemaNode.getTypeTag() == ATypeTag.UNION) {
+ UnionSchemaNode unionNode = (UnionSchemaNode) schemaNode;
+ return unionNode.getNumberOfNumericChildren() > 1;
+ }
+ return false;
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/flush/BatchFinalizerVisitor.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/flush/BatchFinalizerVisitor.java
new file mode 100644
index 0000000..4cbe09b
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/flush/BatchFinalizerVisitor.java
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.column.operation.lsm.flush;
+
+import java.util.Comparator;
+import java.util.List;
+import java.util.PriorityQueue;
+
+import org.apache.asterix.column.metadata.schema.AbstractSchemaNestedNode;
+import org.apache.asterix.column.metadata.schema.AbstractSchemaNode;
+import org.apache.asterix.column.metadata.schema.ISchemaNodeVisitor;
+import org.apache.asterix.column.metadata.schema.ObjectSchemaNode;
+import org.apache.asterix.column.metadata.schema.UnionSchemaNode;
+import org.apache.asterix.column.metadata.schema.collection.AbstractCollectionSchemaNode;
+import org.apache.asterix.column.metadata.schema.primitive.PrimitiveSchemaNode;
+import org.apache.asterix.column.values.IColumnBatchWriter;
+import org.apache.asterix.column.values.IColumnValuesWriter;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public final class BatchFinalizerVisitor implements ISchemaNodeVisitor<Void, AbstractSchemaNestedNode> {
+ private final FlushColumnMetadata columnSchemaMetadata;
+ private final IColumnValuesWriter[] primaryKeyWriters;
+ private final PriorityQueue<IColumnValuesWriter> orderedColumns;
+ private int level;
+
+ public BatchFinalizerVisitor(FlushColumnMetadata columnSchemaMetadata) {
+ this.columnSchemaMetadata = columnSchemaMetadata;
+ orderedColumns = new PriorityQueue<>(Comparator.comparingInt(x -> -x.getEstimatedSize()));
+ int numberOfPrimaryKeys = columnSchemaMetadata.getNumberOfPrimaryKeys();
+ primaryKeyWriters = new IColumnValuesWriter[numberOfPrimaryKeys];
+ for (int i = 0; i < numberOfPrimaryKeys; i++) {
+ primaryKeyWriters[i] = columnSchemaMetadata.getWriter(i);
+ }
+ level = -1;
+ }
+
+ public int finalizeBatch(IColumnBatchWriter batchWriter, FlushColumnMetadata columnMetadata)
+ throws HyracksDataException {
+ orderedColumns.clear();
+
+ columnMetadata.getRoot().accept(this, null);
+ if (columnMetadata.getMetaRoot() != null) {
+ columnMetadata.getMetaRoot().accept(this, null);
+ }
+
+ int allocatedSpace = batchWriter.writePrimaryKeyColumns(primaryKeyWriters);
+ allocatedSpace += batchWriter.writeColumns(orderedColumns);
+ return allocatedSpace;
+ }
+
+ @Override
+ public Void visit(ObjectSchemaNode objectNode, AbstractSchemaNestedNode arg) throws HyracksDataException {
+ level++;
+ columnSchemaMetadata.flushDefinitionLevels(level, arg, objectNode);
+ List<AbstractSchemaNode> children = objectNode.getChildren();
+ for (int i = 0; i < children.size(); i++) {
+ children.get(i).accept(this, objectNode);
+ }
+ objectNode.setCounter(0);
+ columnSchemaMetadata.clearDefinitionLevels(objectNode);
+ level--;
+ return null;
+ }
+
+ @Override
+ public Void visit(AbstractCollectionSchemaNode collectionNode, AbstractSchemaNestedNode arg)
+ throws HyracksDataException {
+ level++;
+ columnSchemaMetadata.flushDefinitionLevels(level, arg, collectionNode);
+ collectionNode.getItemNode().accept(this, collectionNode);
+ collectionNode.setCounter(0);
+ columnSchemaMetadata.clearDefinitionLevels(collectionNode);
+ level--;
+ return null;
+ }
+
+ @Override
+ public Void visit(UnionSchemaNode unionNode, AbstractSchemaNestedNode arg) throws HyracksDataException {
+ columnSchemaMetadata.flushDefinitionLevels(level, arg, unionNode);
+ for (AbstractSchemaNode node : unionNode.getChildren().values()) {
+ node.accept(this, unionNode);
+ }
+ unionNode.setCounter(0);
+ columnSchemaMetadata.clearDefinitionLevels(unionNode);
+ return null;
+ }
+
+ @Override
+ public Void visit(PrimitiveSchemaNode primitiveNode, AbstractSchemaNestedNode arg) throws HyracksDataException {
+ columnSchemaMetadata.flushDefinitionLevels(level, arg, primitiveNode);
+ if (!primitiveNode.isPrimaryKey()) {
+ orderedColumns.add(columnSchemaMetadata.getWriter(primitiveNode.getColumnIndex()));
+ }
+
+ //Prepare for the next batch
+ primitiveNode.setCounter(0);
+ return null;
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/flush/ColumnTransformer.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/flush/ColumnTransformer.java
new file mode 100644
index 0000000..48cd442
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/flush/ColumnTransformer.java
@@ -0,0 +1,183 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.column.operation.lsm.flush;
+
+import org.apache.asterix.column.metadata.schema.AbstractSchemaNestedNode;
+import org.apache.asterix.column.metadata.schema.AbstractSchemaNode;
+import org.apache.asterix.column.metadata.schema.ObjectSchemaNode;
+import org.apache.asterix.column.metadata.schema.UnionSchemaNode;
+import org.apache.asterix.column.metadata.schema.collection.AbstractCollectionSchemaNode;
+import org.apache.asterix.column.metadata.schema.primitive.PrimitiveSchemaNode;
+import org.apache.asterix.column.util.RunLengthIntArray;
+import org.apache.asterix.column.values.IColumnValuesWriter;
+import org.apache.asterix.om.lazy.AbstractLazyVisitablePointable;
+import org.apache.asterix.om.lazy.AbstractListLazyVisitablePointable;
+import org.apache.asterix.om.lazy.FlatLazyVisitablePointable;
+import org.apache.asterix.om.lazy.ILazyVisitablePointableVisitor;
+import org.apache.asterix.om.lazy.RecordLazyVisitablePointable;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.storage.am.lsm.btree.tuples.LSMBTreeTupleReference;
+
+public class ColumnTransformer implements ILazyVisitablePointableVisitor<AbstractSchemaNode, AbstractSchemaNode> {
+ private final FlushColumnMetadata columnMetadata;
+ private final VoidPointable nonTaggedValue;
+ private final ObjectSchemaNode root;
+ private AbstractSchemaNestedNode currentParent;
+ private int primaryKeysLength;
+
+ public ColumnTransformer(FlushColumnMetadata columnMetadata, ObjectSchemaNode root) {
+ this.columnMetadata = columnMetadata;
+ this.root = root;
+ nonTaggedValue = new VoidPointable();
+ }
+
+ /**
+ * Transform a tuple in row format into columns
+ *
+ * @param pointable record pointable
+ * @return the estimated size (possibly overestimated) of the primary key(s) columns
+ */
+ public int transform(RecordLazyVisitablePointable pointable) throws HyracksDataException {
+ primaryKeysLength = 0;
+ pointable.accept(this, root);
+ return primaryKeysLength;
+ }
+
+ public int writeAntiMatter(LSMBTreeTupleReference tuple) throws HyracksDataException {
+ int pkSize = 0;
+ for (int i = 0; i < columnMetadata.getNumberOfPrimaryKeys(); i++) {
+ byte[] bytes = tuple.getFieldData(i);
+ int start = tuple.getFieldStart(i);
+ ATypeTag tag = ATypeTag.VALUE_TYPE_MAPPING[bytes[start]];
+ nonTaggedValue.set(bytes, start + 1, tuple.getFieldLength(i) - 1);
+ IColumnValuesWriter writer = columnMetadata.getWriter(i);
+ writer.writeAntiMatter(tag, nonTaggedValue);
+ pkSize += writer.getEstimatedSize();
+ }
+ return pkSize;
+ }
+
+ @Override
+ public AbstractSchemaNode visit(RecordLazyVisitablePointable pointable, AbstractSchemaNode arg)
+ throws HyracksDataException {
+ columnMetadata.enterNode(currentParent, arg);
+ AbstractSchemaNestedNode previousParent = currentParent;
+
+ ObjectSchemaNode objectNode = (ObjectSchemaNode) arg;
+ currentParent = objectNode;
+ for (int i = 0; i < pointable.getNumberOfChildren(); i++) {
+ pointable.nextChild();
+ IValueReference fieldName = pointable.getFieldName();
+ ATypeTag childTypeTag = pointable.getChildTypeTag();
+ if (childTypeTag != ATypeTag.MISSING) {
+ //Only write actual field values (including NULL) but ignore MISSING fields
+ AbstractSchemaNode childNode = objectNode.getOrCreateChild(fieldName, childTypeTag, columnMetadata);
+ acceptActualNode(pointable.getChildVisitablePointable(), childNode);
+ }
+ }
+
+ columnMetadata.exitNode(arg);
+ currentParent = previousParent;
+ return null;
+ }
+
+ @Override
+ public AbstractSchemaNode visit(AbstractListLazyVisitablePointable pointable, AbstractSchemaNode arg)
+ throws HyracksDataException {
+ columnMetadata.enterNode(currentParent, arg);
+ AbstractSchemaNestedNode previousParent = currentParent;
+
+ AbstractCollectionSchemaNode collectionNode = (AbstractCollectionSchemaNode) arg;
+ RunLengthIntArray defLevels = columnMetadata.getDefinitionLevels(collectionNode);
+ //the level at which an item is missing
+ int missingLevel = columnMetadata.getLevel();
+ currentParent = collectionNode;
+
+ int numberOfChildren = pointable.getNumberOfChildren();
+ for (int i = 0; i < numberOfChildren; i++) {
+ pointable.nextChild();
+ ATypeTag childTypeTag = pointable.getChildTypeTag();
+ AbstractSchemaNode childNode = collectionNode.getOrCreateItem(childTypeTag, columnMetadata);
+ acceptActualNode(pointable.getChildVisitablePointable(), childNode);
+ /*
+ * The array item may change (e.g., BIGINT --> UNION). Thus, new items would be considered as missing
+ */
+ defLevels.add(missingLevel);
+ }
+
+ columnMetadata.exitCollectionNode(collectionNode, numberOfChildren);
+ currentParent = previousParent;
+ return null;
+ }
+
+ @Override
+ public AbstractSchemaNode visit(FlatLazyVisitablePointable pointable, AbstractSchemaNode arg)
+ throws HyracksDataException {
+ columnMetadata.enterNode(currentParent, arg);
+ ATypeTag valueTypeTag = pointable.getTypeTag();
+ PrimitiveSchemaNode node = (PrimitiveSchemaNode) arg;
+ IColumnValuesWriter writer = columnMetadata.getWriter(node.getColumnIndex());
+ if (valueTypeTag == ATypeTag.MISSING) {
+ writer.writeLevel(columnMetadata.getLevel());
+ } else if (valueTypeTag == ATypeTag.NULL) {
+ writer.writeNull(columnMetadata.getLevel());
+ } else if (pointable.isTagged()) {
+ //Remove type tag
+ nonTaggedValue.set(pointable.getByteArray(), pointable.getStartOffset() + 1, pointable.getLength() - 1);
+ writer.writeValue(pointable.getTypeTag(), nonTaggedValue);
+ } else {
+ writer.writeValue(pointable.getTypeTag(), pointable);
+ }
+ if (node.isPrimaryKey()) {
+ primaryKeysLength += writer.getEstimatedSize();
+ }
+ columnMetadata.exitNode(arg);
+ return null;
+ }
+
+ private void acceptActualNode(AbstractLazyVisitablePointable pointable, AbstractSchemaNode node)
+ throws HyracksDataException {
+ if (node.getTypeTag() == ATypeTag.UNION) {
+ columnMetadata.enterNode(currentParent, node);
+ AbstractSchemaNestedNode previousParent = currentParent;
+
+ UnionSchemaNode unionNode = (UnionSchemaNode) node;
+ currentParent = unionNode;
+
+ ATypeTag childTypeTag = pointable.getTypeTag();
+ AbstractSchemaNode actualNode;
+ if (childTypeTag == ATypeTag.NULL || childTypeTag == ATypeTag.MISSING) {
+ actualNode = unionNode.getOriginalType();
+ } else {
+ actualNode = unionNode.getOrCreateChild(pointable.getTypeTag(), columnMetadata);
+ }
+ pointable.accept(this, actualNode);
+
+ currentParent = previousParent;
+ columnMetadata.exitNode(node);
+ } else if (pointable.getTypeTag() == ATypeTag.NULL && node.isNested()) {
+ columnMetadata.addNestedNull(currentParent, (AbstractSchemaNestedNode) node);
+ } else {
+ pointable.accept(this, node);
+ }
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/flush/FlushColumnMetadata.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/flush/FlushColumnMetadata.java
new file mode 100644
index 0000000..8cd1e98
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/flush/FlushColumnMetadata.java
@@ -0,0 +1,572 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.column.operation.lsm.flush;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.asterix.column.metadata.AbstractColumnMetadata;
+import org.apache.asterix.column.metadata.FieldNamesDictionary;
+import org.apache.asterix.column.metadata.PathInfoSerializer;
+import org.apache.asterix.column.metadata.schema.AbstractSchemaNestedNode;
+import org.apache.asterix.column.metadata.schema.AbstractSchemaNode;
+import org.apache.asterix.column.metadata.schema.ObjectSchemaNode;
+import org.apache.asterix.column.metadata.schema.UnionSchemaNode;
+import org.apache.asterix.column.metadata.schema.collection.AbstractCollectionSchemaNode;
+import org.apache.asterix.column.metadata.schema.collection.ArraySchemaNode;
+import org.apache.asterix.column.metadata.schema.collection.MultisetSchemaNode;
+import org.apache.asterix.column.metadata.schema.primitive.PrimitiveSchemaNode;
+import org.apache.asterix.column.metadata.schema.visitor.SchemaBuilderFromIATypeVisitor;
+import org.apache.asterix.column.util.ColumnValuesUtil;
+import org.apache.asterix.column.util.RunLengthIntArray;
+import org.apache.asterix.column.values.IColumnValuesWriter;
+import org.apache.asterix.column.values.IColumnValuesWriterFactory;
+import org.apache.asterix.column.values.writer.AbstractColumnValuesWriter;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.data.std.primitive.IntegerPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnWriteMultiPageOp;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
+
+import it.unimi.dsi.fastutil.ints.IntArrayList;
+
+/**
+ * Flush column metadata belongs to a flushing {@link ILSMMemoryComponent}
+ * The schema here is mutable and can change according to the flushed records
+ */
+public final class FlushColumnMetadata extends AbstractColumnMetadata {
+ private final Map<AbstractSchemaNestedNode, RunLengthIntArray> definitionLevels;
+ private final Mutable<IColumnWriteMultiPageOp> multiPageOpRef;
+ private final FieldNamesDictionary fieldNamesDictionary;
+ private final ObjectSchemaNode root;
+ private final ObjectSchemaNode metaRoot;
+ private final IColumnValuesWriterFactory columnWriterFactory;
+ private final List<IColumnValuesWriter> columnWriters;
+ private final ArrayBackedValueStorage serializedMetadata;
+ private final PathInfoSerializer pathInfoSerializer;
+ private final IntArrayList nullWriterIndexes;
+ private final boolean metaContainsKeys;
+ private boolean changed;
+ private int level;
+ private int repeated;
+
+ public FlushColumnMetadata(ARecordType datasetType, ARecordType metaType, List<List<String>> primaryKeys,
+ List<Integer> keySourceIndicator, IColumnValuesWriterFactory columnWriterFactory,
+ Mutable<IColumnWriteMultiPageOp> multiPageOpRef) throws HyracksDataException {
+ super(datasetType, metaType, primaryKeys.size());
+ this.multiPageOpRef = multiPageOpRef;
+ this.columnWriterFactory = columnWriterFactory;
+ definitionLevels = new HashMap<>();
+ columnWriters = new ArrayList<>();
+ level = -1;
+ repeated = 0;
+ fieldNamesDictionary = new FieldNamesDictionary();
+ root = new ObjectSchemaNode();
+ metaRoot = metaType != null ? new ObjectSchemaNode() : null;
+ pathInfoSerializer = new PathInfoSerializer();
+ nullWriterIndexes = new IntArrayList();
+ //Add definition levels for the root
+ addDefinitionLevelsAndGet(root);
+ SchemaBuilderFromIATypeVisitor builder = new SchemaBuilderFromIATypeVisitor(this, primaryKeys);
+ //Ensure all primary keys take the first column indexes
+ metaContainsKeys = metaType != null && keySourceIndicator.get(0) == 1;
+ if (metaContainsKeys) {
+ addDefinitionLevelsAndGet(metaRoot);
+ metaType.accept(builder, metaRoot);
+ datasetType.accept(builder, root);
+ } else {
+ datasetType.accept(builder, root);
+ if (metaRoot != null) {
+ addDefinitionLevelsAndGet(metaRoot);
+ metaType.accept(builder, metaRoot);
+ }
+ }
+
+ serializedMetadata = new ArrayBackedValueStorage();
+ changed = true;
+ serializeColumnsMetadata();
+ }
+
+ private FlushColumnMetadata(ARecordType datasetType, ARecordType metaType, List<List<String>> primaryKeys,
+ boolean metaContainsKeys, IColumnValuesWriterFactory columnWriterFactory,
+ Mutable<IColumnWriteMultiPageOp> multiPageOpRef, List<IColumnValuesWriter> columnWriters,
+ FieldNamesDictionary fieldNamesDictionary, ObjectSchemaNode root, ObjectSchemaNode metaRoot,
+ Map<AbstractSchemaNestedNode, RunLengthIntArray> definitionLevels,
+ ArrayBackedValueStorage serializedMetadata) {
+ super(datasetType, metaType, primaryKeys.size());
+ this.multiPageOpRef = multiPageOpRef;
+ this.columnWriterFactory = columnWriterFactory;
+ this.definitionLevels = definitionLevels;
+ this.columnWriters = columnWriters;
+ level = -1;
+ repeated = 0;
+ this.fieldNamesDictionary = fieldNamesDictionary;
+ this.root = root;
+ this.metaRoot = metaRoot;
+ this.metaContainsKeys = metaContainsKeys;
+ pathInfoSerializer = new PathInfoSerializer();
+ nullWriterIndexes = new IntArrayList();
+ //Add definition levels for the root
+ addDefinitionLevelsAndGet(root);
+ this.serializedMetadata = serializedMetadata;
+ changed = false;
+ }
+
+ public FieldNamesDictionary getFieldNamesDictionary() {
+ return fieldNamesDictionary;
+ }
+
+ public ObjectSchemaNode getRoot() {
+ return root;
+ }
+
+ public ObjectSchemaNode getMetaRoot() {
+ return metaRoot;
+ }
+
+ public Mutable<IColumnWriteMultiPageOp> getMultiPageOpRef() {
+ return multiPageOpRef;
+ }
+
+ @Override
+ public IValueReference serializeColumnsMetadata() throws HyracksDataException {
+ if (changed) {
+ try {
+ serializeChanges();
+ changed = false;
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ }
+ return serializedMetadata;
+ }
+
+ private void serializeChanges() throws IOException {
+ serializedMetadata.reset();
+ DataOutput output = serializedMetadata.getDataOutput();
+
+ int writersOffsetPointer = reserveInt(output);
+ int fieldNamesOffsetPointer = reserveInt(output);
+ int schemaOffsetPointer = reserveInt(output);
+ int metaSchemaOffsetPointer = reserveInt(output);
+ int pathInfoOffsetPointer = reserveInt(output);
+
+ //ColumnWriterInformation
+ setOffset(writersOffsetPointer);
+ output.writeInt(columnWriters.size());
+ for (IColumnValuesWriter writer : columnWriters) {
+ writer.serialize(output);
+ }
+
+ //FieldNames
+ setOffset(fieldNamesOffsetPointer);
+ fieldNamesDictionary.serialize(output);
+
+ //Schema
+ pathInfoSerializer.reset();
+ setOffset(schemaOffsetPointer);
+ root.serialize(output, pathInfoSerializer);
+ if (metaRoot != null) {
+ //Meta schema
+ setOffset(metaSchemaOffsetPointer);
+ metaRoot.serialize(output, pathInfoSerializer);
+ }
+
+ //Path info
+ setOffset(pathInfoOffsetPointer);
+ pathInfoSerializer.serialize(output, getNumberOfColumns());
+ }
+
+ private int reserveInt(DataOutput output) throws IOException {
+ int offset = serializedMetadata.getLength();
+ output.writeInt(-1);
+ return offset;
+ }
+
+ private void setOffset(int pointer) {
+ int offset = serializedMetadata.getLength();
+ IntegerPointable.setInteger(serializedMetadata.getByteArray(), pointer, offset);
+ }
+
+ public static FlushColumnMetadata create(ARecordType datasetType, ARecordType metaType,
+ List<List<String>> primaryKeys, List<Integer> keySourceIndicator,
+ IColumnValuesWriterFactory columnWriterFactory, Mutable<IColumnWriteMultiPageOp> multiPageOpRef,
+ IValueReference serializedMetadata) throws HyracksDataException {
+ boolean metaContainsKeys = metaType != null && keySourceIndicator.get(0) == 1;
+ try {
+ return createMutableMetadata(datasetType, metaType, primaryKeys, metaContainsKeys, columnWriterFactory,
+ multiPageOpRef, serializedMetadata);
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ }
+
+ private static FlushColumnMetadata createMutableMetadata(ARecordType datasetType, ARecordType metaType,
+ List<List<String>> primaryKeys, boolean metaContainsKeys, IColumnValuesWriterFactory columnWriterFactory,
+ Mutable<IColumnWriteMultiPageOp> multiPageOpRef, IValueReference serializedMetadata) throws IOException {
+ DataInput input = new DataInputStream(new ByteArrayInputStream(serializedMetadata.getByteArray(),
+ serializedMetadata.getStartOffset(), serializedMetadata.getLength()));
+ //Skip offsets
+ input.skipBytes(OFFSETS_SIZE);
+
+ //ColumnWriter
+ List<IColumnValuesWriter> writers = new ArrayList<>();
+ deserializeWriters(input, writers, columnWriterFactory);
+
+ //FieldNames
+ FieldNamesDictionary fieldNamesDictionary = FieldNamesDictionary.deserialize(input);
+
+ //Schema
+ Map<AbstractSchemaNestedNode, RunLengthIntArray> definitionLevels = new HashMap<>();
+ ObjectSchemaNode root = (ObjectSchemaNode) AbstractSchemaNode.deserialize(input, definitionLevels);
+ ObjectSchemaNode metaRoot = null;
+ if (metaType != null) {
+ metaRoot = (ObjectSchemaNode) AbstractSchemaNode.deserialize(input, definitionLevels);
+ }
+
+ ArrayBackedValueStorage schemaStorage = new ArrayBackedValueStorage(serializedMetadata.getLength());
+ schemaStorage.append(serializedMetadata);
+ return new FlushColumnMetadata(datasetType, metaType, primaryKeys, metaContainsKeys, columnWriterFactory,
+ multiPageOpRef, writers, fieldNamesDictionary, root, metaRoot, definitionLevels, schemaStorage);
+ }
+
+ @Override
+ public void abort() throws HyracksDataException {
+ DataInputStream input = new DataInputStream(new ByteArrayInputStream(serializedMetadata.getByteArray()));
+ try {
+ abort(input);
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ }
+
+ private void abort(DataInputStream input) throws IOException {
+ level = -1;
+ repeated = 0;
+ changed = false;
+
+ columnWriters.clear();
+ deserializeWriters(input, columnWriters, columnWriterFactory);
+
+ fieldNamesDictionary.abort(input);
+ definitionLevels.clear();
+ root.abort(input, definitionLevels);
+ }
+
+ public static void deserializeWriters(DataInput input, List<IColumnValuesWriter> writers,
+ IColumnValuesWriterFactory columnWriterFactory) throws IOException {
+ int numberOfWriters = input.readInt();
+ for (int i = 0; i < numberOfWriters; i++) {
+ writers.add(AbstractColumnValuesWriter.deserialize(input, columnWriterFactory));
+ }
+ }
+
+ /* ********************************************************
+ * Column values related methods
+ * ********************************************************
+ */
+
+ /**
+ * Set {@link IColumnWriteMultiPageOp} for {@link IColumnValuesWriter}
+ *
+ * @param multiPageOp multi-buffer allocator
+ */
+ public void init(IColumnWriteMultiPageOp multiPageOp) throws HyracksDataException {
+ multiPageOpRef.setValue(multiPageOp);
+
+ //Reset writer for the first write
+ for (int i = 0; i < columnWriters.size(); i++) {
+ columnWriters.get(i).reset();
+ }
+ }
+
+ public IColumnValuesWriter getWriter(int columnIndex) {
+ return columnWriters.get(columnIndex);
+ }
+
+ /* ********************************************************
+ * Schema related methods
+ * ********************************************************
+ */
+
+ public int getLevel() {
+ return level;
+ }
+
+ @Override
+ public int getNumberOfColumns() {
+ return columnWriters.size();
+ }
+
+ public AbstractSchemaNode getOrCreateChild(AbstractSchemaNode child, ATypeTag childTypeTag)
+ throws HyracksDataException {
+ AbstractSchemaNode currentChild = child;
+ ATypeTag normalizedTypeTag = getNormalizedTypeTag(childTypeTag);
+ if (currentChild == null || normalizedTypeTag != ATypeTag.MISSING && normalizedTypeTag != ATypeTag.NULL
+ && currentChild.getTypeTag() != ATypeTag.UNION && currentChild.getTypeTag() != normalizedTypeTag) {
+ //Create a new child or union type if required type is different from the current child type
+ currentChild = createChild(child, normalizedTypeTag);
+ //Flag that the schema has changed
+ changed = true;
+ }
+ return currentChild;
+ }
+
+ public void enterLevel(AbstractSchemaNestedNode node) {
+ level++;
+ if (node.isCollection()) {
+ repeated++;
+ }
+ }
+
+ public void exitLevel(AbstractSchemaNestedNode node) {
+ level--;
+ if (node.isCollection()) {
+ repeated--;
+ }
+ }
+
+ public void enterNode(AbstractSchemaNestedNode parent, AbstractSchemaNode node) throws HyracksDataException {
+ //Flush all definition levels from parent to child
+ flushDefinitionLevels(level, parent, node);
+ if (node.isObjectOrCollection()) {
+ //Enter one more level for object, array, and multiset
+ level++;
+ if (node.isCollection()) {
+ //Tells nested values that they are repeated
+ repeated++;
+ }
+ }
+ }
+
+ public void exitNode(AbstractSchemaNode node) {
+ if (node.isNested()) {
+ //Add the nested node's level for all missing children (i.e., not entered for a record)
+ definitionLevels.get((AbstractSchemaNestedNode) node).add(level);
+ if (node.isObjectOrCollection()) {
+ //Union nodes should not change the level as they are logical nodes
+ level--;
+ }
+ }
+ node.incrementCounter();
+ }
+
+ public void exitCollectionNode(AbstractCollectionSchemaNode collectionNode, int numberOfItems) {
+ RunLengthIntArray collectionDefLevels = definitionLevels.get(collectionNode);
+ //Add delimiter
+ collectionDefLevels.add(level - 1);
+ level--;
+ repeated--;
+ collectionNode.incrementCounter();
+ }
+
+ /**
+ * Needed by {@link AbstractCollectionSchemaNode} to add the definition level for each item
+ *
+ * @param collectionSchemaNode collection node
+ * @return collection node's definition level
+ */
+ public RunLengthIntArray getDefinitionLevels(AbstractCollectionSchemaNode collectionSchemaNode) {
+ return definitionLevels.get(collectionSchemaNode);
+ }
+
+ public void clearDefinitionLevels(AbstractSchemaNestedNode nestedNode) {
+ definitionLevels.get(nestedNode).reset();
+ }
+
+ public void flushDefinitionLevels(int level, AbstractSchemaNestedNode parent, AbstractSchemaNode node)
+ throws HyracksDataException {
+ if (parent != null) {
+ RunLengthIntArray parentDefLevels = definitionLevels.get(parent);
+ if (node.getCounter() < parentDefLevels.getSize()) {
+ int parentMask = ColumnValuesUtil.getNullMask(level);
+ int childMask = ColumnValuesUtil.getNullMask(level + 1);
+ flushDefinitionLevels(parentMask, childMask, parentDefLevels, node);
+ }
+ }
+ }
+
+ private void flushDefinitionLevels(int parentMask, int childMask, RunLengthIntArray parentDefLevels,
+ AbstractSchemaNode node) throws HyracksDataException {
+ int startIndex = node.getCounter();
+ if (node.isNested()) {
+ RunLengthIntArray childDefLevels = definitionLevels.get((AbstractSchemaNestedNode) node);
+ flushNestedDefinitionLevel(parentMask, childMask, startIndex, parentDefLevels, childDefLevels);
+ } else {
+ IColumnValuesWriter writer = columnWriters.get(((PrimitiveSchemaNode) node).getColumnIndex());
+ flushWriterDefinitionLevels(parentMask, childMask, startIndex, parentDefLevels, writer);
+ }
+ node.setCounter(parentDefLevels.getSize());
+ }
+
+ private void flushNestedDefinitionLevel(int parentMask, int childMask, int startIndex,
+ RunLengthIntArray parentDefLevels, RunLengthIntArray childDefLevels) {
+ if (parentDefLevels.getSize() == 0) {
+ return;
+ }
+ //First, handle the first block as startIndex might be at the middle of a block
+ //Get which block that startIndex resides
+ int blockIndex = parentDefLevels.getBlockIndex(startIndex);
+ //Get the remaining of the first block starting from startIndex
+ int remainingValues = parentDefLevels.getBlockSize(blockIndex, startIndex);
+
+ int firstBlockValue =
+ ColumnValuesUtil.getChildValue(parentMask, childMask, parentDefLevels.getBlockValue(blockIndex));
+ //Batch add all the remaining values
+ childDefLevels.add(firstBlockValue, remainingValues);
+
+ //Add other blocks as batches
+ for (int i = blockIndex + 1; i < parentDefLevels.getNumberOfBlocks(); i++) {
+ int blockValue = ColumnValuesUtil.getChildValue(parentMask, childMask, parentDefLevels.getBlockValue(i));
+ childDefLevels.add(blockValue, parentDefLevels.getBlockSize(i));
+ }
+ }
+
+ private void flushWriterDefinitionLevels(int parentMask, int childMask, int startIndex,
+ RunLengthIntArray parentDefLevels, IColumnValuesWriter writer) throws HyracksDataException {
+ if (parentDefLevels.getSize() == 0) {
+ return;
+ }
+ /*
+ * We might need only a fraction of the first block. Hence, we first determine how many definition level
+ * values we need. Then, we write those definition levels.
+ */
+ int blockIndex = parentDefLevels.getBlockIndex(startIndex);
+ int remainingValues = parentDefLevels.getBlockSize(blockIndex, startIndex);
+ int firstBlockValue =
+ ColumnValuesUtil.getChildValue(parentMask, childMask, parentDefLevels.getBlockValue(blockIndex));
+ writer.writeLevels(firstBlockValue, remainingValues);
+
+ //Write remaining definition levels from the remaining blocks
+ for (int i = blockIndex + 1; i < parentDefLevels.getNumberOfBlocks(); i++) {
+ int blockValue = ColumnValuesUtil.getChildValue(parentMask, childMask, parentDefLevels.getBlockValue(i));
+ writer.writeLevels(blockValue, parentDefLevels.getBlockSize(i));
+ }
+ }
+
+ private AbstractSchemaNode createChild(AbstractSchemaNode child, ATypeTag normalizedTypeTag)
+ throws HyracksDataException {
+ AbstractSchemaNode createdChild;
+ if (child != null) {
+ if (child.getTypeTag() == ATypeTag.NULL) {
+ //The previous child was a NULL. The new child needs to inherit the NULL definition levels
+ int columnIndex = ((PrimitiveSchemaNode) child).getColumnIndex();
+ RunLengthIntArray defLevels = columnWriters.get(columnIndex).getDefinitionLevelsIntArray();
+ //Add the column index to be garbage collected
+ nullWriterIndexes.add(columnIndex);
+ createdChild = createChild(normalizedTypeTag);
+ int mask = ColumnValuesUtil.getNullMask(level);
+ flushDefinitionLevels(mask, mask, defLevels, createdChild);
+ } else {
+ //Different type. Make union
+ createdChild = addDefinitionLevelsAndGet(new UnionSchemaNode(child, createChild(normalizedTypeTag)));
+ }
+ } else {
+ createdChild = createChild(normalizedTypeTag);
+ }
+ return createdChild;
+ }
+
+ private AbstractSchemaNode createChild(ATypeTag normalizedTypeTag) throws HyracksDataException {
+ switch (normalizedTypeTag) {
+ case OBJECT:
+ return addDefinitionLevelsAndGet(new ObjectSchemaNode());
+ case ARRAY:
+ return addDefinitionLevelsAndGet(new ArraySchemaNode());
+ case MULTISET:
+ return addDefinitionLevelsAndGet(new MultisetSchemaNode());
+ case NULL:
+ case MISSING:
+ case BOOLEAN:
+ case DOUBLE:
+ case BIGINT:
+ case STRING:
+ case UUID:
+ int columnIndex = nullWriterIndexes.isEmpty() ? columnWriters.size() : nullWriterIndexes.removeInt(0);
+ boolean primaryKey = columnIndex < getNumberOfPrimaryKeys();
+ boolean writeAlways = primaryKey || repeated > 0;
+ boolean filtered = !primaryKey;
+ int maxLevel = primaryKey ? 1 : level + 1;
+ IColumnValuesWriter writer = columnWriterFactory.createValueWriter(normalizedTypeTag, columnIndex,
+ maxLevel, writeAlways, filtered);
+ if (multiPageOpRef.getValue() != null) {
+ writer.reset();
+ }
+ addColumn(columnIndex, writer);
+ return new PrimitiveSchemaNode(columnIndex, normalizedTypeTag, primaryKey);
+ default:
+ throw new IllegalStateException("Unsupported type " + normalizedTypeTag);
+
+ }
+ }
+
+ private void addColumn(int index, IColumnValuesWriter writer) {
+ if (index == columnWriters.size()) {
+ columnWriters.add(writer);
+ } else {
+ columnWriters.set(index, writer);
+ }
+ }
+
+ private AbstractSchemaNode addDefinitionLevelsAndGet(AbstractSchemaNestedNode nestedNode) {
+ definitionLevels.put(nestedNode, new RunLengthIntArray());
+ return nestedNode;
+ }
+
+ public static ATypeTag getNormalizedTypeTag(ATypeTag typeTag) {
+ switch (typeTag) {
+ case TINYINT:
+ case SMALLINT:
+ case INTEGER:
+ return ATypeTag.BIGINT;
+ case FLOAT:
+ return ATypeTag.DOUBLE;
+ default:
+ return typeTag;
+ }
+ }
+
+ public void close() {
+ //Dereference multiPageOp
+ multiPageOpRef.setValue(null);
+ for (int i = 0; i < columnWriters.size(); i++) {
+ columnWriters.get(i).close();
+ }
+ }
+
+ public void addNestedNull(AbstractSchemaNestedNode parent, AbstractSchemaNestedNode node)
+ throws HyracksDataException {
+ //Flush all definition levels from parent to the current node
+ flushDefinitionLevels(level, parent, node);
+ //Add null value (+2) to say that both the parent and the child are present
+ definitionLevels.get(node).add(ColumnValuesUtil.getNullMask(level + 2) | level);
+ node.incrementCounter();
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/flush/FlushColumnTupleReaderWriterFactory.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/flush/FlushColumnTupleReaderWriterFactory.java
new file mode 100644
index 0000000..9b1b0a2
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/flush/FlushColumnTupleReaderWriterFactory.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.column.operation.lsm.flush;
+
+import org.apache.asterix.column.metadata.AbstractColumnImmutableReadMetadata;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.AbstractColumnTupleReader;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.AbstractColumnTupleReaderWriterFactory;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.AbstractColumnTupleWriter;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnMetadata;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.projection.IColumnProjectionInfo;
+
+public class FlushColumnTupleReaderWriterFactory extends AbstractColumnTupleReaderWriterFactory {
+ private static final long serialVersionUID = -9197679192729634493L;
+
+ public FlushColumnTupleReaderWriterFactory(int pageSize, int maxNumberOfTuples, float tolerance) {
+ super(pageSize, maxNumberOfTuples, tolerance);
+ }
+
+ @Override
+ public AbstractColumnTupleWriter createColumnWriter(IColumnMetadata columnMetadata) {
+ FlushColumnMetadata flushColumnMetadata = (FlushColumnMetadata) columnMetadata;
+ if (flushColumnMetadata.getMetaType() == null) {
+ //no meta
+ return new FlushColumnTupleWriter(flushColumnMetadata, pageSize, maxNumberOfTuples, tolerance);
+ }
+ return new FlushColumnTupleWithMetaWriter(flushColumnMetadata, pageSize, maxNumberOfTuples, tolerance);
+ }
+
+ @Override
+ public AbstractColumnTupleReader createColumnReader(IColumnProjectionInfo columnProjectionInfo) {
+ return ((AbstractColumnImmutableReadMetadata) columnProjectionInfo).createTupleReader();
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/flush/FlushColumnTupleWithMetaWriter.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/flush/FlushColumnTupleWithMetaWriter.java
new file mode 100644
index 0000000..9c527da
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/flush/FlushColumnTupleWithMetaWriter.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.column.operation.lsm.flush;
+
+import org.apache.asterix.om.lazy.RecordLazyVisitablePointable;
+import org.apache.asterix.om.lazy.TypedRecordLazyVisitablePointable;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.btree.tuples.LSMBTreeTupleReference;
+
+public class FlushColumnTupleWithMetaWriter extends FlushColumnTupleWriter {
+ private final ColumnTransformer metaColumnTransformer;
+ private final RecordLazyVisitablePointable metaPointable;
+
+ public FlushColumnTupleWithMetaWriter(FlushColumnMetadata columnMetadata, int pageSize, int maxNumberOfTuples,
+ float tolerance) {
+ super(columnMetadata, pageSize, maxNumberOfTuples, tolerance);
+ metaColumnTransformer = new ColumnTransformer(columnMetadata, columnMetadata.getMetaRoot());
+ metaPointable = new TypedRecordLazyVisitablePointable(columnMetadata.getMetaType());
+ }
+
+ @Override
+ protected void writeMeta(LSMBTreeTupleReference btreeTuple) throws HyracksDataException {
+ if (btreeTuple.isAntimatter()) {
+ return;
+ }
+
+ int metaFieldId = columnMetadata.getMetaRecordFieldIndex();
+ metaPointable.set(btreeTuple.getFieldData(metaFieldId), btreeTuple.getFieldStart(metaFieldId),
+ btreeTuple.getFieldLength(metaFieldId));
+ //In case the primary key is not in the meta part, we take the maximum
+ primaryKeysEstimatedSize = Math.max(metaColumnTransformer.transform(metaPointable), primaryKeysEstimatedSize);
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/flush/FlushColumnTupleWriter.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/flush/FlushColumnTupleWriter.java
new file mode 100644
index 0000000..1af043f
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/flush/FlushColumnTupleWriter.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.column.operation.lsm.flush;
+
+import java.nio.ByteBuffer;
+
+import org.apache.asterix.column.values.writer.ColumnBatchWriter;
+import org.apache.asterix.column.values.writer.filters.AbstractColumnFilterWriter;
+import org.apache.asterix.om.lazy.RecordLazyVisitablePointable;
+import org.apache.asterix.om.lazy.TypedRecordLazyVisitablePointable;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.AbstractColumnTupleWriter;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnWriteMultiPageOp;
+import org.apache.hyracks.storage.am.lsm.btree.tuples.LSMBTreeTupleReference;
+
+public class FlushColumnTupleWriter extends AbstractColumnTupleWriter {
+ protected final FlushColumnMetadata columnMetadata;
+ protected final BatchFinalizerVisitor finalizer;
+ protected final ColumnBatchWriter writer;
+
+ private final ColumnTransformer transformer;
+ private final RecordLazyVisitablePointable pointable;
+ private final int maxNumberOfTuples;
+
+ protected int primaryKeysEstimatedSize;
+
+ public FlushColumnTupleWriter(FlushColumnMetadata columnMetadata, int pageSize, int maxNumberOfTuples,
+ float tolerance) {
+ this.columnMetadata = columnMetadata;
+ transformer = new ColumnTransformer(columnMetadata, columnMetadata.getRoot());
+ finalizer = new BatchFinalizerVisitor(columnMetadata);
+ writer = new ColumnBatchWriter(columnMetadata.getMultiPageOpRef(), pageSize, tolerance);
+ this.maxNumberOfTuples = maxNumberOfTuples;
+ pointable = new TypedRecordLazyVisitablePointable(columnMetadata.getDatasetType());
+ }
+
+ @Override
+ public final void init(IColumnWriteMultiPageOp multiPageOp) throws HyracksDataException {
+ columnMetadata.init(multiPageOp);
+ }
+
+ @Override
+ public final int getNumberOfColumns() {
+ return columnMetadata.getNumberOfColumns();
+ }
+
+ @Override
+ public final int bytesRequired(ITupleReference tuple) {
+ int primaryKeysSize = 0;
+ for (int i = 0; i < columnMetadata.getNumberOfPrimaryKeys(); i++) {
+ primaryKeysSize += tuple.getFieldLength(i);
+ }
+
+ //Mostly it is an overestimated size
+ return primaryKeysSize;
+ }
+
+ @Override
+ public final int getOccupiedSpace() {
+ int numberOfColumns = getNumberOfColumns();
+ int filterSize = numberOfColumns * AbstractColumnFilterWriter.FILTER_SIZE;
+ return primaryKeysEstimatedSize + filterSize;
+ }
+
+ @Override
+ public final int getMaxNumberOfTuples() {
+ return maxNumberOfTuples;
+ }
+
+ @Override
+ public final void close() {
+ columnMetadata.close();
+ }
+
+ @Override
+ public void writeTuple(ITupleReference tuple) throws HyracksDataException {
+ //This from an in-memory component, hence the cast
+ LSMBTreeTupleReference btreeTuple = (LSMBTreeTupleReference) tuple;
+ if (btreeTuple.isAntimatter()) {
+ //Write only the primary keys of an anti-matter tuple
+ primaryKeysEstimatedSize = transformer.writeAntiMatter(btreeTuple);
+ return;
+ }
+ writeRecord(tuple);
+ writeMeta(btreeTuple);
+ }
+
+ @Override
+ public final int flush(ByteBuffer pageZero) throws HyracksDataException {
+ writer.setPageZeroBuffer(pageZero, getNumberOfColumns(), columnMetadata.getNumberOfPrimaryKeys());
+ return finalizer.finalizeBatch(writer, columnMetadata);
+ }
+
+ protected void writeRecord(ITupleReference tuple) throws HyracksDataException {
+ int recordFieldId = columnMetadata.getRecordFieldIndex();
+ pointable.set(tuple.getFieldData(recordFieldId), tuple.getFieldStart(recordFieldId),
+ tuple.getFieldLength(recordFieldId));
+ primaryKeysEstimatedSize = transformer.transform(pointable);
+ }
+
+ protected void writeMeta(LSMBTreeTupleReference btreeTuple) throws HyracksDataException {
+ //NoOp
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/load/LoadColumnTupleReaderWriterFactory.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/load/LoadColumnTupleReaderWriterFactory.java
new file mode 100644
index 0000000..0c1990f
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/load/LoadColumnTupleReaderWriterFactory.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.column.operation.lsm.load;
+
+import org.apache.asterix.column.operation.lsm.flush.FlushColumnMetadata;
+import org.apache.asterix.column.operation.lsm.flush.FlushColumnTupleReaderWriterFactory;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.AbstractColumnTupleWriter;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnMetadata;
+
+public class LoadColumnTupleReaderWriterFactory extends FlushColumnTupleReaderWriterFactory {
+ private static final long serialVersionUID = -7583574057314353873L;
+
+ public LoadColumnTupleReaderWriterFactory(int pageSize, int maxNumberOfTuples, float tolerance) {
+ super(pageSize, maxNumberOfTuples, tolerance);
+ }
+
+ @Override
+ public AbstractColumnTupleWriter createColumnWriter(IColumnMetadata columnMetadata) {
+ return new LoadColumnTupleWriter((FlushColumnMetadata) columnMetadata, pageSize, maxNumberOfTuples, tolerance);
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/load/LoadColumnTupleWriter.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/load/LoadColumnTupleWriter.java
new file mode 100644
index 0000000..e4604da
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/load/LoadColumnTupleWriter.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.column.operation.lsm.load;
+
+import org.apache.asterix.column.operation.lsm.flush.FlushColumnMetadata;
+import org.apache.asterix.column.operation.lsm.flush.FlushColumnTupleWriter;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+public class LoadColumnTupleWriter extends FlushColumnTupleWriter {
+ public LoadColumnTupleWriter(FlushColumnMetadata columnMetadata, int pageSize, int maxNumberOfTuples,
+ float tolerance) {
+ super(columnMetadata, pageSize, maxNumberOfTuples, tolerance);
+ }
+
+ @Override
+ public void writeTuple(ITupleReference tuple) throws HyracksDataException {
+ writeRecord(tuple);
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/merge/IEndOfPageCallBack.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/merge/IEndOfPageCallBack.java
new file mode 100644
index 0000000..93df021
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/merge/IEndOfPageCallBack.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.column.operation.lsm.merge;
+
+import org.apache.asterix.column.tuple.MergeColumnTupleReference;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.btree.column.impls.btree.ColumnBTreeRangeSearchCursor;
+
+/**
+ * An interface to signal {@link MergeColumnTupleWriter} that a component's page has reached the end.
+ */
+@FunctionalInterface
+public interface IEndOfPageCallBack {
+ /**
+ * Call {@link MergeColumnTupleWriter} to finish the current "vertical" merging batch.
+ * The caller of this method is {@link MergeColumnTupleReference#lastTupleReached()}
+ *
+ * @see ColumnBTreeRangeSearchCursor#doHasNext()
+ */
+ void callEnd(MergeColumnTupleReference columnTuple) throws HyracksDataException;
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/merge/MergeColumnReadMetadata.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/merge/MergeColumnReadMetadata.java
new file mode 100644
index 0000000..11f3059
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/merge/MergeColumnReadMetadata.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.column.operation.lsm.merge;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.IOException;
+
+import org.apache.asterix.column.metadata.AbstractColumnImmutableReadMetadata;
+import org.apache.asterix.column.operation.lsm.flush.FlushColumnMetadata;
+import org.apache.asterix.column.values.IColumnValuesReader;
+import org.apache.asterix.column.values.IColumnValuesReaderFactory;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.data.std.primitive.IntegerPointable;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.AbstractColumnTupleReader;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
+
+/**
+ * Merge column read metadata belongs to read an {@link ILSMDiskComponent}
+ * This only for reading an existing on-disk component for a merge operation. The schema here is immutable and cannot
+ * be changed.
+ */
+public final class MergeColumnReadMetadata extends AbstractColumnImmutableReadMetadata {
+ private final IColumnValuesReader[] columnReaders;
+
+ private MergeColumnReadMetadata(ARecordType datasetType, ARecordType metaType, int numberOfPrimaryKeys,
+ IColumnValuesReader[] columnReaders, IValueReference serializedMetadata) {
+ super(datasetType, metaType, numberOfPrimaryKeys, serializedMetadata, columnReaders.length);
+ this.columnReaders = columnReaders;
+ }
+
+ /**
+ * create ColumnMergeReadMetadata from columnMetadata
+ *
+ * @param serializedMetadata columnMetadata
+ * @return {@link MergeColumnReadMetadata}
+ * @see FlushColumnMetadata#serializeColumnsMetadata() for more information about serialization order
+ */
+ public static MergeColumnReadMetadata create(ARecordType datasetType, ARecordType metaType, int numberOfPrimaryKeys,
+ IColumnValuesReaderFactory readerFactory, IValueReference serializedMetadata) throws IOException {
+ byte[] bytes = serializedMetadata.getByteArray();
+ int offset = serializedMetadata.getStartOffset();
+ int length = serializedMetadata.getLength();
+
+ int pathInfoStart = offset + IntegerPointable.getInteger(bytes, offset + PATH_INFO_POINTER);
+ DataInput input = new DataInputStream(new ByteArrayInputStream(bytes, pathInfoStart, length));
+ int numberOfColumns = input.readInt();
+ IColumnValuesReader[] columnReaders = new IColumnValuesReader[numberOfColumns];
+ for (int i = 0; i < numberOfColumns; i++) {
+ IColumnValuesReader columnReader = readerFactory.createValueReader(input);
+ //The order at which the path info was written is not ordered by the column index
+ columnReaders[columnReader.getColumnIndex()] = columnReader;
+ }
+
+ return new MergeColumnReadMetadata(datasetType, metaType, numberOfPrimaryKeys, columnReaders,
+ serializedMetadata);
+ }
+
+ public IColumnValuesReader[] getColumnReaders() {
+ return columnReaders;
+ }
+
+ @Override
+ public int getColumnIndex(int ordinal) {
+ return ordinal;
+ }
+
+ @Override
+ public int getNumberOfProjectedColumns() {
+ return columnReaders.length;
+ }
+
+ @Override
+ public AbstractColumnTupleReader createTupleReader() {
+ return new MergeColumnTupleReader(this);
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/merge/MergeColumnTupleProjector.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/merge/MergeColumnTupleProjector.java
new file mode 100644
index 0000000..f03506e
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/merge/MergeColumnTupleProjector.java
@@ -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.
+ */
+package org.apache.asterix.column.operation.lsm.merge;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.asterix.column.values.IColumnValuesReaderFactory;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.projection.IColumnProjectionInfo;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.projection.IColumnTupleProjector;
+
+public class MergeColumnTupleProjector implements IColumnTupleProjector {
+ private final ARecordType datasetType;
+ private final ARecordType metaType;
+ private final int numberOfPrimaryKeys;
+ private final IColumnValuesReaderFactory readerFactory;
+
+ public MergeColumnTupleProjector(ARecordType datasetType, ARecordType metaType, int numberOfPrimaryKeys,
+ IColumnValuesReaderFactory readerFactory) {
+ this.datasetType = datasetType;
+ this.metaType = metaType;
+ this.numberOfPrimaryKeys = numberOfPrimaryKeys;
+ this.readerFactory = readerFactory;
+ }
+
+ @Override
+ public IColumnProjectionInfo createProjectionInfo(IValueReference columnMetadata) throws HyracksDataException {
+ try {
+ return MergeColumnReadMetadata.create(datasetType, metaType, numberOfPrimaryKeys, readerFactory,
+ columnMetadata);
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ }
+
+ @Override
+ public ITupleReference project(ITupleReference tuple, DataOutput dos, ArrayTupleBuilder tb) throws IOException {
+ throw new IllegalAccessError(getClass().getName());
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/merge/MergeColumnTupleReader.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/merge/MergeColumnTupleReader.java
new file mode 100644
index 0000000..4114f10
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/merge/MergeColumnTupleReader.java
@@ -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.
+ */
+package org.apache.asterix.column.operation.lsm.merge;
+
+import org.apache.asterix.column.metadata.AbstractColumnImmutableReadMetadata;
+import org.apache.asterix.column.tuple.MergeColumnTupleReference;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.AbstractColumnTupleReader;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnReadMultiPageOp;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnTupleIterator;
+import org.apache.hyracks.storage.am.lsm.btree.column.impls.btree.ColumnBTreeReadLeafFrame;
+
+public class MergeColumnTupleReader extends AbstractColumnTupleReader {
+ private final MergeColumnReadMetadata columnMetadata;
+
+ public MergeColumnTupleReader(AbstractColumnImmutableReadMetadata columnMetadata) {
+ this.columnMetadata = (MergeColumnReadMetadata) columnMetadata;
+ }
+
+ @Override
+ public IColumnTupleIterator createTupleIterator(ColumnBTreeReadLeafFrame frame, int componentIndex,
+ IColumnReadMultiPageOp multiPageOp) {
+ return new MergeColumnTupleReference(componentIndex, frame, columnMetadata, multiPageOp);
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/merge/MergeColumnTupleReaderWriterFactory.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/merge/MergeColumnTupleReaderWriterFactory.java
new file mode 100644
index 0000000..1ac94fe
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/merge/MergeColumnTupleReaderWriterFactory.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.column.operation.lsm.merge;
+
+import org.apache.asterix.column.metadata.AbstractColumnImmutableReadMetadata;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.AbstractColumnTupleReader;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.AbstractColumnTupleReaderWriterFactory;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.AbstractColumnTupleWriter;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnMetadata;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.projection.IColumnProjectionInfo;
+
+public class MergeColumnTupleReaderWriterFactory extends AbstractColumnTupleReaderWriterFactory {
+ private static final long serialVersionUID = -2131401304338796428L;
+
+ public MergeColumnTupleReaderWriterFactory(int pageSize, int maxNumberOfTuples, float tolerance) {
+ super(pageSize, maxNumberOfTuples, tolerance);
+ }
+
+ @Override
+ public AbstractColumnTupleWriter createColumnWriter(IColumnMetadata columnMetadata) {
+ MergeColumnWriteMetadata mergeWriteMetadata = (MergeColumnWriteMetadata) columnMetadata;
+ return new MergeColumnTupleWriter(mergeWriteMetadata, pageSize, maxNumberOfTuples, tolerance);
+ }
+
+ @Override
+ public AbstractColumnTupleReader createColumnReader(IColumnProjectionInfo columnProjectionInfo) {
+ return ((AbstractColumnImmutableReadMetadata) columnProjectionInfo).createTupleReader();
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/merge/MergeColumnTupleWriter.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/merge/MergeColumnTupleWriter.java
new file mode 100644
index 0000000..fbda6d0
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/merge/MergeColumnTupleWriter.java
@@ -0,0 +1,192 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.column.operation.lsm.merge;
+
+import java.nio.ByteBuffer;
+import java.util.Comparator;
+import java.util.List;
+import java.util.PriorityQueue;
+
+import org.apache.asterix.column.tuple.MergeColumnTupleReference;
+import org.apache.asterix.column.util.RunLengthIntArray;
+import org.apache.asterix.column.values.IColumnValuesReader;
+import org.apache.asterix.column.values.IColumnValuesWriter;
+import org.apache.asterix.column.values.writer.ColumnBatchWriter;
+import org.apache.asterix.column.values.writer.filters.AbstractColumnFilterWriter;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.AbstractColumnTupleWriter;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnTupleIterator;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnWriteMultiPageOp;
+
+public class MergeColumnTupleWriter extends AbstractColumnTupleWriter {
+ private final MergeColumnWriteMetadata columnMetadata;
+ private final MergeColumnTupleReference[] componentsTuples;
+ private final RunLengthIntArray writtenComponents;
+
+ private final IColumnValuesWriter[] primaryKeyWriters;
+ private final PriorityQueue<IColumnValuesWriter> orderedColumns;
+ private final ColumnBatchWriter writer;
+ private final int maxNumberOfTuples;
+ private int primaryKeysEstimatedSize;
+
+ public MergeColumnTupleWriter(MergeColumnWriteMetadata columnMetadata, int pageSize, int maxNumberOfTuples,
+ float tolerance) {
+ this.columnMetadata = columnMetadata;
+ List<IColumnTupleIterator> componentsTuplesList = columnMetadata.getComponentsTuples();
+ this.componentsTuples = new MergeColumnTupleReference[componentsTuplesList.size()];
+ for (int i = 0; i < componentsTuplesList.size(); i++) {
+ MergeColumnTupleReference mergeTuple = (MergeColumnTupleReference) componentsTuplesList.get(i);
+ this.componentsTuples[i] = mergeTuple;
+ mergeTuple.registerEndOfPageCallBack(this::writeAllColumns);
+ }
+ this.writtenComponents = new RunLengthIntArray();
+ this.maxNumberOfTuples = maxNumberOfTuples;
+ writer = new ColumnBatchWriter(columnMetadata.getMultiPageOpRef(), pageSize, tolerance);
+ writtenComponents.reset();
+
+ primaryKeyWriters = new IColumnValuesWriter[columnMetadata.getNumberOfPrimaryKeys()];
+ for (int i = 0; i < primaryKeyWriters.length; i++) {
+ primaryKeyWriters[i] = columnMetadata.getWriter(i);
+ }
+ orderedColumns = new PriorityQueue<>(Comparator.comparingInt(x -> -x.getEstimatedSize()));
+ }
+
+ @Override
+ public int bytesRequired(ITupleReference tuple) {
+ int primaryKeysSize = 0;
+ for (int i = 0; i < columnMetadata.getNumberOfPrimaryKeys(); i++) {
+ primaryKeysSize += tuple.getFieldLength(i);
+ }
+
+ return primaryKeysSize;
+ }
+
+ @Override
+ public void init(IColumnWriteMultiPageOp multiPageOp) throws HyracksDataException {
+ columnMetadata.init(multiPageOp);
+ }
+
+ @Override
+ public int getNumberOfColumns() {
+ return columnMetadata.getNumberOfColumns();
+ }
+
+ @Override
+ public int getMaxNumberOfTuples() {
+ return maxNumberOfTuples;
+ }
+
+ @Override
+ public int getOccupiedSpace() {
+ int numberOfColumns = getNumberOfColumns();
+ int filterSize = numberOfColumns * AbstractColumnFilterWriter.FILTER_SIZE;
+ return primaryKeysEstimatedSize + filterSize;
+ }
+
+ @Override
+ public void writeTuple(ITupleReference tuple) throws HyracksDataException {
+ MergeColumnTupleReference columnTuple = (MergeColumnTupleReference) tuple;
+ int componentIndex = columnTuple.getComponentIndex();
+ int skipCount = columnTuple.getAndResetSkipCount();
+ if (skipCount > 0) {
+ writtenComponents.add(-componentIndex, skipCount);
+ }
+ if (columnTuple.isAntimatter()) {
+ writtenComponents.add(-componentIndex);
+ } else {
+ writtenComponents.add(componentIndex);
+ }
+ writePrimaryKeys(columnTuple);
+ }
+
+ private void writePrimaryKeys(MergeColumnTupleReference columnTuple) throws HyracksDataException {
+ int primaryKeySize = 0;
+ for (int i = 0; i < columnMetadata.getNumberOfPrimaryKeys(); i++) {
+ IColumnValuesReader columnReader = columnTuple.getReader(i);
+ IColumnValuesWriter columnWriter = primaryKeyWriters[i];
+ columnReader.write(columnWriter, false);
+ primaryKeySize += columnWriter.getEstimatedSize();
+ }
+ primaryKeysEstimatedSize = primaryKeySize;
+ }
+
+ private void writeNonKeyColumns() throws HyracksDataException {
+ for (int i = 0; i < writtenComponents.getNumberOfBlocks(); i++) {
+ int componentIndex = writtenComponents.getBlockValue(i);
+ if (componentIndex < 0) {
+ //Skip writing values of deleted tuples
+ componentIndex = -componentIndex;
+ skipReaders(componentIndex, writtenComponents.getBlockSize(i));
+ continue;
+ }
+ MergeColumnTupleReference componentTuple = componentsTuples[componentIndex];
+ int count = writtenComponents.getBlockSize(i);
+ for (int j = columnMetadata.getNumberOfPrimaryKeys(); j < columnMetadata.getNumberOfColumns(); j++) {
+ IColumnValuesReader columnReader = componentTuple.getReader(j);
+ IColumnValuesWriter columnWriter = columnMetadata.getWriter(j);
+ columnReader.write(columnWriter, count);
+ }
+ }
+ }
+
+ private void skipReaders(int componentIndex, int count) throws HyracksDataException {
+ MergeColumnTupleReference componentTuple = componentsTuples[componentIndex];
+ for (int j = columnMetadata.getNumberOfPrimaryKeys(); j < columnMetadata.getNumberOfColumns(); j++) {
+ IColumnValuesReader columnReader = componentTuple.getReader(j);
+ columnReader.skip(count);
+ }
+ }
+
+ @Override
+ public int flush(ByteBuffer pageZero) throws HyracksDataException {
+ int numberOfColumns = columnMetadata.getNumberOfColumns();
+ int numberOfPrimaryKeys = columnMetadata.getNumberOfPrimaryKeys();
+ if (writtenComponents.getSize() > 0) {
+ writeNonKeyColumns();
+ writtenComponents.reset();
+ }
+ for (int i = numberOfPrimaryKeys; i < numberOfColumns; i++) {
+ orderedColumns.add(columnMetadata.getWriter(i));
+ }
+ writer.setPageZeroBuffer(pageZero, numberOfColumns, numberOfPrimaryKeys);
+ int allocatedSpace = writer.writePrimaryKeyColumns(primaryKeyWriters);
+ allocatedSpace += writer.writeColumns(orderedColumns);
+ return allocatedSpace;
+ }
+
+ @Override
+ public void close() {
+ columnMetadata.close();
+ }
+
+ private void writeAllColumns(MergeColumnTupleReference columnTuple) throws HyracksDataException {
+ /*
+ * The last tuple from one of the components was reached. Since we are going to the next leaf, we will not be
+ * able to access the readers of this component's leaf after this tuple. So, we are going to write
+ * the values of all columns as recorded in writtenComponents
+ */
+ int skipCount = columnTuple.getAndResetSkipCount();
+ if (skipCount > 0) {
+ writtenComponents.add(-columnTuple.getComponentIndex(), skipCount);
+ }
+ writeNonKeyColumns();
+ writtenComponents.reset();
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/merge/MergeColumnWriteMetadata.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/merge/MergeColumnWriteMetadata.java
new file mode 100644
index 0000000..b0d1a01
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/merge/MergeColumnWriteMetadata.java
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.column.operation.lsm.merge;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.asterix.column.metadata.AbstractColumnImmutableMetadata;
+import org.apache.asterix.column.operation.lsm.flush.FlushColumnMetadata;
+import org.apache.asterix.column.values.IColumnValuesWriter;
+import org.apache.asterix.column.values.IColumnValuesWriterFactory;
+import org.apache.asterix.column.values.writer.ColumnValuesWriterFactory;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.data.std.primitive.IntegerPointable;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnTupleIterator;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnWriteMultiPageOp;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
+
+/**
+ * Merge column write metadata belongs to write a new merge {@link ILSMDiskComponent}
+ * This is for writing a new on-disk component by merging two or more on disk components. The final schema for this
+ * component will the most recent schema, which belongs to the newest merged component. The schema here is immutable
+ * and cannot be changed.
+ */
+public final class MergeColumnWriteMetadata extends AbstractColumnImmutableMetadata {
+ private final Mutable<IColumnWriteMultiPageOp> multiPageOpRef;
+ private final List<IColumnValuesWriter> columnWriters;
+ private final List<IColumnTupleIterator> componentsTuples;
+
+ /**
+ * For LSM Merge
+ */
+ private MergeColumnWriteMetadata(ARecordType datasetType, ARecordType metaType, int numberOfPrimaryKeys,
+ Mutable<IColumnWriteMultiPageOp> multiPageOpRef, List<IColumnValuesWriter> columnWriters,
+ IValueReference serializedMetadata, List<IColumnTupleIterator> componentsTuples) {
+ super(datasetType, metaType, numberOfPrimaryKeys, serializedMetadata, columnWriters.size());
+ this.multiPageOpRef = multiPageOpRef;
+ this.columnWriters = columnWriters;
+ this.componentsTuples = componentsTuples;
+ }
+
+ /**
+ * Set {@link IColumnWriteMultiPageOp} for {@link IColumnValuesWriter}
+ *
+ * @param multiPageOp multi-buffer allocator
+ */
+ public void init(IColumnWriteMultiPageOp multiPageOp) throws HyracksDataException {
+ multiPageOpRef.setValue(multiPageOp);
+
+ //Reset writer for the first write
+ for (int i = 0; i < columnWriters.size(); i++) {
+ columnWriters.get(i).reset();
+ }
+ }
+
+ public Mutable<IColumnWriteMultiPageOp> getMultiPageOpRef() {
+ return multiPageOpRef;
+ }
+
+ public IColumnValuesWriter getWriter(int columnIndex) {
+ return columnWriters.get(columnIndex);
+ }
+
+ public void close() {
+ multiPageOpRef.setValue(null);
+ for (int i = 0; i < columnWriters.size(); i++) {
+ columnWriters.get(i).close();
+ }
+ }
+
+ public static MergeColumnWriteMetadata create(ARecordType datasetType, ARecordType metaType,
+ int numberOfPrimaryKeys, Mutable<IColumnWriteMultiPageOp> multiPageOpRef,
+ IValueReference serializedMetadata, List<IColumnTupleIterator> componentsTuples) throws IOException {
+ byte[] bytes = serializedMetadata.getByteArray();
+ int offset = serializedMetadata.getStartOffset();
+ int length = serializedMetadata.getLength();
+
+ int writersOffset = offset + IntegerPointable.getInteger(bytes, offset + WRITERS_POINTER);
+ DataInput input = new DataInputStream(new ByteArrayInputStream(bytes, writersOffset, length));
+
+ IColumnValuesWriterFactory writerFactory = new ColumnValuesWriterFactory(multiPageOpRef);
+ List<IColumnValuesWriter> writers = new ArrayList<>();
+ FlushColumnMetadata.deserializeWriters(input, writers, writerFactory);
+
+ return new MergeColumnWriteMetadata(datasetType, metaType, numberOfPrimaryKeys, multiPageOpRef, writers,
+ serializedMetadata, componentsTuples);
+ }
+
+ public List<IColumnTupleIterator> getComponentsTuples() {
+ return componentsTuples;
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/tuple/AbstractAsterixColumnTupleReference.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/tuple/AbstractAsterixColumnTupleReference.java
new file mode 100644
index 0000000..df6b554
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/tuple/AbstractAsterixColumnTupleReference.java
@@ -0,0 +1,140 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.column.tuple;
+
+import org.apache.asterix.column.assembler.value.IValueGetter;
+import org.apache.asterix.column.assembler.value.ValueGetterFactory;
+import org.apache.asterix.column.bytes.stream.in.AbstractBytesInputStream;
+import org.apache.asterix.column.bytes.stream.in.ByteBufferInputStream;
+import org.apache.asterix.column.bytes.stream.in.MultiByteBufferInputStream;
+import org.apache.asterix.column.values.IColumnValuesReader;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnBufferProvider;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnReadMultiPageOp;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnTupleIterator;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.projection.IColumnProjectionInfo;
+import org.apache.hyracks.storage.am.lsm.btree.column.impls.btree.ColumnBTreeReadLeafFrame;
+import org.apache.hyracks.storage.am.lsm.btree.column.impls.lsm.tuples.AbstractColumnTupleReference;
+
+public abstract class AbstractAsterixColumnTupleReference extends AbstractColumnTupleReference {
+ private final IValueGetter[] primaryKeysValueGetters;
+ protected final ByteBufferInputStream[] primaryKeyStreams;
+ protected final IColumnValuesReader[] primaryKeyReaders;
+ protected final VoidPointable[] primaryKeys;
+ protected final AbstractBytesInputStream[] columnStreams;
+
+ protected AbstractAsterixColumnTupleReference(int componentIndex, ColumnBTreeReadLeafFrame frame,
+ IColumnProjectionInfo info, IColumnReadMultiPageOp multiPageOp) {
+ super(componentIndex, frame, info, multiPageOp);
+ primaryKeyReaders = getPrimaryKeyReaders(info);
+ int numberOfPrimaryKeys = primaryKeyReaders.length;
+
+ this.primaryKeyStreams = new ByteBufferInputStream[numberOfPrimaryKeys];
+ primaryKeysValueGetters = new IValueGetter[numberOfPrimaryKeys];
+ primaryKeys = new VoidPointable[numberOfPrimaryKeys];
+
+ for (int i = 0; i < numberOfPrimaryKeys; i++) {
+ primaryKeyStreams[i] = new ByteBufferInputStream();
+ primaryKeysValueGetters[i] =
+ ValueGetterFactory.INSTANCE.createValueGetter(primaryKeyReaders[i].getTypeTag());
+ primaryKeys[i] = new VoidPointable();
+ }
+
+ this.columnStreams = new AbstractBytesInputStream[info.getNumberOfProjectedColumns()];
+ for (int i = 0; i < columnStreams.length; i++) {
+ if (info.getColumnIndex(i) >= numberOfPrimaryKeys) {
+ columnStreams[i] = new MultiByteBufferInputStream();
+ } else {
+ columnStreams[i] = new ByteBufferInputStream();
+ }
+ }
+ }
+
+ protected abstract IColumnValuesReader[] getPrimaryKeyReaders(IColumnProjectionInfo info);
+
+ @Override
+ protected final void startPrimaryKey(IColumnBufferProvider provider, int startIndex, int ordinal,
+ int numberOfTuples) throws HyracksDataException {
+ ByteBufferInputStream primaryKeyStream = primaryKeyStreams[ordinal];
+ primaryKeyStream.reset(provider);
+ IColumnValuesReader reader = primaryKeyReaders[ordinal];
+ reader.reset(primaryKeyStream, numberOfTuples);
+ reader.skip(startIndex);
+ }
+
+ @Override
+ protected final void onNext() throws HyracksDataException {
+ for (int i = 0; i < primaryKeys.length; i++) {
+ IColumnValuesReader reader = primaryKeyReaders[i];
+ reader.next();
+ primaryKeys[i].set(primaryKeysValueGetters[i].getValue(reader));
+ }
+ }
+
+ @Override
+ public void lastTupleReached() throws HyracksDataException {
+ //Default: noOp
+ }
+
+ @Override
+ public final int getFieldCount() {
+ return primaryKeys.length;
+ }
+
+ @Override
+ public final byte[] getFieldData(int fIdx) {
+ return primaryKeys[fIdx].getByteArray();
+ }
+
+ @Override
+ public final int getFieldStart(int fIdx) {
+ return primaryKeys[fIdx].getStartOffset();
+ }
+
+ @Override
+ public final int getFieldLength(int fIdx) {
+ return primaryKeys[fIdx].getLength();
+ }
+
+ @Override
+ public final int getTupleSize() {
+ return -1;
+ }
+
+ @Override
+ public final boolean isAntimatter() {
+ /*
+ * The primary key cannot be missing, but the actual tuple is missing. There is no need to check other
+ * primary key readers (for composite primary keys). One primary key reader is sufficient to determine if a
+ * tuple is an anti-matter tuple.
+ */
+ return primaryKeyReaders[0].isMissing();
+ }
+
+ @Override
+ public final int compareTo(IColumnTupleIterator o) {
+ AbstractAsterixColumnTupleReference other = (AbstractAsterixColumnTupleReference) o;
+ int compare = 0;
+ for (int i = 0; i < primaryKeys.length && compare == 0; i++) {
+ compare = primaryKeyReaders[i].compareTo(other.primaryKeyReaders[i]);
+ }
+ return compare;
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/tuple/MergeColumnTupleReference.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/tuple/MergeColumnTupleReference.java
new file mode 100644
index 0000000..c10d415
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/tuple/MergeColumnTupleReference.java
@@ -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.
+ */
+package org.apache.asterix.column.tuple;
+
+import java.nio.ByteBuffer;
+
+import org.apache.asterix.column.bytes.stream.in.MultiByteBufferInputStream;
+import org.apache.asterix.column.operation.lsm.merge.IEndOfPageCallBack;
+import org.apache.asterix.column.operation.lsm.merge.MergeColumnReadMetadata;
+import org.apache.asterix.column.values.IColumnValuesReader;
+import org.apache.asterix.column.values.writer.filters.AbstractColumnFilterWriter;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnBufferProvider;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnReadMultiPageOp;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.projection.IColumnProjectionInfo;
+import org.apache.hyracks.storage.am.lsm.btree.column.impls.btree.ColumnBTreeReadLeafFrame;
+
+public final class MergeColumnTupleReference extends AbstractAsterixColumnTupleReference {
+ private final IColumnValuesReader[] columnReaders;
+ private int skipCount;
+ private IEndOfPageCallBack endOfPageCallBack;
+
+ public MergeColumnTupleReference(int componentIndex, ColumnBTreeReadLeafFrame frame,
+ MergeColumnReadMetadata columnMetadata, IColumnReadMultiPageOp multiPageOp) {
+ super(componentIndex, frame, columnMetadata, multiPageOp);
+ this.columnReaders = columnMetadata.getColumnReaders();
+ }
+
+ @Override
+ protected IColumnValuesReader[] getPrimaryKeyReaders(IColumnProjectionInfo info) {
+ MergeColumnReadMetadata columnMetadata = (MergeColumnReadMetadata) info;
+ int numberOfPrimaryKeys = columnMetadata.getNumberOfPrimaryKeys();
+ IColumnValuesReader[] primaryKeyReaders = new IColumnValuesReader[numberOfPrimaryKeys];
+ System.arraycopy(columnMetadata.getColumnReaders(), 0, primaryKeyReaders, 0, numberOfPrimaryKeys);
+ return primaryKeyReaders;
+ }
+
+ @Override
+ protected boolean startNewPage(ByteBuffer pageZero, int numberOfColumns, int numberOfTuples) {
+ //Skip filters
+ pageZero.position(pageZero.position() + numberOfColumns * AbstractColumnFilterWriter.FILTER_SIZE);
+ skipCount = 0;
+ return true;
+ }
+
+ @Override
+ protected void startColumn(IColumnBufferProvider buffersProvider, int startIndex, int ordinal, int numberOfTuples)
+ throws HyracksDataException {
+ int numberOfPrimaryKeys = primaryKeys.length;
+ if (ordinal < numberOfPrimaryKeys) {
+ //Skip primary key
+ return;
+ }
+ MultiByteBufferInputStream columnStream = (MultiByteBufferInputStream) columnStreams[ordinal];
+ columnStream.reset(buffersProvider);
+ IColumnValuesReader reader = columnReaders[ordinal];
+ reader.reset(columnStream, numberOfTuples);
+ reader.skip(startIndex);
+ }
+
+ @Override
+ public void skip(int count) throws HyracksDataException {
+ skipCount += count;
+ }
+
+ @Override
+ public void lastTupleReached() throws HyracksDataException {
+ endOfPageCallBack.callEnd(this);
+ }
+
+ public int getAndResetSkipCount() {
+ int currentSkipCount = skipCount;
+ skipCount = 0;
+ return currentSkipCount;
+ }
+
+ public IColumnValuesReader getReader(int columnIndex) {
+ return columnReaders[columnIndex];
+ }
+
+ public void registerEndOfPageCallBack(IEndOfPageCallBack endOfPageCallBack) {
+ this.endOfPageCallBack = endOfPageCallBack;
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/util/ColumnValuesUtil.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/util/ColumnValuesUtil.java
new file mode 100644
index 0000000..0ecdeef
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/util/ColumnValuesUtil.java
@@ -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.
+ */
+package org.apache.asterix.column.util;
+
+public class ColumnValuesUtil {
+ private ColumnValuesUtil() {
+ }
+
+ public static int getBitWidth(int level) {
+ //+1 for the null bit
+ return (32 - Integer.numberOfLeadingZeros(level)) + 1;
+ }
+
+ public static int getNullMask(int level) {
+ return 1 << getBitWidth(level) - 1;
+ }
+
+ public static boolean isNull(int mask, int level) {
+ return (mask & level) == mask;
+ }
+
+ public static int getChildValue(int parentMask, int childMask, int level) {
+ if (isNull(parentMask, level)) {
+ return clearNullBit(parentMask, level) | childMask;
+ }
+ return level;
+ }
+
+ public static int clearNullBit(int nullBitMask, int level) {
+ return (nullBitMask - 1) & level;
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/util/RunLengthIntArray.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/util/RunLengthIntArray.java
new file mode 100644
index 0000000..df238cb
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/util/RunLengthIntArray.java
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.column.util;
+
+import java.util.Arrays;
+
+import it.unimi.dsi.fastutil.ints.IntArrayList;
+
+/**
+ * Run-length integer array is to be used for storing repetitive integer values. This is intended for
+ * storing a large number of repeated integers (~1000s). It is not recommended for storing smaller number of integers.
+ * This structure maintains two arrays:
+ * - blockValues: stores the array values
+ * - blockCounts: stores the counts of values in <code>blockValues</code> in a monotonic fashion
+ * <pr>
+ * Example:
+ * Original Array: [1,1,1,1,1,1,2,2,2,1,1,1]
+ * blockValues: [1,2,1]
+ * blockCounts: [6,10,13]
+ */
+public final class RunLengthIntArray {
+ private final IntArrayList blockValues;
+ private int[] blockCounts;
+ private int lastSeen;
+ private int size;
+
+ public RunLengthIntArray() {
+ blockValues = new IntArrayList();
+ blockCounts = new int[32];
+ reset();
+ }
+
+ public void reset() {
+ blockValues.clear();
+ lastSeen = -1;
+ size = 0;
+ }
+
+ public void add(int value) {
+ if (size == 0 || value != lastSeen) {
+ lastSeen = value;
+ newBlock();
+ blockValues.add(value);
+ }
+ blockCounts[blockValues.size() - 1]++;
+ size++;
+ }
+
+ public void add(int value, int count) {
+ if (count == 0) {
+ return;
+ }
+ if (size == 0 || value != lastSeen) {
+ lastSeen = value;
+ newBlock();
+ blockValues.add(value);
+ }
+ blockCounts[blockValues.size() - 1] += count;
+ size += count;
+ }
+
+ public int getSize() {
+ return size;
+ }
+
+ public int getNumberOfBlocks() {
+ return blockValues.size();
+ }
+
+ public int getBlockValue(int blockIndex) {
+ return blockValues.getInt(blockIndex);
+ }
+
+ public int getBlockSize(int blockIndex) {
+ if (blockIndex == 0) {
+ return blockCounts[blockIndex];
+ }
+ return blockCounts[blockIndex] - blockCounts[blockIndex - 1];
+ }
+
+ public int getBlockSize(int blockIndex, int startIndex) {
+ return blockCounts[blockIndex] - startIndex;
+ }
+
+ public int getBlockIndex(int startIndex) {
+ if (startIndex >= size) {
+ throw new IndexOutOfBoundsException("startIndex: " + startIndex + " >= size:" + size);
+ }
+ int index = Arrays.binarySearch(blockCounts, 0, blockValues.size(), startIndex);
+ if (index < 0) {
+ index = Math.abs(index) - 1;
+ }
+ return index;
+ }
+
+ public void add(RunLengthIntArray other, int startIndex) {
+ if (startIndex >= other.size) {
+ throw new IndexOutOfBoundsException("startIndex: " + startIndex + " >= other size:" + size);
+ }
+ //First, handle the first block as startIndex might be at the middle of a block
+ //Get which block that startIndex resides
+ int otherBlockIndex = other.getBlockIndex(startIndex);
+ //Get the remaining of the first block starting from startIndex
+ int otherBlockSizeRemaining = other.getBlockSize(otherBlockIndex, startIndex);
+ //Batch add all the remaining values
+ add(other.getBlockValue(otherBlockIndex), otherBlockSizeRemaining);
+
+ //Add other blocks as batches
+ for (int i = otherBlockIndex + 1; i < other.getNumberOfBlocks(); i++) {
+ add(other.getBlockValue(i), other.getBlockSize(i));
+ }
+ }
+
+ private void newBlock() {
+ int newBlockIndex = blockValues.size();
+ if (newBlockIndex == blockCounts.length) {
+ int[] newRepCount = new int[blockCounts.length * 2];
+ System.arraycopy(blockCounts, 0, newRepCount, 0, blockCounts.length);
+ blockCounts = newRepCount;
+ }
+ if (newBlockIndex > 0) {
+ /*
+ * To easily compute where the actual block resides, the block counts are always increasing.
+ * For example:
+ * - Let blockCounts = [5, 6, 13] and blockValues = [1, 0, 1]
+ * - The block sizes are 5, 1, and 7 respectively
+ * - Let say that we want to know what is the value at index 11 by calling getValue(11)
+ * - by searching blockCounts, we know it is at the block with index 2
+ * - Then the value is 1
+ */
+ blockCounts[newBlockIndex] = blockCounts[newBlockIndex - 1];
+ } else {
+ blockCounts[0] = 0;
+ }
+ }
+
+ @Override
+ public String toString() {
+ if (size == 0) {
+ return "[]";
+ }
+ StringBuilder builder = new StringBuilder();
+ int i = 0;
+ builder.append("size: ");
+ builder.append(getSize());
+ builder.append(" [");
+ for (; i < getNumberOfBlocks() - 1; i++) {
+ appendBlockInfo(i, builder);
+ builder.append(',');
+ }
+ appendBlockInfo(i, builder);
+ builder.append(']');
+ return builder.toString();
+ }
+
+ private void appendBlockInfo(int blockIndex, StringBuilder builder) {
+ builder.append('(');
+ builder.append(getBlockValue(blockIndex));
+ builder.append(',');
+ builder.append(getBlockSize(blockIndex));
+ builder.append(')');
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/IColumnBatchWriter.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/IColumnBatchWriter.java
new file mode 100644
index 0000000..fc1173f
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/IColumnBatchWriter.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.column.values;
+
+import java.nio.ByteBuffer;
+import java.util.PriorityQueue;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public interface IColumnBatchWriter {
+ void setPageZeroBuffer(ByteBuffer pageZeroBuffer, int numberOfColumns, int numberOfPrimaryKeys);
+
+ /**
+ * Writes the primary keys' values to Page0
+ *
+ * @param primaryKeyWriters primary keys' writers
+ * @return the allocated space for the primary keys' writers
+ */
+ int writePrimaryKeyColumns(IColumnValuesWriter[] primaryKeyWriters) throws HyracksDataException;
+
+ /**
+ * Writes the non-key values to multiple pages
+ *
+ * @param nonKeysColumnWriters non-key values' writers
+ * @return the allocated space for the non-key values' writers
+ */
+ int writeColumns(PriorityQueue<IColumnValuesWriter> nonKeysColumnWriters) throws HyracksDataException;
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/IColumnValuesReader.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/IColumnValuesReader.java
new file mode 100644
index 0000000..0f4cc0c
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/IColumnValuesReader.java
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.column.values;
+
+import org.apache.asterix.column.bytes.stream.in.AbstractBytesInputStream;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+
+public interface IColumnValuesReader extends Comparable<IColumnValuesReader> {
+ /**
+ * Reset the reader
+ *
+ * @param in input stream that contains the values
+ * @param tupleCount tuple count this column batch belongs to
+ */
+ void reset(AbstractBytesInputStream in, int tupleCount) throws HyracksDataException;
+
+ /* ***********************
+ * Iteration functions
+ * ***********************
+ */
+
+ /**
+ * Move the next value
+ *
+ * @return true if next value was found, false if the end of the values
+ */
+ boolean next() throws HyracksDataException;
+
+ /* ***********************
+ * Information functions
+ * ***********************
+ */
+ ATypeTag getTypeTag();
+
+ /**
+ * @return columnIndex
+ */
+ int getColumnIndex();
+
+ /**
+ * @return Level of the value, which determines if it is NULL, MISSING, or VALUE
+ */
+ int getLevel();
+
+ /**
+ * @return is the current value MISSING
+ */
+ boolean isMissing();
+
+ /**
+ * @return is the current value NULL
+ */
+ boolean isNull();
+
+ /**
+ * @return is an actual value (i.e., neither NULL or MISSING)
+ */
+ boolean isValue();
+
+ /**
+ * @return is this column belongs to an array or multiset
+ */
+ boolean isRepeated();
+
+ /**
+ * @return is it an end of an array (arrays could be nested, and we can hit different delimiters)
+ */
+ boolean isDelimiter();
+
+ /**
+ * @return which delimiter was returned (nested arrays have different delimiter indexes)
+ */
+ int getDelimiterIndex();
+
+ /* ***********************
+ * Value functions
+ * ***********************
+ */
+
+ long getLong();
+
+ double getDouble();
+
+ boolean getBoolean();
+
+ IValueReference getBytes();
+
+ /* ***********************
+ * Write function
+ * ***********************
+ */
+
+ /**
+ * Write the content of reader to the writer
+ *
+ * @param writer to which is the content of this reader is written to
+ * @param callNext should call next on write
+ */
+ void write(IColumnValuesWriter writer, boolean callNext) throws HyracksDataException;
+
+ /**
+ * Write the content of reader to the writer
+ *
+ * @param writer to which is the content of this reader is written to
+ * @param count number of values to write
+ */
+ void write(IColumnValuesWriter writer, int count) throws HyracksDataException;
+
+ /**
+ * Skip values
+ *
+ * @param count the number of values should be skipped
+ */
+ void skip(int count) throws HyracksDataException;
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/IColumnValuesReaderFactory.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/IColumnValuesReaderFactory.java
new file mode 100644
index 0000000..7c41512
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/IColumnValuesReaderFactory.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.column.values;
+
+import java.io.DataInput;
+import java.io.IOException;
+
+import org.apache.asterix.column.metadata.PathInfoSerializer;
+import org.apache.asterix.om.types.ATypeTag;
+
+public interface IColumnValuesReaderFactory {
+ /**
+ * Create reader for a non-repeated primitive type
+ *
+ * @param typeTag primitive type tag
+ * @param columnIndex column index
+ * @param maxLevel maximum definition levels
+ * @param primaryKey is the value belongs to a primary key?
+ * @return columnar reader
+ */
+ IColumnValuesReader createValueReader(ATypeTag typeTag, int columnIndex, int maxLevel, boolean primaryKey);
+
+ /**
+ * Create a reader for a repeated primitive type
+ *
+ * @param typeTag primitive type tag
+ * @param columnIndex column index
+ * @param maxLevel maximum definition levels
+ * @param delimiters the definition levels for array delimiters
+ * @return columnar reader
+ */
+ IColumnValuesReader createValueReader(ATypeTag typeTag, int columnIndex, int maxLevel, int[] delimiters);
+
+ /**
+ * Create a reader from a serialized path info
+ *
+ * @param input column metadata info
+ * @return columnar reader
+ * @see PathInfoSerializer#writePathInfo(ATypeTag, int, boolean) for more information on how the path info is serialized
+ */
+ IColumnValuesReader createValueReader(DataInput input) throws IOException;
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/IColumnValuesWriter.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/IColumnValuesWriter.java
new file mode 100644
index 0000000..d4e6099
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/IColumnValuesWriter.java
@@ -0,0 +1,138 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.column.values;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.OutputStream;
+
+import org.apache.asterix.column.util.RunLengthIntArray;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+
+/**
+ * Column writer for values
+ */
+public interface IColumnValuesWriter {
+
+ /**
+ * Reset the writer
+ */
+ void reset() throws HyracksDataException;
+
+ /**
+ * @return the corresponding index of a column
+ */
+ int getColumnIndex();
+
+ /**
+ * Write a value that are not MISSING or NULL
+ *
+ * @param tag value type tag
+ * @param value value reference
+ */
+ void writeValue(ATypeTag tag, IValueReference value) throws HyracksDataException;
+
+ /**
+ * Writing an anti-matter primary key value
+ *
+ * @param value value reference
+ */
+ void writeAntiMatter(ATypeTag tag, IValueReference value) throws HyracksDataException;
+
+ /**
+ * Write level
+ *
+ * @param level level of the value
+ */
+ void writeLevel(int level) throws HyracksDataException;
+
+ /**
+ * Convenient way to write a level multiple times
+ *
+ * @param level level of the value
+ * @param count the number of level occurrences
+ */
+ void writeLevels(int level, int count) throws HyracksDataException;
+
+ /**
+ * For all writers except for {@link ATypeTag#NULL} writer, this method will return null
+ *
+ * @return the definition levels if this is a {@link ATypeTag#NULL} writer, {@code null} otherwise
+ */
+ RunLengthIntArray getDefinitionLevelsIntArray();
+
+ /**
+ * Write NULL
+ *
+ * @param level at what level the NULL occurred
+ */
+ void writeNull(int level) throws HyracksDataException;
+
+ /**
+ * Write a non-unknown value from a reader. Not intended for writing {@link ATypeTag#NULL} or
+ * {@link ATypeTag#MISSING}
+ */
+ void writeValue(IColumnValuesReader reader) throws HyracksDataException;
+
+ /**
+ * @return (probably) an overestimated size of the encoded values
+ */
+ int getEstimatedSize();
+
+ /**
+ * @return the allocated space in bytes
+ */
+ int getAllocatedSpace();
+
+ /**
+ * @return the total count of values
+ */
+ int getCount();
+
+ /**
+ * @return normalized minimum column value
+ */
+ long getNormalizedMinValue();
+
+ /**
+ * @return normalized maximum column value
+ */
+ long getNormalizedMaxValue();
+
+ /**
+ * Flush the columns value to output stream
+ *
+ * @param out output stream
+ */
+ void flush(OutputStream out) throws HyracksDataException;
+
+ /**
+ * Close the writer and release all allocated buffers
+ */
+ void close();
+
+ /**
+ * Serialize the writer
+ *
+ * @param output destination to which the writer should be serialized to
+ */
+ void serialize(DataOutput output) throws IOException;
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/IColumnValuesWriterFactory.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/IColumnValuesWriterFactory.java
new file mode 100644
index 0000000..c858376
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/IColumnValuesWriterFactory.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.column.values;
+
+import org.apache.asterix.om.types.ATypeTag;
+
+public interface IColumnValuesWriterFactory {
+ /**
+ * Create a writer
+ *
+ * @param tag column type
+ * @param columnIndex column index
+ * @param level maximum level that determine a value is not null or missing
+ * @param writeAlways should writer always despite the fact all values were missing/null
+ * @param filtered has a column filter
+ * @return a writer
+ */
+ IColumnValuesWriter createValueWriter(ATypeTag tag, int columnIndex, int level, boolean writeAlways,
+ boolean filtered);
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/AbstractColumnValuesReader.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/AbstractColumnValuesReader.java
new file mode 100644
index 0000000..c0cf18a
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/AbstractColumnValuesReader.java
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.column.values.reader;
+
+import java.io.IOException;
+
+import org.apache.asterix.column.bytes.decoder.ParquetRunLengthBitPackingHybridDecoder;
+import org.apache.asterix.column.bytes.stream.in.AbstractBytesInputStream;
+import org.apache.asterix.column.bytes.stream.in.ByteBufferInputStream;
+import org.apache.asterix.column.bytes.stream.in.MultiByteBufferInputStream;
+import org.apache.asterix.column.util.ColumnValuesUtil;
+import org.apache.asterix.column.values.IColumnValuesReader;
+import org.apache.asterix.column.values.IColumnValuesWriter;
+import org.apache.asterix.column.values.reader.value.AbstractValueReader;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.parquet.bytes.BytesUtils;
+
+abstract class AbstractColumnValuesReader implements IColumnValuesReader {
+ protected final AbstractValueReader valueReader;
+ protected final int columnIndex;
+ protected final int maxLevel;
+ protected final ParquetRunLengthBitPackingHybridDecoder definitionLevels;
+ protected final AbstractBytesInputStream valuesStream;
+ protected int level;
+ protected int valueCount;
+ protected int valueIndex;
+
+ private int nullBitMask;
+ private boolean nullLevel;
+ private boolean allMissing;
+
+ AbstractColumnValuesReader(AbstractValueReader valueReader, int columnIndex, int maxLevel, boolean primaryKey) {
+ this.valueReader = valueReader;
+ this.columnIndex = columnIndex;
+ this.maxLevel = maxLevel;
+ definitionLevels = new ParquetRunLengthBitPackingHybridDecoder(ColumnValuesUtil.getBitWidth(maxLevel));
+ valuesStream = primaryKey ? new ByteBufferInputStream() : new MultiByteBufferInputStream();
+ }
+
+ final void nextLevel() throws HyracksDataException {
+ if (allMissing) {
+ return;
+ }
+ try {
+ int actualLevel = definitionLevels.readInt();
+ //Check whether the level is for a null value
+ nullLevel = ColumnValuesUtil.isNull(nullBitMask, actualLevel);
+ //Clear the null bit to allow repeated value readers determine the correct delimiter for null values
+ level = ColumnValuesUtil.clearNullBit(nullBitMask, actualLevel);
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ }
+
+ abstract void resetValues();
+
+ @Override
+ public final void reset(AbstractBytesInputStream in, int tupleCount) throws HyracksDataException {
+ valueIndex = 0;
+ if (in.available() == 0) {
+ allMissing = true;
+ level = 0;
+ valueCount = tupleCount;
+ return;
+ }
+ allMissing = false;
+ try {
+ nullBitMask = ColumnValuesUtil.getNullMask(BytesUtils.readZigZagVarInt(in));
+ int defLevelsSize = BytesUtils.readZigZagVarInt(in);
+ valueCount = BytesUtils.readZigZagVarInt(in);
+ definitionLevels.reset(in);
+ valuesStream.resetAt(defLevelsSize, in);
+ int valueLength = BytesUtils.readZigZagVarInt(valuesStream);
+ if (valueLength > 0) {
+ valueReader.resetValue(valuesStream);
+ }
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ resetValues();
+ }
+
+ @Override
+ public final ATypeTag getTypeTag() {
+ return valueReader.getTypeTag();
+ }
+
+ @Override
+ public final int getColumnIndex() {
+ return columnIndex;
+ }
+
+ @Override
+ public int getLevel() {
+ return level;
+ }
+
+ @Override
+ public final boolean isMissing() {
+ return !isDelimiter() && level < maxLevel;
+ }
+
+ @Override
+ public final boolean isNull() {
+ return nullLevel;
+ }
+
+ @Override
+ public final boolean isValue() {
+ return !isNull() && level == maxLevel;
+ }
+
+ @Override
+ public final long getLong() {
+ return valueReader.getLong();
+ }
+
+ @Override
+ public final double getDouble() {
+ return valueReader.getDouble();
+ }
+
+ @Override
+ public final boolean getBoolean() {
+ return valueReader.getBoolean();
+ }
+
+ @Override
+ public final IValueReference getBytes() {
+ return valueReader.getBytes();
+ }
+
+ @Override
+ public final int compareTo(IColumnValuesReader o) {
+ return valueReader.compareTo(((AbstractColumnValuesReader) o).valueReader);
+ }
+
+ @Override
+ public final void write(IColumnValuesWriter writer, int count) throws HyracksDataException {
+ for (int i = 0; i < count; i++) {
+ write(writer, true);
+ }
+ }
+
+ @Override
+ public void skip(int count) throws HyracksDataException {
+ for (int i = 0; i < count; i++) {
+ next();
+ }
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/ColumnValueReaderFactory.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/ColumnValueReaderFactory.java
new file mode 100644
index 0000000..b233482
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/ColumnValueReaderFactory.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.column.values.reader;
+
+import java.io.DataInput;
+import java.io.IOException;
+
+import org.apache.asterix.column.values.IColumnValuesReader;
+import org.apache.asterix.column.values.IColumnValuesReaderFactory;
+import org.apache.asterix.column.values.reader.value.AbstractValueReader;
+import org.apache.asterix.column.values.reader.value.BooleanValueReader;
+import org.apache.asterix.column.values.reader.value.DoubleValueReader;
+import org.apache.asterix.column.values.reader.value.LongValueReader;
+import org.apache.asterix.column.values.reader.value.NoOpValueReader;
+import org.apache.asterix.column.values.reader.value.StringValueReader;
+import org.apache.asterix.column.values.reader.value.UUIDValueReader;
+import org.apache.asterix.om.types.ATypeTag;
+
+public class ColumnValueReaderFactory implements IColumnValuesReaderFactory {
+ @Override
+ public IColumnValuesReader createValueReader(ATypeTag typeTag, int columnIndex, int maxLevel, boolean primaryKey) {
+ return new PrimitiveColumnValuesReader(createReader(typeTag), columnIndex, maxLevel, primaryKey);
+ }
+
+ @Override
+ public IColumnValuesReader createValueReader(ATypeTag typeTag, int columnIndex, int maxLevel, int[] delimiters) {
+ return new RepeatedPrimitiveColumnValuesReader(createReader(typeTag), columnIndex, maxLevel, delimiters);
+ }
+
+ @Override
+ public IColumnValuesReader createValueReader(DataInput input) throws IOException {
+ ATypeTag typeTag = ATypeTag.VALUE_TYPE_MAPPING[input.readByte()];
+ int columnIndex = input.readInt();
+ int maxLevel = input.readInt();
+ boolean primaryKey = input.readBoolean();
+ boolean collection = input.readBoolean();
+ if (collection) {
+ int[] delimiters = new int[input.readInt()];
+ for (int i = 0; i < delimiters.length; i++) {
+ delimiters[i] = input.readInt();
+ }
+ return createValueReader(typeTag, columnIndex, maxLevel, delimiters);
+ }
+ return createValueReader(typeTag, columnIndex, maxLevel, primaryKey);
+ }
+
+ private AbstractValueReader createReader(ATypeTag typeTag) {
+ switch (typeTag) {
+ case MISSING:
+ case NULL:
+ return NoOpValueReader.INSTANCE;
+ case BOOLEAN:
+ return new BooleanValueReader();
+ case BIGINT:
+ return new LongValueReader();
+ case DOUBLE:
+ return new DoubleValueReader();
+ case STRING:
+ return new StringValueReader();
+ case UUID:
+ return new UUIDValueReader();
+ default:
+ throw new UnsupportedOperationException(typeTag + " is not supported");
+ }
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/PrimitiveColumnValuesReader.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/PrimitiveColumnValuesReader.java
new file mode 100644
index 0000000..e8c7bc5
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/PrimitiveColumnValuesReader.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.column.values.reader;
+
+import java.io.IOException;
+
+import org.apache.asterix.column.values.IColumnValuesWriter;
+import org.apache.asterix.column.values.reader.value.AbstractValueReader;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * Reader for a non-repeated primitive value
+ */
+public final class PrimitiveColumnValuesReader extends AbstractColumnValuesReader {
+ /**
+ * A primary key value is always present. Anti-matter can be determined by checking whether the definition level
+ * indicates that the tuple's values are missing (i.e., by calling {@link #isMissing()}).
+ */
+ private final boolean primaryKey;
+
+ public PrimitiveColumnValuesReader(AbstractValueReader reader, int columnIndex, int maxLevel, boolean primaryKey) {
+ super(reader, columnIndex, maxLevel, primaryKey);
+ this.primaryKey = primaryKey;
+ }
+
+ @Override
+ public void resetValues() {
+ //NoOp
+ }
+
+ @Override
+ public boolean next() throws HyracksDataException {
+ if (valueIndex == valueCount) {
+ return false;
+ }
+ valueIndex++;
+
+ try {
+ nextLevel();
+ if (primaryKey || level == maxLevel) {
+ valueReader.nextValue();
+ }
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ return true;
+ }
+
+ @Override
+ public boolean isRepeated() {
+ return false;
+ }
+
+ @Override
+ public boolean isDelimiter() {
+ return false;
+ }
+
+ @Override
+ public int getDelimiterIndex() {
+ throw new IllegalStateException("Not a repeated reader");
+ }
+
+ @Override
+ public void write(IColumnValuesWriter writer, boolean callNext) throws HyracksDataException {
+ if (callNext && !next()) {
+ throw new IllegalStateException("No more values");
+ }
+
+ writer.writeLevel(level);
+ if (primaryKey || isValue()) {
+ try {
+ writer.writeValue(this);
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ }
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/RepeatedPrimitiveColumnValuesReader.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/RepeatedPrimitiveColumnValuesReader.java
new file mode 100644
index 0000000..0fb98be
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/RepeatedPrimitiveColumnValuesReader.java
@@ -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.
+ */
+package org.apache.asterix.column.values.reader;
+
+import java.io.IOException;
+
+import org.apache.asterix.column.values.IColumnValuesWriter;
+import org.apache.asterix.column.values.reader.value.AbstractValueReader;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * For primitive values that belong to an {@link ATypeTag#ARRAY} and {@link ATypeTag#MULTISET}
+ */
+public final class RepeatedPrimitiveColumnValuesReader extends AbstractColumnValuesReader {
+ private final int[] delimiters;
+ private final int[] levelToDelimiterMap;
+ private int delimiterIndex;
+
+ RepeatedPrimitiveColumnValuesReader(AbstractValueReader valueReader, int columnIndex, int maxLevel,
+ int[] delimiters) {
+ super(valueReader, columnIndex, maxLevel, false);
+ this.delimiters = delimiters;
+ delimiterIndex = delimiters.length;
+
+ levelToDelimiterMap = new int[maxLevel + 1];
+ int currentDelimiterIndex = 0;
+ for (int level = maxLevel; level >= 0; level--) {
+ if (currentDelimiterIndex < delimiters.length && level == delimiters[currentDelimiterIndex]) {
+ currentDelimiterIndex++;
+ }
+ levelToDelimiterMap[level] = currentDelimiterIndex;
+ }
+ }
+
+ @Override
+ protected void resetValues() {
+ delimiterIndex = delimiters.length;
+ }
+
+ @Override
+ public boolean next() throws HyracksDataException {
+ if (valueIndex == valueCount) {
+ return false;
+ }
+
+ consumeDelimiterIfAny();
+ nextLevel();
+ setDelimiterIndex();
+ if (level == maxLevel) {
+ valueReader.nextValue();
+ }
+ valueIndex++;
+ return true;
+ }
+
+ @Override
+ public boolean isRepeated() {
+ return true;
+ }
+
+ @Override
+ public boolean isDelimiter() {
+ return delimiterIndex < delimiters.length && level == delimiters[delimiterIndex];
+ }
+
+ @Override
+ public int getDelimiterIndex() {
+ return delimiterIndex;
+ }
+
+ @Override
+ public void write(IColumnValuesWriter writer, boolean callNext) throws HyracksDataException {
+ //We always call next as repeated values cannot be primary keys
+ if (!next()) {
+ throw new IllegalStateException("No more values");
+ }
+
+ if (isRepeatedValue()) {
+ while (!isLastDelimiter()) {
+ writer.writeLevel(level);
+ if (isValue()) {
+ try {
+ writer.writeValue(this);
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ }
+ next();
+ }
+ }
+ //Add last delimiter, or NULL/MISSING
+ writer.writeLevel(level);
+ }
+
+ private boolean isRepeatedValue() {
+ return levelToDelimiterMap[level] < delimiters.length;
+ }
+
+ private boolean isLastDelimiter() {
+ return isDelimiter() && delimiterIndex == delimiters.length - 1;
+ }
+
+ private void consumeDelimiterIfAny() {
+ if (isDelimiter()) {
+ delimiterIndex++;
+ }
+ }
+
+ private void setDelimiterIndex() {
+ if (isDelimiter() || level <= delimiters[delimiters.length - 1]) {
+ return;
+ }
+ delimiterIndex = levelToDelimiterMap[level];
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/FilterAccessorProvider.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/FilterAccessorProvider.java
new file mode 100644
index 0000000..2c8cf19
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/FilterAccessorProvider.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.column.values.reader.filter;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.asterix.column.metadata.schema.ObjectSchemaNode;
+import org.apache.asterix.column.metadata.schema.primitive.PrimitiveSchemaNode;
+import org.apache.asterix.column.metadata.schema.visitor.PathExtractorVisitor;
+import org.apache.asterix.column.metadata.schema.visitor.SchemaClipperVisitor;
+import org.apache.asterix.column.values.reader.filter.value.ColumnFilterValueAccessor;
+import org.apache.asterix.column.values.reader.filter.value.NoOpColumnFilterValueAccessor;
+import org.apache.asterix.column.values.writer.filters.AbstractColumnFilterWriter;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class FilterAccessorProvider {
+ private final ObjectSchemaNode root;
+ private final ObjectSchemaNode metaRoot;
+ private final SchemaClipperVisitor clipperVisitor;
+ private final PathExtractorVisitor pathExtractorVisitor;
+ private final Map<ARecordType, PrimitiveSchemaNode> cachedNodes;
+ private final List<IColumnFilterValueAccessor> filterAccessors;
+
+ public FilterAccessorProvider(ObjectSchemaNode root, SchemaClipperVisitor clipperVisitor) {
+ this(root, null, clipperVisitor);
+ }
+
+ public FilterAccessorProvider(ObjectSchemaNode root, ObjectSchemaNode metaRoot,
+ SchemaClipperVisitor clipperVisitor) {
+ this.root = root;
+ this.metaRoot = metaRoot;
+ this.clipperVisitor = clipperVisitor;
+ this.pathExtractorVisitor = new PathExtractorVisitor();
+ cachedNodes = new HashMap<>();
+ filterAccessors = new ArrayList<>();
+ }
+
+ public IColumnFilterValueAccessor createColumnFilterValueAccessor(ARecordType path, boolean min)
+ throws HyracksDataException {
+ PrimitiveSchemaNode node = getNode(path);
+ ATypeTag typeTag = node.getTypeTag();
+ if (typeTag == ATypeTag.MISSING) {
+ return NoOpColumnFilterValueAccessor.INSTANCE;
+ }
+ IColumnFilterValueAccessor accessor = new ColumnFilterValueAccessor(node.getColumnIndex(), typeTag, min);
+ filterAccessors.add(accessor);
+ return accessor;
+ }
+
+ public List<IColumnFilterValueAccessor> getFilterAccessors() {
+ return filterAccessors;
+ }
+
+ public static void setFilterValues(List<IColumnFilterValueAccessor> filterValueAccessors, ByteBuffer pageZero,
+ int numberOfColumns) {
+ for (int i = 0; i < filterValueAccessors.size(); i++) {
+ ColumnFilterValueAccessor accessor = (ColumnFilterValueAccessor) filterValueAccessors.get(i);
+ int columnIndex = accessor.getColumnIndex();
+ long normalizedValue;
+ if (columnIndex < numberOfColumns) {
+ int filterOffset = pageZero.position() + columnIndex * AbstractColumnFilterWriter.FILTER_SIZE;
+ normalizedValue =
+ accessor.isMin() ? pageZero.getLong(filterOffset) : pageZero.getLong(filterOffset + Long.BYTES);
+ } else {
+ // Column is missing
+ normalizedValue = accessor.isMin() ? Long.MAX_VALUE : Long.MIN_VALUE;
+ }
+ accessor.setNormalizedValue(normalizedValue);
+ }
+ }
+
+ private PrimitiveSchemaNode getNode(ARecordType path) throws HyracksDataException {
+ PrimitiveSchemaNode node = cachedNodes.get(path);
+ if (node == null) {
+ ObjectSchemaNode dataPath = (ObjectSchemaNode) path.accept(clipperVisitor, root);
+ node = (PrimitiveSchemaNode) dataPath.accept(pathExtractorVisitor, null);
+ if (node.getTypeTag() == ATypeTag.MISSING && metaRoot != null) {
+ //Try meta
+ ObjectSchemaNode metaPath = (ObjectSchemaNode) path.accept(clipperVisitor, metaRoot);
+ node = (PrimitiveSchemaNode) metaPath.accept(pathExtractorVisitor, null);
+ }
+ cachedNodes.put(path, node);
+ }
+ return node;
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/IColumnFilterEvaluator.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/IColumnFilterEvaluator.java
new file mode 100644
index 0000000..bc1f51c
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/IColumnFilterEvaluator.java
@@ -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.
+ */
+package org.apache.asterix.column.values.reader.filter;
+
+public interface IColumnFilterEvaluator {
+ boolean evaluate();
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/IColumnFilterEvaluatorFactory.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/IColumnFilterEvaluatorFactory.java
new file mode 100644
index 0000000..62abb92
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/IColumnFilterEvaluatorFactory.java
@@ -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.
+ */
+package org.apache.asterix.column.values.reader.filter;
+
+import java.io.Serializable;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+@FunctionalInterface
+public interface IColumnFilterEvaluatorFactory extends Serializable {
+ IColumnFilterEvaluator create(FilterAccessorProvider filterAccessorProvider) throws HyracksDataException;
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/IColumnFilterValueAccessor.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/IColumnFilterValueAccessor.java
new file mode 100644
index 0000000..69abb37
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/IColumnFilterValueAccessor.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.column.values.reader.filter;
+
+import org.apache.asterix.om.types.ATypeTag;
+
+public interface IColumnFilterValueAccessor {
+ long getNormalizedValue();
+
+ ATypeTag getTypeTag();
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/IColumnFilterValueAccessorFactory.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/IColumnFilterValueAccessorFactory.java
new file mode 100644
index 0000000..9621c02
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/IColumnFilterValueAccessorFactory.java
@@ -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.
+ */
+package org.apache.asterix.column.values.reader.filter;
+
+import java.io.Serializable;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+@FunctionalInterface
+public interface IColumnFilterValueAccessorFactory extends Serializable {
+ IColumnFilterValueAccessor create(FilterAccessorProvider filterAccessorProvider) throws HyracksDataException;
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/compartor/AbstractColumnFilterComparatorFactory.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/compartor/AbstractColumnFilterComparatorFactory.java
new file mode 100644
index 0000000..8244876
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/compartor/AbstractColumnFilterComparatorFactory.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.column.values.reader.filter.compartor;
+
+import org.apache.asterix.column.values.reader.filter.FilterAccessorProvider;
+import org.apache.asterix.column.values.reader.filter.IColumnFilterEvaluator;
+import org.apache.asterix.column.values.reader.filter.IColumnFilterEvaluatorFactory;
+import org.apache.asterix.column.values.reader.filter.IColumnFilterValueAccessor;
+import org.apache.asterix.column.values.reader.filter.IColumnFilterValueAccessorFactory;
+import org.apache.asterix.column.values.reader.filter.evaluator.FalseColumnFilterEvaluator;
+import org.apache.asterix.column.values.reader.filter.evaluator.TrueColumnFilterEvaluator;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+abstract class AbstractColumnFilterComparatorFactory implements IColumnFilterEvaluatorFactory {
+ private static final long serialVersionUID = 4229059703449173694L;
+ private final IColumnFilterValueAccessorFactory left;
+ private final IColumnFilterValueAccessorFactory right;
+
+ AbstractColumnFilterComparatorFactory(IColumnFilterValueAccessorFactory left,
+ IColumnFilterValueAccessorFactory right) {
+ this.left = left;
+ this.right = right;
+ }
+
+ @Override
+ public final IColumnFilterEvaluator create(FilterAccessorProvider filterAccessorProvider)
+ throws HyracksDataException {
+ IColumnFilterValueAccessor leftAccessor = left.create(filterAccessorProvider);
+ IColumnFilterValueAccessor rightAccessor = right.create(filterAccessorProvider);
+
+ ATypeTag leftTypeTag = leftAccessor.getTypeTag();
+ ATypeTag rightTypeTag = rightAccessor.getTypeTag();
+ if (leftTypeTag != rightTypeTag && ATypeHierarchy.isCompatible(leftTypeTag, rightTypeTag)) {
+ // Cannot compare comparable values with different types. Bail out.
+ return TrueColumnFilterEvaluator.INSTANCE;
+ } else if (cannotCompare(leftTypeTag, rightTypeTag)) {
+ return FalseColumnFilterEvaluator.INSTANCE;
+ }
+ return createComparator(leftAccessor, rightAccessor);
+ }
+
+ private boolean cannotCompare(ATypeTag leftTypeTag, ATypeTag rightTypeTag) {
+ return rightTypeTag == ATypeTag.MISSING || leftTypeTag != rightTypeTag;
+ }
+
+ protected abstract IColumnFilterEvaluator createComparator(IColumnFilterValueAccessor left,
+ IColumnFilterValueAccessor right);
+
+ protected abstract String getOpt();
+
+ @Override
+ public String toString() {
+ return left.toString() + " " + getOpt() + " " + right.toString();
+ }
+
+ static abstract class AbstractComparator implements IColumnFilterEvaluator {
+ protected final IColumnFilterValueAccessor left;
+ protected final IColumnFilterValueAccessor right;
+
+ AbstractComparator(IColumnFilterValueAccessor left, IColumnFilterValueAccessor right) {
+ this.left = left;
+ this.right = right;
+ }
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/compartor/GEColumnFilterEvaluatorFactory.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/compartor/GEColumnFilterEvaluatorFactory.java
new file mode 100644
index 0000000..81d58db
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/compartor/GEColumnFilterEvaluatorFactory.java
@@ -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.
+ */
+package org.apache.asterix.column.values.reader.filter.compartor;
+
+import org.apache.asterix.column.values.reader.filter.IColumnFilterEvaluator;
+import org.apache.asterix.column.values.reader.filter.IColumnFilterValueAccessor;
+import org.apache.asterix.column.values.reader.filter.IColumnFilterValueAccessorFactory;
+
+public class GEColumnFilterEvaluatorFactory extends AbstractColumnFilterComparatorFactory {
+ private static final long serialVersionUID = 6879193736347174789L;
+
+ public GEColumnFilterEvaluatorFactory(IColumnFilterValueAccessorFactory left,
+ IColumnFilterValueAccessorFactory right) {
+ super(left, right);
+ }
+
+ @Override
+ protected IColumnFilterEvaluator createComparator(IColumnFilterValueAccessor left,
+ IColumnFilterValueAccessor right) {
+ return new AbstractComparator(left, right) {
+ @Override
+ public boolean evaluate() {
+ return left.getNormalizedValue() >= right.getNormalizedValue();
+ }
+ };
+ }
+
+ @Override
+ protected String getOpt() {
+ return ">=";
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/compartor/GTColumnFilterEvaluatorFactory.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/compartor/GTColumnFilterEvaluatorFactory.java
new file mode 100644
index 0000000..b24e18bf
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/compartor/GTColumnFilterEvaluatorFactory.java
@@ -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.
+ */
+package org.apache.asterix.column.values.reader.filter.compartor;
+
+import org.apache.asterix.column.values.reader.filter.IColumnFilterEvaluator;
+import org.apache.asterix.column.values.reader.filter.IColumnFilterValueAccessor;
+import org.apache.asterix.column.values.reader.filter.IColumnFilterValueAccessorFactory;
+
+public class GTColumnFilterEvaluatorFactory extends AbstractColumnFilterComparatorFactory {
+ private static final long serialVersionUID = -3104103170926445020L;
+
+ public GTColumnFilterEvaluatorFactory(IColumnFilterValueAccessorFactory left,
+ IColumnFilterValueAccessorFactory right) {
+ super(left, right);
+ }
+
+ @Override
+ protected IColumnFilterEvaluator createComparator(IColumnFilterValueAccessor left,
+ IColumnFilterValueAccessor right) {
+ return new AbstractComparator(left, right) {
+ @Override
+ public boolean evaluate() {
+ return left.getNormalizedValue() > right.getNormalizedValue();
+ }
+ };
+ }
+
+ @Override
+ protected String getOpt() {
+ return ">";
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/compartor/LEColumnFilterEvaluatorFactory.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/compartor/LEColumnFilterEvaluatorFactory.java
new file mode 100644
index 0000000..f195d03
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/compartor/LEColumnFilterEvaluatorFactory.java
@@ -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.
+ */
+package org.apache.asterix.column.values.reader.filter.compartor;
+
+import org.apache.asterix.column.values.reader.filter.IColumnFilterEvaluator;
+import org.apache.asterix.column.values.reader.filter.IColumnFilterValueAccessor;
+import org.apache.asterix.column.values.reader.filter.IColumnFilterValueAccessorFactory;
+
+public class LEColumnFilterEvaluatorFactory extends AbstractColumnFilterComparatorFactory {
+ private static final long serialVersionUID = 1068661809768620550L;
+
+ public LEColumnFilterEvaluatorFactory(IColumnFilterValueAccessorFactory left,
+ IColumnFilterValueAccessorFactory right) {
+ super(left, right);
+ }
+
+ @Override
+ protected IColumnFilterEvaluator createComparator(IColumnFilterValueAccessor left,
+ IColumnFilterValueAccessor right) {
+ return new AbstractComparator(left, right) {
+ @Override
+ public boolean evaluate() {
+ return left.getNormalizedValue() <= right.getNormalizedValue();
+ }
+ };
+ }
+
+ @Override
+ protected String getOpt() {
+ return "<=";
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/compartor/LTColumnFilterEvaluatorFactory.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/compartor/LTColumnFilterEvaluatorFactory.java
new file mode 100644
index 0000000..41c8018
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/compartor/LTColumnFilterEvaluatorFactory.java
@@ -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.
+ */
+package org.apache.asterix.column.values.reader.filter.compartor;
+
+import org.apache.asterix.column.values.reader.filter.IColumnFilterEvaluator;
+import org.apache.asterix.column.values.reader.filter.IColumnFilterValueAccessor;
+import org.apache.asterix.column.values.reader.filter.IColumnFilterValueAccessorFactory;
+
+public class LTColumnFilterEvaluatorFactory extends AbstractColumnFilterComparatorFactory {
+ private static final long serialVersionUID = -4066709771630858677L;
+
+ public LTColumnFilterEvaluatorFactory(IColumnFilterValueAccessorFactory left,
+ IColumnFilterValueAccessorFactory right) {
+ super(left, right);
+ }
+
+ @Override
+ protected IColumnFilterEvaluator createComparator(IColumnFilterValueAccessor left,
+ IColumnFilterValueAccessor right) {
+ return new AbstractComparator(left, right) {
+ @Override
+ public boolean evaluate() {
+ return left.getNormalizedValue() < right.getNormalizedValue();
+ }
+ };
+ }
+
+ @Override
+ protected String getOpt() {
+ return "<";
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/evaluator/ANDColumnFilterEvaluatorFactory.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/evaluator/ANDColumnFilterEvaluatorFactory.java
new file mode 100644
index 0000000..03ae64e
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/evaluator/ANDColumnFilterEvaluatorFactory.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.column.values.reader.filter.evaluator;
+
+import org.apache.asterix.column.values.reader.filter.FilterAccessorProvider;
+import org.apache.asterix.column.values.reader.filter.IColumnFilterEvaluator;
+import org.apache.asterix.column.values.reader.filter.IColumnFilterEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class ANDColumnFilterEvaluatorFactory extends AbstractColumnFilterEvaluatorFactory {
+ private static final long serialVersionUID = -7902069740719309586L;
+
+ public ANDColumnFilterEvaluatorFactory(IColumnFilterEvaluatorFactory left, IColumnFilterEvaluatorFactory right) {
+ super(left, right);
+ }
+
+ @Override
+ public IColumnFilterEvaluator create(FilterAccessorProvider filterAccessorProvider) throws HyracksDataException {
+ IColumnFilterEvaluator leftEval = left.create(filterAccessorProvider);
+ IColumnFilterEvaluator rightEval = right.create(filterAccessorProvider);
+ if (leftEval == FalseColumnFilterEvaluator.INSTANCE || rightEval == FalseColumnFilterEvaluator.INSTANCE) {
+ // Either is false, then return false
+ return FalseColumnFilterEvaluator.INSTANCE;
+ } else if (leftEval == TrueColumnFilterEvaluator.INSTANCE && rightEval == TrueColumnFilterEvaluator.INSTANCE) {
+ //Skip both operands and return TrueColumnFilterEvaluator
+ return TrueColumnFilterEvaluator.INSTANCE;
+ } else if (leftEval == TrueColumnFilterEvaluator.INSTANCE) {
+ //Left is true, return the right evaluator
+ return rightEval;
+ } else if (rightEval == TrueColumnFilterEvaluator.INSTANCE) {
+ //Same as above but the right is true
+ return leftEval;
+ } else {
+ //Both are actual filters
+ return create(leftEval, rightEval);
+ }
+ }
+
+ private IColumnFilterEvaluator create(IColumnFilterEvaluator left, IColumnFilterEvaluator right) {
+ return new AbstractEvaluator(left, right) {
+ @Override
+ public boolean evaluate() {
+ return left.evaluate() && right.evaluate();
+ }
+ };
+ }
+
+ @Override
+ protected String getOp() {
+ return "&&";
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/evaluator/AbstractColumnFilterEvaluatorFactory.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/evaluator/AbstractColumnFilterEvaluatorFactory.java
new file mode 100644
index 0000000..d1a53d1
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/evaluator/AbstractColumnFilterEvaluatorFactory.java
@@ -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.
+ */
+package org.apache.asterix.column.values.reader.filter.evaluator;
+
+import org.apache.asterix.column.values.reader.filter.IColumnFilterEvaluator;
+import org.apache.asterix.column.values.reader.filter.IColumnFilterEvaluatorFactory;
+
+public abstract class AbstractColumnFilterEvaluatorFactory implements IColumnFilterEvaluatorFactory {
+ private static final long serialVersionUID = 1436531448052787426L;
+
+ protected final IColumnFilterEvaluatorFactory left;
+ protected final IColumnFilterEvaluatorFactory right;
+
+ public AbstractColumnFilterEvaluatorFactory(IColumnFilterEvaluatorFactory left,
+ IColumnFilterEvaluatorFactory right) {
+ this.left = left;
+ this.right = right;
+ }
+
+ protected abstract String getOp();
+
+ @Override
+ public String toString() {
+ return left.toString() + " " + getOp() + " " + right.toString();
+ }
+
+ static abstract class AbstractEvaluator implements IColumnFilterEvaluator {
+ protected final IColumnFilterEvaluator left;
+ protected final IColumnFilterEvaluator right;
+
+ AbstractEvaluator(IColumnFilterEvaluator left, IColumnFilterEvaluator right) {
+ this.left = left;
+ this.right = right;
+ }
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/evaluator/FalseColumnFilterEvaluator.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/evaluator/FalseColumnFilterEvaluator.java
new file mode 100644
index 0000000..fccd015
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/evaluator/FalseColumnFilterEvaluator.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.column.values.reader.filter.evaluator;
+
+import org.apache.asterix.column.values.reader.filter.IColumnFilterEvaluator;
+
+public class FalseColumnFilterEvaluator implements IColumnFilterEvaluator {
+ public static final IColumnFilterEvaluator INSTANCE = new FalseColumnFilterEvaluator();
+
+ private FalseColumnFilterEvaluator() {
+ }
+
+ @Override
+ public boolean evaluate() {
+ return false;
+ }
+
+ @Override
+ public String toString() {
+ return "FALSE";
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/evaluator/NoOpColumnFilterEvaluatorFactory.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/evaluator/NoOpColumnFilterEvaluatorFactory.java
new file mode 100644
index 0000000..765f3c6
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/evaluator/NoOpColumnFilterEvaluatorFactory.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.column.values.reader.filter.evaluator;
+
+import org.apache.asterix.column.values.reader.filter.FilterAccessorProvider;
+import org.apache.asterix.column.values.reader.filter.IColumnFilterEvaluator;
+import org.apache.asterix.column.values.reader.filter.IColumnFilterEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class NoOpColumnFilterEvaluatorFactory implements IColumnFilterEvaluatorFactory {
+ private static final long serialVersionUID = -7122361396576592000L;
+ public static final IColumnFilterEvaluatorFactory INSTANCE = new NoOpColumnFilterEvaluatorFactory();
+
+ private NoOpColumnFilterEvaluatorFactory() {
+ }
+
+ @Override
+ public IColumnFilterEvaluator create(FilterAccessorProvider filterAccessorProvider) throws HyracksDataException {
+ // True is also NoOp
+ return TrueColumnFilterEvaluator.INSTANCE;
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/evaluator/ORColumnFilterEvaluatorFactory.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/evaluator/ORColumnFilterEvaluatorFactory.java
new file mode 100644
index 0000000..aedc9f4
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/evaluator/ORColumnFilterEvaluatorFactory.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.column.values.reader.filter.evaluator;
+
+import org.apache.asterix.column.values.reader.filter.FilterAccessorProvider;
+import org.apache.asterix.column.values.reader.filter.IColumnFilterEvaluator;
+import org.apache.asterix.column.values.reader.filter.IColumnFilterEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class ORColumnFilterEvaluatorFactory extends AbstractColumnFilterEvaluatorFactory {
+ private static final long serialVersionUID = 9029706131191375500L;
+
+ public ORColumnFilterEvaluatorFactory(IColumnFilterEvaluatorFactory left, IColumnFilterEvaluatorFactory right) {
+ super(left, right);
+ }
+
+ @Override
+ public IColumnFilterEvaluator create(FilterAccessorProvider filterAccessorProvider) throws HyracksDataException {
+ IColumnFilterEvaluator leftEval = left.create(filterAccessorProvider);
+ IColumnFilterEvaluator rightEval = right.create(filterAccessorProvider);
+ if (leftEval == TrueColumnFilterEvaluator.INSTANCE || rightEval == TrueColumnFilterEvaluator.INSTANCE) {
+ // Either is true, then return true
+ return TrueColumnFilterEvaluator.INSTANCE;
+ } else if (leftEval == FalseColumnFilterEvaluator.INSTANCE
+ && rightEval == FalseColumnFilterEvaluator.INSTANCE) {
+ // Both are false, then return false
+ return FalseColumnFilterEvaluator.INSTANCE;
+ } else if (leftEval == FalseColumnFilterEvaluator.INSTANCE) {
+ //Left is NoOp, which evaluates to true, return the right evaluator
+ return rightEval;
+ } else if (rightEval == FalseColumnFilterEvaluator.INSTANCE) {
+ //Same as above but the right is NoOp
+ return leftEval;
+ } else {
+ //Both are actual evaluators
+ return create(leftEval, rightEval);
+ }
+ }
+
+ private IColumnFilterEvaluator create(IColumnFilterEvaluator left, IColumnFilterEvaluator right) {
+ return new AbstractEvaluator(left, right) {
+ @Override
+ public boolean evaluate() {
+ return left.evaluate() || right.evaluate();
+ }
+ };
+ }
+
+ @Override
+ protected String getOp() {
+ return "||";
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/evaluator/TrueColumnFilterEvaluator.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/evaluator/TrueColumnFilterEvaluator.java
new file mode 100644
index 0000000..37f1c5c
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/evaluator/TrueColumnFilterEvaluator.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.column.values.reader.filter.evaluator;
+
+import org.apache.asterix.column.values.reader.filter.IColumnFilterEvaluator;
+
+/**
+ * This evaluator is also used to indicate a NoOp filter
+ */
+public class TrueColumnFilterEvaluator implements IColumnFilterEvaluator {
+ public static final IColumnFilterEvaluator INSTANCE = new TrueColumnFilterEvaluator();
+
+ private TrueColumnFilterEvaluator() {
+ }
+
+ @Override
+ public boolean evaluate() {
+ return true;
+ }
+
+ @Override
+ public String toString() {
+ return "TRUE";
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/value/ColumnFilterValueAccessor.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/value/ColumnFilterValueAccessor.java
new file mode 100644
index 0000000..0aa2b31
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/value/ColumnFilterValueAccessor.java
@@ -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.
+ */
+package org.apache.asterix.column.values.reader.filter.value;
+
+import org.apache.asterix.column.values.reader.filter.IColumnFilterValueAccessor;
+import org.apache.asterix.om.types.ATypeTag;
+
+public class ColumnFilterValueAccessor implements IColumnFilterValueAccessor {
+ private final int columnIndex;
+ private final ATypeTag typeTag;
+ private final boolean min;
+ private long normalizedValue;
+
+ public ColumnFilterValueAccessor(int columnIndex, ATypeTag typeTag, boolean min) {
+ this.columnIndex = columnIndex;
+ this.typeTag = typeTag;
+ this.min = min;
+ }
+
+ public int getColumnIndex() {
+ return columnIndex;
+ }
+
+ public boolean isMin() {
+ return min;
+ }
+
+ public void setNormalizedValue(long normalizedValue) {
+ this.normalizedValue = normalizedValue;
+ }
+
+ @Override
+ public long getNormalizedValue() {
+ return normalizedValue;
+ }
+
+ @Override
+ public ATypeTag getTypeTag() {
+ return typeTag;
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/value/ColumnFilterValueAccessorFactory.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/value/ColumnFilterValueAccessorFactory.java
new file mode 100644
index 0000000..fb02321
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/value/ColumnFilterValueAccessorFactory.java
@@ -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.
+ */
+package org.apache.asterix.column.values.reader.filter.value;
+
+import org.apache.asterix.column.values.reader.filter.FilterAccessorProvider;
+import org.apache.asterix.column.values.reader.filter.IColumnFilterValueAccessor;
+import org.apache.asterix.column.values.reader.filter.IColumnFilterValueAccessorFactory;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.visitor.PathStringBuilderForIATypeVisitor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class ColumnFilterValueAccessorFactory implements IColumnFilterValueAccessorFactory {
+ private static final long serialVersionUID = -6341611172763952841L;
+ private final ARecordType path;
+ private final boolean min;
+
+ public ColumnFilterValueAccessorFactory(ARecordType path, boolean min) {
+ this.path = path;
+ this.min = min;
+ }
+
+ @Override
+ public IColumnFilterValueAccessor create(FilterAccessorProvider filterAccessorProvider)
+ throws HyracksDataException {
+ return filterAccessorProvider.createColumnFilterValueAccessor(path, min);
+ }
+
+ @Override
+ public String toString() {
+ PathStringBuilderForIATypeVisitor pathBuilder = new PathStringBuilderForIATypeVisitor();
+ StringBuilder stringBuilder = new StringBuilder();
+
+ stringBuilder.append(min ? "min" : "max");
+ stringBuilder.append('(');
+ path.accept(pathBuilder, stringBuilder);
+ stringBuilder.append(')');
+
+ return stringBuilder.toString();
+ }
+
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/value/ConstantColumnFilterValueAccessor.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/value/ConstantColumnFilterValueAccessor.java
new file mode 100644
index 0000000..db9ef9c
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/value/ConstantColumnFilterValueAccessor.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.column.values.reader.filter.value;
+
+import org.apache.asterix.column.values.reader.filter.IColumnFilterValueAccessor;
+import org.apache.asterix.om.types.ATypeTag;
+
+public final class ConstantColumnFilterValueAccessor implements IColumnFilterValueAccessor {
+ private final long normalizedValue;
+ private final ATypeTag typeTag;
+
+ //TODO add UUID
+
+ public ConstantColumnFilterValueAccessor(long normalizedValue, ATypeTag typeTag) {
+ this.normalizedValue = normalizedValue;
+ this.typeTag = typeTag;
+ }
+
+ @Override
+ public long getNormalizedValue() {
+ return normalizedValue;
+ }
+
+ @Override
+ public ATypeTag getTypeTag() {
+ return typeTag;
+ }
+
+ @Override
+ public String toString() {
+ return Long.toString(normalizedValue);
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/value/ConstantColumnFilterValueAccessorFactory.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/value/ConstantColumnFilterValueAccessorFactory.java
new file mode 100644
index 0000000..eeb903d
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/value/ConstantColumnFilterValueAccessorFactory.java
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.column.values.reader.filter.value;
+
+import org.apache.asterix.column.values.reader.filter.FilterAccessorProvider;
+import org.apache.asterix.column.values.reader.filter.IColumnFilterValueAccessor;
+import org.apache.asterix.column.values.reader.filter.IColumnFilterValueAccessorFactory;
+import org.apache.asterix.om.base.ADouble;
+import org.apache.asterix.om.base.AInt64;
+import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.base.IAObject;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class ConstantColumnFilterValueAccessorFactory implements IColumnFilterValueAccessorFactory {
+ private static final long serialVersionUID = -4835407779342615453L;
+ private final long normalizedValue;
+ private final ATypeTag typeTag;
+ private final String stringValue;
+
+ private ConstantColumnFilterValueAccessorFactory(String stringValue, long normalizedValue, ATypeTag typeTag) {
+ this.stringValue = stringValue;
+ this.normalizedValue = normalizedValue;
+ this.typeTag = typeTag;
+ }
+
+ public static ConstantColumnFilterValueAccessorFactory createFactory(IAObject value) {
+ String stringValue;
+ long normalizedValue;
+ ATypeTag typeTag = value.getType().getTypeTag();
+ switch (typeTag) {
+ case BIGINT:
+ long longVal = ((AInt64) value).getLongValue();
+ stringValue = Long.toString(longVal);
+ normalizedValue = longVal;
+ break;
+ case DOUBLE:
+ double doubleVal = ((ADouble) value).getDoubleValue();
+ stringValue = Double.toString(doubleVal);
+ normalizedValue = Double.doubleToLongBits(doubleVal);
+ break;
+ case STRING:
+ stringValue = ((AString) value).getStringValue();
+ normalizedValue = normalize(stringValue);
+ break;
+ default:
+ return null;
+ }
+
+ return new ConstantColumnFilterValueAccessorFactory(stringValue, normalizedValue, typeTag);
+ }
+
+ @Override
+ public IColumnFilterValueAccessor create(FilterAccessorProvider filterAccessorProvider)
+ throws HyracksDataException {
+ return new ConstantColumnFilterValueAccessor(normalizedValue, typeTag);
+ }
+
+ @Override
+ public String toString() {
+ if (typeTag == ATypeTag.STRING) {
+ return "\"" + stringValue + "\"";
+ }
+ return stringValue;
+ }
+
+ private static long normalize(String value) {
+ long nk = 0;
+ for (int i = 0; i < 4; ++i) {
+ nk <<= 16;
+ if (i < value.length()) {
+ nk += value.charAt(i) & 0xffff;
+ }
+ }
+ //Make it always positive
+ return nk >>> 1;
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/value/NoOpColumnFilterValueAccessor.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/value/NoOpColumnFilterValueAccessor.java
new file mode 100644
index 0000000..c52eeda
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/filter/value/NoOpColumnFilterValueAccessor.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.asterix.column.values.reader.filter.value;
+
+import org.apache.asterix.column.values.reader.filter.IColumnFilterValueAccessor;
+import org.apache.asterix.om.types.ATypeTag;
+
+public class NoOpColumnFilterValueAccessor implements IColumnFilterValueAccessor {
+ public static final IColumnFilterValueAccessor INSTANCE = new NoOpColumnFilterValueAccessor();
+
+ private NoOpColumnFilterValueAccessor() {
+ }
+
+ @Override
+ public long getNormalizedValue() {
+ throw new IllegalStateException("should not be invoked");
+ }
+
+ @Override
+ public ATypeTag getTypeTag() {
+ return ATypeTag.MISSING;
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/value/AbstractValueReader.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/value/AbstractValueReader.java
new file mode 100644
index 0000000..3d4c744
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/value/AbstractValueReader.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.column.values.reader.value;
+
+import java.io.IOException;
+
+import org.apache.asterix.column.bytes.stream.in.AbstractBytesInputStream;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+
+public abstract class AbstractValueReader implements Comparable<AbstractValueReader> {
+
+ public abstract void resetValue(AbstractBytesInputStream in) throws IOException;
+
+ public abstract void nextValue() throws HyracksDataException;
+
+ public abstract ATypeTag getTypeTag();
+
+ public boolean getBoolean() {
+ throw new UnsupportedOperationException(getClass().getName());
+ }
+
+ public long getLong() {
+ throw new UnsupportedOperationException(getClass().getName());
+ }
+
+ public double getDouble() {
+ throw new UnsupportedOperationException(getClass().getName());
+ }
+
+ public IValueReference getBytes() {
+ throw new UnsupportedOperationException(getClass().getName());
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/value/BooleanValueReader.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/value/BooleanValueReader.java
new file mode 100644
index 0000000..3417773
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/value/BooleanValueReader.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.column.values.reader.value;
+
+import java.io.IOException;
+
+import org.apache.asterix.column.bytes.decoder.ParquetRunLengthBitPackingHybridDecoder;
+import org.apache.asterix.column.bytes.stream.in.AbstractBytesInputStream;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public final class BooleanValueReader extends AbstractValueReader {
+ private final ParquetRunLengthBitPackingHybridDecoder booleanReader;
+ private boolean nextValue;
+
+ public BooleanValueReader() {
+ booleanReader = new ParquetRunLengthBitPackingHybridDecoder(1);
+ }
+
+ @Override
+ public void resetValue(AbstractBytesInputStream in) {
+ booleanReader.reset(in);
+ }
+
+ @Override
+ public void nextValue() throws HyracksDataException {
+ try {
+ nextValue = booleanReader.readInt() == 1;
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+
+ }
+
+ @Override
+ public boolean getBoolean() {
+ return nextValue;
+ }
+
+ @Override
+ public ATypeTag getTypeTag() {
+ return ATypeTag.BOOLEAN;
+ }
+
+ @Override
+ public int compareTo(AbstractValueReader o) {
+ return Boolean.compare(nextValue, o.getBoolean());
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/value/DoubleValueReader.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/value/DoubleValueReader.java
new file mode 100644
index 0000000..24155f2
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/value/DoubleValueReader.java
@@ -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.
+ */
+package org.apache.asterix.column.values.reader.value;
+
+import java.io.IOException;
+
+import org.apache.asterix.column.bytes.decoder.ParquetDoublePlainValuesReader;
+import org.apache.asterix.column.bytes.stream.in.AbstractBytesInputStream;
+import org.apache.asterix.om.types.ATypeTag;
+
+public final class DoubleValueReader extends AbstractValueReader {
+ private final ParquetDoublePlainValuesReader doubleReader;
+ private double nextValue;
+
+ public DoubleValueReader() {
+ doubleReader = new ParquetDoublePlainValuesReader();
+ }
+
+ @Override
+ public void resetValue(AbstractBytesInputStream in) throws IOException {
+ doubleReader.initFromPage(in);
+ }
+
+ @Override
+ public void nextValue() {
+ nextValue = doubleReader.readDouble();
+ }
+
+ @Override
+ public double getDouble() {
+ return nextValue;
+ }
+
+ @Override
+ public ATypeTag getTypeTag() {
+ return ATypeTag.DOUBLE;
+ }
+
+ @Override
+ public int compareTo(AbstractValueReader o) {
+ return Double.compare(nextValue, o.getDouble());
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/value/LongValueReader.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/value/LongValueReader.java
new file mode 100644
index 0000000..09413d9
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/value/LongValueReader.java
@@ -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.
+ */
+package org.apache.asterix.column.values.reader.value;
+
+import java.io.IOException;
+
+import org.apache.asterix.column.bytes.decoder.ParquetDeltaBinaryPackingValuesReader;
+import org.apache.asterix.column.bytes.stream.in.AbstractBytesInputStream;
+import org.apache.asterix.om.types.ATypeTag;
+
+public final class LongValueReader extends AbstractValueReader {
+ private final ParquetDeltaBinaryPackingValuesReader longReader;
+ private long nextValue;
+
+ public LongValueReader() {
+ longReader = new ParquetDeltaBinaryPackingValuesReader();
+ }
+
+ @Override
+ public void resetValue(AbstractBytesInputStream in) throws IOException {
+ longReader.initFromPage(in);
+ }
+
+ @Override
+ public void nextValue() {
+ nextValue = longReader.readLong();
+ }
+
+ @Override
+ public long getLong() {
+ return nextValue;
+ }
+
+ @Override
+ public ATypeTag getTypeTag() {
+ return ATypeTag.BIGINT;
+ }
+
+ @Override
+ public int compareTo(AbstractValueReader o) {
+ return Long.compare(nextValue, o.getLong());
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/value/NoOpValueReader.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/value/NoOpValueReader.java
new file mode 100644
index 0000000..fd56ff2
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/value/NoOpValueReader.java
@@ -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.
+ */
+package org.apache.asterix.column.values.reader.value;
+
+import java.io.IOException;
+
+import org.apache.asterix.column.bytes.stream.in.AbstractBytesInputStream;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class NoOpValueReader extends AbstractValueReader {
+ public static final AbstractValueReader INSTANCE = new NoOpValueReader();
+
+ private NoOpValueReader() {
+ }
+
+ @Override
+ public void resetValue(AbstractBytesInputStream in) throws IOException {
+ throw new UnsupportedOperationException(getClass().getName());
+ }
+
+ @Override
+ public void nextValue() throws HyracksDataException {
+ throw new UnsupportedOperationException(getClass().getName());
+ }
+
+ @Override
+ public ATypeTag getTypeTag() {
+ throw new UnsupportedOperationException(getClass().getName());
+ }
+
+ @Override
+ public int compareTo(AbstractValueReader o) {
+ throw new UnsupportedOperationException(getClass().getName());
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/value/StringValueReader.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/value/StringValueReader.java
new file mode 100644
index 0000000..8fd8874
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/value/StringValueReader.java
@@ -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.
+ */
+package org.apache.asterix.column.values.reader.value;
+
+import java.io.IOException;
+
+import org.apache.asterix.column.bytes.decoder.ParquetDeltaByteArrayReader;
+import org.apache.asterix.column.bytes.stream.in.AbstractBytesInputStream;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+
+public final class StringValueReader extends AbstractValueReader {
+ private final ParquetDeltaByteArrayReader stringReader;
+ private IValueReference nextValue;
+
+ public StringValueReader() {
+ stringReader = new ParquetDeltaByteArrayReader(true);
+ }
+
+ @Override
+ public void resetValue(AbstractBytesInputStream in) throws IOException {
+ stringReader.initFromPage(in);
+ }
+
+ @Override
+ public void nextValue() {
+ nextValue = stringReader.readBytes();
+ }
+
+ @Override
+ public IValueReference getBytes() {
+ return nextValue;
+ }
+
+ @Override
+ public ATypeTag getTypeTag() {
+ return ATypeTag.STRING;
+ }
+
+ @Override
+ public int compareTo(AbstractValueReader o) {
+ return UTF8StringPointable.compare(nextValue, o.getBytes());
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/value/UUIDValueReader.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/value/UUIDValueReader.java
new file mode 100644
index 0000000..4f240e9
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/value/UUIDValueReader.java
@@ -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.
+ */
+package org.apache.asterix.column.values.reader.value;
+
+import java.io.IOException;
+
+import org.apache.asterix.column.bytes.decoder.ParquetDeltaByteArrayReader;
+import org.apache.asterix.column.bytes.stream.in.AbstractBytesInputStream;
+import org.apache.asterix.dataflow.data.nontagged.comparators.AUUIDPartialBinaryComparatorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.data.std.api.IValueReference;
+
+public final class UUIDValueReader extends AbstractValueReader {
+ private final ParquetDeltaByteArrayReader uuidReader;
+ private IValueReference nextValue;
+
+ public UUIDValueReader() {
+ uuidReader = new ParquetDeltaByteArrayReader(false);
+ }
+
+ @Override
+ public void resetValue(AbstractBytesInputStream in) throws IOException {
+ uuidReader.initFromPage(in);
+ }
+
+ @Override
+ public void nextValue() {
+ nextValue = uuidReader.readBytes();
+ }
+
+ @Override
+ public IValueReference getBytes() {
+ return nextValue;
+ }
+
+ @Override
+ public ATypeTag getTypeTag() {
+ return ATypeTag.UUID;
+ }
+
+ @Override
+ public int compareTo(AbstractValueReader o) {
+ return AUUIDPartialBinaryComparatorFactory.compare(nextValue, o.getBytes());
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/writer/AbstractColumnValuesWriter.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/writer/AbstractColumnValuesWriter.java
new file mode 100644
index 0000000..87eda82
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/writer/AbstractColumnValuesWriter.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.column.values.writer;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.OutputStream;
+
+import org.apache.asterix.column.bytes.encoder.ParquetRunLengthBitPackingHybridEncoder;
+import org.apache.asterix.column.util.ColumnValuesUtil;
+import org.apache.asterix.column.util.RunLengthIntArray;
+import org.apache.asterix.column.values.IColumnValuesReader;
+import org.apache.asterix.column.values.IColumnValuesWriter;
+import org.apache.asterix.column.values.IColumnValuesWriterFactory;
+import org.apache.asterix.column.values.writer.filters.AbstractColumnFilterWriter;
+import org.apache.asterix.column.values.writer.filters.NoOpColumnFilterWriter;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.bytes.BytesUtils;
+
+public abstract class AbstractColumnValuesWriter implements IColumnValuesWriter {
+ protected final AbstractColumnFilterWriter filterWriter;
+ protected final ParquetRunLengthBitPackingHybridEncoder definitionLevels;
+ protected final int level;
+
+ private final int columnIndex;
+ private final boolean collection;
+ private final int nullBitMask;
+ private int count;
+ private boolean writeValues;
+
+ AbstractColumnValuesWriter(int columnIndex, int level, boolean collection, boolean filtered) {
+ this.columnIndex = columnIndex;
+ this.level = level;
+ this.collection = collection;
+ nullBitMask = ColumnValuesUtil.getNullMask(level);
+ int width = ColumnValuesUtil.getBitWidth(level);
+ definitionLevels = new ParquetRunLengthBitPackingHybridEncoder(width);
+ this.filterWriter = filtered ? createFilter() : NoOpColumnFilterWriter.INSTANCE;
+ }
+
+ @Override
+ public final int getColumnIndex() {
+ return columnIndex;
+ }
+
+ @Override
+ public final int getEstimatedSize() {
+ return definitionLevels.getEstimatedSize() + getValuesEstimatedSize();
+ }
+
+ @Override
+ public final int getAllocatedSpace() {
+ return definitionLevels.getAllocatedSize() + getValuesAllocatedSize();
+ }
+
+ @Override
+ public final int getCount() {
+ return count;
+ }
+
+ @Override
+ public final void writeValue(ATypeTag tag, IValueReference value) throws HyracksDataException {
+ addLevel(level);
+ try {
+ addValue(tag, value);
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ }
+
+ @Override
+ public final void writeLevel(int level) throws HyracksDataException {
+ addLevel(level);
+ }
+
+ @Override
+ public void writeLevels(int level, int count) throws HyracksDataException {
+ writeValues = writeValues || this.level == level;
+ this.count += count;
+ try {
+ for (int i = 0; i < count; i++) {
+ definitionLevels.writeInt(level);
+ }
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ }
+
+ @Override
+ public RunLengthIntArray getDefinitionLevelsIntArray() {
+ return null;
+ }
+
+ @Override
+ public final void writeNull(int level) throws HyracksDataException {
+ addLevel(level | nullBitMask);
+ }
+
+ @Override
+ public void writeValue(IColumnValuesReader reader) throws HyracksDataException {
+ try {
+ addValue(reader);
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ }
+
+ @Override
+ public void writeAntiMatter(ATypeTag tag, IValueReference value) throws HyracksDataException {
+ addLevel(0);
+ try {
+ addValue(tag, value);
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ }
+
+ @Override
+ public final void close() {
+ definitionLevels.close();
+ closeValues();
+ }
+
+ @Override
+ public final long getNormalizedMinValue() {
+ if (!writeValues) {
+ // ignore values as everything is missing/null
+ return Long.MAX_VALUE;
+ }
+ return filterWriter.getMinNormalizedValue();
+ }
+
+ @Override
+ public final long getNormalizedMaxValue() {
+ if (!writeValues) {
+ // ignore values as everything is missing/null
+ return Long.MIN_VALUE;
+ }
+ return filterWriter.getMaxNormalizedValue();
+ }
+
+ @Override
+ public final void flush(OutputStream out) throws HyracksDataException {
+ BytesInput values;
+ BytesInput defLevelBytes;
+ try {
+ BytesUtils.writeZigZagVarInt(level, out);
+ defLevelBytes = definitionLevels.toBytes();
+ BytesUtils.writeZigZagVarInt((int) defLevelBytes.size(), out);
+ BytesUtils.writeZigZagVarInt(count, out);
+ defLevelBytes.writeAllTo(out);
+ if (writeValues || collection) {
+ values = getBytes();
+ int valueSize = (int) values.size();
+ BytesUtils.writeZigZagVarInt(valueSize, out);
+ values.writeAllTo(out);
+ } else {
+ /*
+ * Do not write the values if all values are null/missing
+ */
+ BytesUtils.writeZigZagVarInt(0, out);
+ }
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ reset();
+ }
+
+ @Override
+ public final void reset() throws HyracksDataException {
+ definitionLevels.reset();
+ writeValues = false;
+ count = 0;
+ filterWriter.reset();
+ resetValues();
+ }
+
+ @Override
+ public final void serialize(DataOutput output) throws IOException {
+ output.write(getTypeTag().serialize());
+ output.writeInt(columnIndex);
+ output.writeInt(level);
+ output.writeBoolean(collection);
+ output.writeBoolean(filterWriter != NoOpColumnFilterWriter.INSTANCE);
+ }
+
+ public static IColumnValuesWriter deserialize(DataInput input, IColumnValuesWriterFactory writerFactory)
+ throws IOException {
+ ATypeTag typeTag = ATypeTag.VALUE_TYPE_MAPPING[input.readByte()];
+ int columnIndex = input.readInt();
+ int level = input.readInt();
+ boolean collection = input.readBoolean();
+ boolean filtered = input.readBoolean();
+ return writerFactory.createValueWriter(typeTag, columnIndex, level, collection, filtered);
+ }
+
+ protected void addLevel(int level) throws HyracksDataException {
+ try {
+ writeValues = writeValues || this.level == level;
+ definitionLevels.writeInt(level);
+ count++;
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ }
+
+ protected abstract ATypeTag getTypeTag();
+
+ protected abstract void addValue(ATypeTag tag, IValueReference value) throws IOException;
+
+ protected abstract void addValue(IColumnValuesReader reader) throws IOException;
+
+ protected abstract BytesInput getBytes() throws IOException;
+
+ protected abstract int getValuesEstimatedSize();
+
+ protected abstract int getValuesAllocatedSize();
+
+ protected abstract AbstractColumnFilterWriter createFilter();
+
+ protected abstract void resetValues() throws HyracksDataException;
+
+ protected abstract void closeValues();
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/writer/BooleanColumnValuesWriter.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/writer/BooleanColumnValuesWriter.java
new file mode 100644
index 0000000..0058f18
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/writer/BooleanColumnValuesWriter.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.column.values.writer;
+
+import java.io.IOException;
+
+import org.apache.asterix.column.bytes.encoder.ParquetRunLengthBitPackingHybridEncoder;
+import org.apache.asterix.column.values.IColumnValuesReader;
+import org.apache.asterix.column.values.writer.filters.AbstractColumnFilterWriter;
+import org.apache.asterix.column.values.writer.filters.LongColumnFilterWriter;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.parquet.bytes.BytesInput;
+
+public final class BooleanColumnValuesWriter extends AbstractColumnValuesWriter {
+ private final ParquetRunLengthBitPackingHybridEncoder booleanWriter;
+
+ public BooleanColumnValuesWriter(int columnIndex, int level, boolean collection, boolean filtered) {
+ super(columnIndex, level, collection, filtered);
+ booleanWriter = new ParquetRunLengthBitPackingHybridEncoder(1);
+ }
+
+ @Override
+ protected void addValue(ATypeTag tag, IValueReference value) throws IOException {
+ byte booleanValue = value.getByteArray()[value.getStartOffset()];
+ booleanWriter.writeInt(booleanValue);
+ filterWriter.addLong(booleanValue);
+ }
+
+ @Override
+ protected void resetValues() {
+ booleanWriter.reset();
+ }
+
+ @Override
+ protected BytesInput getBytes() throws IOException {
+ return booleanWriter.toBytes();
+ }
+
+ @Override
+ protected int getValuesEstimatedSize() {
+ return booleanWriter.getEstimatedSize();
+ }
+
+ @Override
+ protected int getValuesAllocatedSize() {
+ return booleanWriter.getAllocatedSize();
+ }
+
+ @Override
+ protected void addValue(IColumnValuesReader reader) throws IOException {
+ int value = reader.getBoolean() ? 1 : 0;
+ booleanWriter.writeInt(value);
+ }
+
+ @Override
+ protected AbstractColumnFilterWriter createFilter() {
+ return new LongColumnFilterWriter();
+ }
+
+ @Override
+ protected void closeValues() {
+ booleanWriter.close();
+ }
+
+ @Override
+ protected ATypeTag getTypeTag() {
+ return ATypeTag.BOOLEAN;
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/writer/ColumnBatchWriter.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/writer/ColumnBatchWriter.java
new file mode 100644
index 0000000..490afe7
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/writer/ColumnBatchWriter.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.column.values.writer;
+
+import static org.apache.asterix.column.values.writer.filters.AbstractColumnFilterWriter.FILTER_SIZE;
+
+import java.nio.ByteBuffer;
+import java.util.PriorityQueue;
+
+import org.apache.asterix.column.bytes.stream.out.ByteBufferOutputStream;
+import org.apache.asterix.column.bytes.stream.out.MultiPersistentBufferBytesOutputStream;
+import org.apache.asterix.column.bytes.stream.out.pointer.IReservedPointer;
+import org.apache.asterix.column.values.IColumnBatchWriter;
+import org.apache.asterix.column.values.IColumnValuesWriter;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnWriteMultiPageOp;
+
+/**
+ * A writer for a batch columns' values
+ */
+public final class ColumnBatchWriter implements IColumnBatchWriter {
+ private final ByteBufferOutputStream primaryKeys;
+ private final MultiPersistentBufferBytesOutputStream columns;
+ private final int pageSize;
+ private final float tolerance;
+ private final IReservedPointer columnLengthPointer;
+
+ private ByteBuffer pageZero;
+ private int columnsOffset;
+ private int filtersOffset;
+ private int primaryKeysOffset;
+ private int nonKeyColumnStartOffset;
+
+ public ColumnBatchWriter(Mutable<IColumnWriteMultiPageOp> multiPageOpRef, int pageSize, float tolerance) {
+ this.pageSize = pageSize;
+ this.tolerance = tolerance;
+ primaryKeys = new ByteBufferOutputStream();
+ columns = new MultiPersistentBufferBytesOutputStream(multiPageOpRef);
+ columnLengthPointer = columns.createPointer();
+ }
+
+ @Override
+ public void setPageZeroBuffer(ByteBuffer pageZero, int numberOfColumns, int numberOfPrimaryKeys) {
+ this.pageZero = pageZero;
+ int offset = pageZero.position();
+
+ columnsOffset = offset;
+ offset += numberOfColumns * Integer.BYTES;
+
+ filtersOffset = offset;
+ offset += numberOfColumns * FILTER_SIZE;
+
+ pageZero.position(offset);
+ primaryKeysOffset = offset;
+ primaryKeys.reset(pageZero);
+ nonKeyColumnStartOffset = pageZero.capacity();
+ }
+
+ @Override
+ public int writePrimaryKeyColumns(IColumnValuesWriter[] primaryKeyWriters) throws HyracksDataException {
+ int allocatedSpace = 0;
+ for (int i = 0; i < primaryKeyWriters.length; i++) {
+ IColumnValuesWriter writer = primaryKeyWriters[i];
+ setColumnOffset(i, primaryKeysOffset + primaryKeys.size());
+ writer.flush(primaryKeys);
+ allocatedSpace += writer.getAllocatedSpace();
+ }
+ return allocatedSpace;
+ }
+
+ @Override
+ public int writeColumns(PriorityQueue<IColumnValuesWriter> nonKeysColumnWriters) throws HyracksDataException {
+ int allocatedSpace = 0;
+ columns.reset();
+ while (!nonKeysColumnWriters.isEmpty()) {
+ IColumnValuesWriter writer = nonKeysColumnWriters.poll();
+ writeColumn(writer);
+ allocatedSpace += writer.getAllocatedSpace();
+ }
+ return allocatedSpace;
+ }
+
+ private void writeColumn(IColumnValuesWriter writer) throws HyracksDataException {
+ if (!hasEnoughSpace(columns.getCurrentBufferPosition(), writer)) {
+ /*
+ * We reset the columns stream to write all pages and confiscate a new buffer to minimize splitting
+ * the columns value into multiple pages.
+ */
+ nonKeyColumnStartOffset += columns.capacity();
+ columns.reset();
+ }
+
+ int columnRelativeOffset = columns.size();
+ columns.reserveInteger(columnLengthPointer);
+ setColumnOffset(writer.getColumnIndex(), nonKeyColumnStartOffset + columnRelativeOffset);
+
+ writeFilter(writer);
+ writer.flush(columns);
+
+ int length = columns.size() - columnRelativeOffset;
+ columnLengthPointer.setInteger(length);
+ }
+
+ private boolean hasEnoughSpace(int bufferPosition, IColumnValuesWriter columnWriter) {
+ //Estimated size mostly overestimate the size
+ int columnSize = columnWriter.getEstimatedSize();
+ float remainingPercentage = (pageSize - bufferPosition) / (float) pageSize;
+ if (columnSize > pageSize) {
+ /*
+ * If the column size is larger than the page size, we check whether the remaining space is less than
+ * the tolerance percentage
+ * - true --> allocate new buffer and tolerate empty space
+ * - false --> we split the column into two pages
+ */
+ return remainingPercentage >= tolerance;
+ }
+
+ int freeSpace = pageSize - (bufferPosition + columnSize);
+
+ /*
+ * Check if the free space is enough to fit the column or the free space is less that the tolerance percentage
+ * - true --> we allocate new buffer and tolerate empty space
+ * - false --> we split the column into two pages
+ */
+ return freeSpace > columnSize || remainingPercentage >= tolerance;
+ }
+
+ private void setColumnOffset(int columnIndex, int offset) {
+ pageZero.putInt(columnsOffset + Integer.BYTES * columnIndex, offset);
+ }
+
+ private void writeFilter(IColumnValuesWriter writer) {
+ int offset = filtersOffset + writer.getColumnIndex() * FILTER_SIZE;
+ pageZero.putLong(offset, writer.getNormalizedMinValue());
+ pageZero.putLong(offset + Long.BYTES, writer.getNormalizedMaxValue());
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/writer/ColumnValuesWriterFactory.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/writer/ColumnValuesWriterFactory.java
new file mode 100644
index 0000000..6a514ff
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/writer/ColumnValuesWriterFactory.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.column.values.writer;
+
+import org.apache.asterix.column.values.IColumnValuesWriter;
+import org.apache.asterix.column.values.IColumnValuesWriterFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnWriteMultiPageOp;
+
+public class ColumnValuesWriterFactory implements IColumnValuesWriterFactory {
+ private final Mutable<IColumnWriteMultiPageOp> multiPageOpRef;
+
+ public ColumnValuesWriterFactory(Mutable<IColumnWriteMultiPageOp> multiPageOpRef) {
+ this.multiPageOpRef = multiPageOpRef;
+ }
+
+ @Override
+ public IColumnValuesWriter createValueWriter(ATypeTag typeTag, int columnIndex, int maxLevel, boolean writeAlways,
+ boolean filtered) {
+ switch (typeTag) {
+ case MISSING:
+ case NULL:
+ return new NullMissingColumnValuesWriter(columnIndex, maxLevel, writeAlways, filtered);
+ case BOOLEAN:
+ return new BooleanColumnValuesWriter(columnIndex, maxLevel, writeAlways, filtered);
+ case BIGINT:
+ return new LongColumnValuesWriter(multiPageOpRef, columnIndex, maxLevel, writeAlways, filtered);
+ case DOUBLE:
+ return new DoubleColumnValuesWriter(multiPageOpRef, columnIndex, maxLevel, writeAlways, filtered);
+ case STRING:
+ return new StringColumnValuesWriter(multiPageOpRef, columnIndex, maxLevel, writeAlways, filtered);
+ case UUID:
+ return new UUIDColumnValuesWriter(multiPageOpRef, columnIndex, maxLevel, writeAlways, filtered);
+ default:
+ throw new UnsupportedOperationException(typeTag + " is not supported");
+ }
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/writer/DoubleColumnValuesWriter.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/writer/DoubleColumnValuesWriter.java
new file mode 100644
index 0000000..ca5cbb1
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/writer/DoubleColumnValuesWriter.java
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.column.values.writer;
+
+import java.io.IOException;
+
+import org.apache.asterix.column.bytes.encoder.ParquetPlainValuesWriter;
+import org.apache.asterix.column.values.IColumnValuesReader;
+import org.apache.asterix.column.values.writer.filters.AbstractColumnFilterWriter;
+import org.apache.asterix.column.values.writer.filters.DoubleColumnFilterWriter;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.data.std.primitive.DoublePointable;
+import org.apache.hyracks.data.std.primitive.FloatPointable;
+import org.apache.hyracks.data.std.primitive.IntegerPointable;
+import org.apache.hyracks.data.std.primitive.LongPointable;
+import org.apache.hyracks.data.std.primitive.ShortPointable;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnWriteMultiPageOp;
+import org.apache.parquet.bytes.BytesInput;
+
+public final class DoubleColumnValuesWriter extends AbstractColumnValuesWriter {
+ private final ParquetPlainValuesWriter doubleWriter;
+
+ public DoubleColumnValuesWriter(Mutable<IColumnWriteMultiPageOp> multiPageOpRef, int columnIndex, int level,
+ boolean collection, boolean filtered) {
+ super(columnIndex, level, collection, filtered);
+ doubleWriter = new ParquetPlainValuesWriter(multiPageOpRef);
+ }
+
+ @Override
+ protected void addValue(ATypeTag tag, IValueReference value) throws IOException {
+ final double normalizedDouble = getValue(tag, value.getByteArray(), value.getStartOffset());
+ doubleWriter.writeDouble(normalizedDouble);
+ filterWriter.addDouble(normalizedDouble);
+ }
+
+ private double getValue(ATypeTag typeTag, byte[] byteArray, int offset) {
+ switch (typeTag) {
+ case TINYINT:
+ return byteArray[offset];
+ case SMALLINT:
+ return ShortPointable.getShort(byteArray, offset);
+ case INTEGER:
+ return IntegerPointable.getInteger(byteArray, offset);
+ case BIGINT:
+ return LongPointable.getLong(byteArray, offset);
+ case FLOAT:
+ return FloatPointable.getFloat(byteArray, offset);
+ case DOUBLE:
+ return DoublePointable.getDouble(byteArray, offset);
+ default:
+ throw new IllegalAccessError(typeTag + "is not of floating type");
+ }
+ }
+
+ @Override
+ protected void resetValues() throws HyracksDataException {
+ doubleWriter.reset();
+ }
+
+ @Override
+ protected BytesInput getBytes() throws IOException {
+ return doubleWriter.getBytes();
+ }
+
+ @Override
+ protected int getValuesEstimatedSize() {
+ return doubleWriter.getEstimatedSize();
+ }
+
+ @Override
+ protected int getValuesAllocatedSize() {
+ return doubleWriter.getAllocatedSize();
+ }
+
+ @Override
+ protected void addValue(IColumnValuesReader reader) throws IOException {
+ double value = reader.getDouble();
+ doubleWriter.writeDouble(value);
+ filterWriter.addDouble(value);
+ }
+
+ @Override
+ protected AbstractColumnFilterWriter createFilter() {
+ return new DoubleColumnFilterWriter();
+ }
+
+ @Override
+ protected void closeValues() {
+ doubleWriter.close();
+ }
+
+ @Override
+ protected ATypeTag getTypeTag() {
+ return ATypeTag.DOUBLE;
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/writer/LongColumnValuesWriter.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/writer/LongColumnValuesWriter.java
new file mode 100644
index 0000000..e71ec73
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/writer/LongColumnValuesWriter.java
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.column.values.writer;
+
+import java.io.IOException;
+
+import org.apache.asterix.column.bytes.encoder.ParquetDeltaBinaryPackingValuesWriterForLong;
+import org.apache.asterix.column.values.IColumnValuesReader;
+import org.apache.asterix.column.values.writer.filters.AbstractColumnFilterWriter;
+import org.apache.asterix.column.values.writer.filters.LongColumnFilterWriter;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.data.std.primitive.IntegerPointable;
+import org.apache.hyracks.data.std.primitive.LongPointable;
+import org.apache.hyracks.data.std.primitive.ShortPointable;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnWriteMultiPageOp;
+import org.apache.parquet.bytes.BytesInput;
+
+final class LongColumnValuesWriter extends AbstractColumnValuesWriter {
+ private final ParquetDeltaBinaryPackingValuesWriterForLong longWriter;
+
+ public LongColumnValuesWriter(Mutable<IColumnWriteMultiPageOp> multiPageOpRef, int columnIndex, int level,
+ boolean collection, boolean filtered) {
+ super(columnIndex, level, collection, filtered);
+ longWriter = new ParquetDeltaBinaryPackingValuesWriterForLong(multiPageOpRef);
+ }
+
+ @Override
+ protected void addValue(ATypeTag tag, IValueReference value) throws IOException {
+ final long normalizedInt = getValue(tag, value.getByteArray(), value.getStartOffset());
+ longWriter.writeLong(normalizedInt);
+ filterWriter.addLong(normalizedInt);
+ }
+
+ private long getValue(ATypeTag typeTag, byte[] byteArray, int offset) {
+ switch (typeTag) {
+ case TINYINT:
+ return byteArray[offset];
+ case SMALLINT:
+ return ShortPointable.getShort(byteArray, offset);
+ case INTEGER:
+ return IntegerPointable.getInteger(byteArray, offset);
+ case BIGINT:
+ return LongPointable.getLong(byteArray, offset);
+ default:
+ throw new IllegalAccessError(typeTag + "is not of type integer");
+ }
+ }
+
+ @Override
+ protected void resetValues() throws HyracksDataException {
+ longWriter.reset();
+ }
+
+ @Override
+ protected BytesInput getBytes() throws IOException {
+ return longWriter.getBytes();
+ }
+
+ @Override
+ protected int getValuesEstimatedSize() {
+ return longWriter.getEstimatedSize();
+ }
+
+ @Override
+ protected int getValuesAllocatedSize() {
+ return longWriter.getAllocatedSize();
+ }
+
+ @Override
+ protected void addValue(IColumnValuesReader reader) throws IOException {
+ long value = reader.getLong();
+ longWriter.writeLong(value);
+ filterWriter.addLong(value);
+ }
+
+ @Override
+ protected AbstractColumnFilterWriter createFilter() {
+ return new LongColumnFilterWriter();
+ }
+
+ @Override
+ protected void closeValues() {
+ longWriter.close();
+ }
+
+ @Override
+ protected ATypeTag getTypeTag() {
+ return ATypeTag.BIGINT;
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/writer/NullMissingColumnValuesWriter.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/writer/NullMissingColumnValuesWriter.java
new file mode 100644
index 0000000..edc9fe2
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/writer/NullMissingColumnValuesWriter.java
@@ -0,0 +1,102 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.column.values.writer;
+
+import java.io.IOException;
+
+import org.apache.asterix.column.util.RunLengthIntArray;
+import org.apache.asterix.column.values.IColumnValuesReader;
+import org.apache.asterix.column.values.writer.filters.AbstractColumnFilterWriter;
+import org.apache.asterix.column.values.writer.filters.NoOpColumnFilterWriter;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.parquet.bytes.BytesInput;
+
+public class NullMissingColumnValuesWriter extends AbstractColumnValuesWriter {
+ private static final BytesInput EMPTY = BytesInput.empty();
+ private final RunLengthIntArray defLevelsIntArray;
+
+ NullMissingColumnValuesWriter(int columnIndex, int level, boolean collection, boolean filtered) {
+ super(columnIndex, level, collection, filtered);
+ defLevelsIntArray = new RunLengthIntArray();
+ }
+
+ @Override
+ protected void addLevel(int level) throws HyracksDataException {
+ defLevelsIntArray.add(level);
+ super.addLevel(level);
+ }
+
+ @Override
+ public void writeLevels(int level, int count) throws HyracksDataException {
+ defLevelsIntArray.add(level, count);
+ super.writeLevels(level, count);
+ }
+
+ @Override
+ protected ATypeTag getTypeTag() {
+ return ATypeTag.NULL;
+ }
+
+ @Override
+ protected void addValue(ATypeTag tag, IValueReference value) throws IOException {
+ throw new IllegalStateException("Null writer should not add value");
+ }
+
+ @Override
+ protected void addValue(IColumnValuesReader reader) throws IOException {
+ throw new IllegalStateException("Null writer should not add value");
+ }
+
+ @Override
+ protected BytesInput getBytes() throws IOException {
+ return EMPTY;
+ }
+
+ @Override
+ protected int getValuesEstimatedSize() {
+ return 0;
+ }
+
+ @Override
+ protected int getValuesAllocatedSize() {
+ return 0;
+ }
+
+ @Override
+ protected AbstractColumnFilterWriter createFilter() {
+ return NoOpColumnFilterWriter.INSTANCE;
+ }
+
+ @Override
+ protected void resetValues() throws HyracksDataException {
+ defLevelsIntArray.reset();
+ }
+
+ @Override
+ protected void closeValues() {
+ defLevelsIntArray.reset();
+ }
+
+ @Override
+ public RunLengthIntArray getDefinitionLevelsIntArray() {
+ return defLevelsIntArray;
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/writer/StringColumnValuesWriter.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/writer/StringColumnValuesWriter.java
new file mode 100644
index 0000000..e1a3ffd
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/writer/StringColumnValuesWriter.java
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.column.values.writer;
+
+import java.io.IOException;
+
+import org.apache.asterix.column.bytes.encoder.ParquetDeltaByteArrayWriter;
+import org.apache.asterix.column.values.IColumnValuesReader;
+import org.apache.asterix.column.values.writer.filters.AbstractColumnFilterWriter;
+import org.apache.asterix.column.values.writer.filters.StringColumnFilterWriter;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnWriteMultiPageOp;
+import org.apache.parquet.bytes.BytesInput;
+
+public class StringColumnValuesWriter extends AbstractColumnValuesWriter {
+ private final ParquetDeltaByteArrayWriter stringWriter;
+ private final boolean skipLengthBytes;
+
+ public StringColumnValuesWriter(Mutable<IColumnWriteMultiPageOp> multiPageOpRef, int columnIndex, int level,
+ boolean collection, boolean filtered) {
+ this(multiPageOpRef, columnIndex, level, collection, filtered, true);
+ }
+
+ protected StringColumnValuesWriter(Mutable<IColumnWriteMultiPageOp> multiPageOpRef, int columnIndex, int level,
+ boolean collection, boolean filtered, boolean skipLengthBytes) {
+ super(columnIndex, level, collection, filtered);
+ stringWriter = new ParquetDeltaByteArrayWriter(multiPageOpRef);
+ this.skipLengthBytes = skipLengthBytes;
+ }
+
+ @Override
+ protected final void addValue(ATypeTag tag, IValueReference value) throws IOException {
+ stringWriter.writeBytes(value, skipLengthBytes);
+ filterWriter.addValue(value);
+ }
+
+ @Override
+ protected final void resetValues() throws HyracksDataException {
+ stringWriter.reset();
+ }
+
+ @Override
+ protected final BytesInput getBytes() throws IOException {
+ return stringWriter.getBytes();
+ }
+
+ @Override
+ protected final int getValuesEstimatedSize() {
+ return stringWriter.getEstimatedSize();
+ }
+
+ @Override
+ protected final int getValuesAllocatedSize() {
+ return stringWriter.getAllocatedSize();
+ }
+
+ @Override
+ protected final void addValue(IColumnValuesReader reader) throws IOException {
+ IValueReference value = reader.getBytes();
+ stringWriter.writeBytes(value, skipLengthBytes);
+ filterWriter.addValue(value);
+ }
+
+ @Override
+ protected AbstractColumnFilterWriter createFilter() {
+ return new StringColumnFilterWriter();
+ }
+
+ @Override
+ protected final void closeValues() {
+ stringWriter.close();
+ }
+
+ @Override
+ protected ATypeTag getTypeTag() {
+ return ATypeTag.STRING;
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/writer/UUIDColumnValuesWriter.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/writer/UUIDColumnValuesWriter.java
new file mode 100644
index 0000000..1e98754
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/writer/UUIDColumnValuesWriter.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.asterix.column.values.writer;
+
+import org.apache.asterix.column.values.writer.filters.AbstractColumnFilterWriter;
+import org.apache.asterix.column.values.writer.filters.UUIDColumnFilterWriter;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnWriteMultiPageOp;
+
+final class UUIDColumnValuesWriter extends StringColumnValuesWriter {
+
+ public UUIDColumnValuesWriter(Mutable<IColumnWriteMultiPageOp> multiPageOpRef, int columnIndex, int level,
+ boolean collection, boolean filtered) {
+ super(multiPageOpRef, columnIndex, level, collection, filtered, false);
+ }
+
+ @Override
+ protected AbstractColumnFilterWriter createFilter() {
+ return new UUIDColumnFilterWriter();
+ }
+
+ @Override
+ protected ATypeTag getTypeTag() {
+ return ATypeTag.UUID;
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/writer/filters/AbstractColumnFilterWriter.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/writer/filters/AbstractColumnFilterWriter.java
new file mode 100644
index 0000000..abbe314
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/writer/filters/AbstractColumnFilterWriter.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.column.values.writer.filters;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+
+public abstract class AbstractColumnFilterWriter {
+ public static final int FILTER_SIZE = Long.BYTES * 2;
+
+ public void addLong(long value) {
+ throw new UnsupportedOperationException(getClass().getName());
+ }
+
+ public void addDouble(double value) {
+ throw new UnsupportedOperationException(getClass().getName());
+ }
+
+ public void addValue(IValueReference value) throws HyracksDataException {
+ throw new UnsupportedOperationException(getClass().getName());
+ }
+
+ public abstract long getMinNormalizedValue();
+
+ public abstract long getMaxNormalizedValue();
+
+ public abstract void reset();
+
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/writer/filters/DoubleColumnFilterWriter.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/writer/filters/DoubleColumnFilterWriter.java
new file mode 100644
index 0000000..6fccabe
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/writer/filters/DoubleColumnFilterWriter.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.column.values.writer.filters;
+
+public class DoubleColumnFilterWriter extends AbstractColumnFilterWriter {
+ private double min;
+ private double max;
+
+ public DoubleColumnFilterWriter() {
+ reset();
+ }
+
+ @Override
+ public void addDouble(double value) {
+ min = Math.min(min, value);
+ max = Math.max(max, value);
+ }
+
+ @Override
+ public long getMinNormalizedValue() {
+ return Double.doubleToLongBits(min);
+ }
+
+ @Override
+ public long getMaxNormalizedValue() {
+ return Double.doubleToLongBits(max);
+ }
+
+ @Override
+ public void reset() {
+ min = Double.MIN_VALUE;
+ max = Double.MAX_VALUE;
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/writer/filters/LongColumnFilterWriter.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/writer/filters/LongColumnFilterWriter.java
new file mode 100644
index 0000000..25cb94c
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/writer/filters/LongColumnFilterWriter.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.column.values.writer.filters;
+
+public class LongColumnFilterWriter extends AbstractColumnFilterWriter {
+ private long min;
+ private long max;
+
+ public LongColumnFilterWriter() {
+ reset();
+ }
+
+ @Override
+ public void addLong(long value) {
+ min = Math.min(min, value);
+ max = Math.max(max, value);
+ }
+
+ @Override
+ public long getMinNormalizedValue() {
+ return min;
+ }
+
+ @Override
+ public long getMaxNormalizedValue() {
+ return max;
+ }
+
+ @Override
+ public void reset() {
+ min = Long.MAX_VALUE;
+ max = Long.MIN_VALUE;
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/writer/filters/NoOpColumnFilterWriter.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/writer/filters/NoOpColumnFilterWriter.java
new file mode 100644
index 0000000..c4f6f6f
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/writer/filters/NoOpColumnFilterWriter.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.column.values.writer.filters;
+
+import org.apache.hyracks.data.std.api.IValueReference;
+
+public class NoOpColumnFilterWriter extends AbstractColumnFilterWriter {
+ public static final AbstractColumnFilterWriter INSTANCE = new NoOpColumnFilterWriter();
+
+ private NoOpColumnFilterWriter() {
+ }
+
+ @Override
+ public void addLong(long value) {
+ //NoOp
+ }
+
+ @Override
+ public void addDouble(double value) {
+ //NoOp
+ }
+
+ @Override
+ public void addValue(IValueReference value) {
+ //NoOp
+ }
+
+ @Override
+ public long getMinNormalizedValue() {
+ return 0;
+ }
+
+ @Override
+ public long getMaxNormalizedValue() {
+ return 0;
+ }
+
+ @Override
+ public void reset() {
+ //NoOp
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/writer/filters/StringColumnFilterWriter.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/writer/filters/StringColumnFilterWriter.java
new file mode 100644
index 0000000..77d82c0
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/writer/filters/StringColumnFilterWriter.java
@@ -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.
+ */
+package org.apache.asterix.column.values.writer.filters;
+
+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.getNumBytesToStoreLength;
+import static org.apache.hyracks.util.string.UTF8StringUtil.getUTFLength;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+
+public class StringColumnFilterWriter extends LongColumnFilterWriter {
+ @Override
+ public void addValue(IValueReference value) throws HyracksDataException {
+ addLong(normalize(value));
+ }
+
+ /**
+ * Normalizes the string in a {@link Long}
+ *
+ * @see org.apache.hyracks.util.string.UTF8StringUtil#normalize(byte[], int)
+ */
+ public static long normalize(IValueReference value) {
+ byte[] bytes = value.getByteArray();
+ int start = value.getStartOffset();
+
+ long nk = 0;
+ int offset = start + getNumBytesToStoreLength(getUTFLength(bytes, start));
+ int end = start + value.getLength();
+ for (int i = 0; i < 4; ++i) {
+ nk <<= 16;
+ if (offset < end) {
+ nk += (charAt(bytes, offset)) & 0xffff;
+ offset += charSize(bytes, offset);
+ }
+ }
+ return nk >>> 1;
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/writer/filters/UUIDColumnFilterWriter.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/writer/filters/UUIDColumnFilterWriter.java
new file mode 100644
index 0000000..5e2bc61
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/writer/filters/UUIDColumnFilterWriter.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.column.values.writer.filters;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.data.std.primitive.LongPointable;
+
+/**
+ * UUID filters are the LSB component of all written UUIDs. This could provide false positives UUIDs; however, this
+ * still can filter out non-matching UUIDs.
+ */
+public class UUIDColumnFilterWriter extends LongColumnFilterWriter {
+
+ @Override
+ public void addValue(IValueReference value) throws HyracksDataException {
+ addLong(getLSB(value));
+ }
+
+ public static long getLSB(IValueReference value) {
+ byte[] bytes = value.getByteArray();
+ int start = value.getStartOffset();
+ return LongPointable.getLong(bytes, start + Long.BYTES);
+ }
+}
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/config/CompilerProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
index 5538a0a..9296339 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,21 @@
COMPILER_ARRAYINDEX(
BOOLEAN,
AlgebricksConfig.ARRAY_INDEX_DEFAULT,
- "Enable/disable using array-indexes in queries");
+ "Enable/disable using array-indexes in queries"),
+ COMPILER_BATCH_LOOKUP(
+ BOOLEAN,
+ AlgebricksConfig.BATCH_LOOKUP_DEFAULT,
+ "Enable/disable batch point-lookups when running queries with secondary indexes"),
+ 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 +152,7 @@
@Override
public boolean hidden() {
- return this == COMPILER_EXTERNALSCANMEMORY;
+ return this == COMPILER_EXTERNALSCANMEMORY || this == COMPILER_CBOTEST;
}
}
@@ -173,6 +188,16 @@
public static final String COMPILER_EXTERNALSCANMEMORY_KEY = Option.COMPILER_EXTERNALSCANMEMORY.ini();
+ public static final String COMPILER_BATCHED_LOOKUP_KEY = Option.COMPILER_BATCH_LOOKUP.ini();
+
+ public static final 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 +271,29 @@
public int getExternalScanMemorySize() {
return accessor.getInt(Option.COMPILER_EXTERNALSCANMEMORY);
}
+
+ public boolean isBatchLookup() {
+ return accessor.getBoolean(Option.COMPILER_BATCH_LOOKUP);
+ }
+
+ 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;
+ }
}
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..e1edc1c 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/OptimizationConfUtil.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/OptimizationConfUtil.java
@@ -77,6 +77,16 @@
int externalScanBufferSize = getExternalScanBufferSize(
(String) querySpecificConfig.get(CompilerProperties.COMPILER_EXTERNALSCANMEMORY_KEY),
compilerProperties.getExternalScanMemorySize(), sourceLoc);
+ boolean batchLookup = getBoolean(querySpecificConfig, CompilerProperties.COMPILER_BATCHED_LOOKUP_KEY,
+ compilerProperties.isBatchLookup());
+ 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 +105,11 @@
physOptConf.setMinMemoryAllocation(minMemoryAllocation);
physOptConf.setArrayIndexEnabled(arrayIndex);
physOptConf.setExternalScanBufferSize(externalScanBufferSize);
+ physOptConf.setBatchLookup(batchLookup);
+ physOptConf.setCBOMode(cbo);
+ physOptConf.setCBOTestMode(cboTest);
+ physOptConf.setForceJoinOrderMode(forceJoinOrder);
+ physOptConf.setQueryPlanShapeMode(queryPlanShape);
return physOptConf;
}
@@ -117,6 +132,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 +187,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/dataflow/DatasetLocalResource.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/DatasetLocalResource.java
index 249505b..58d5f9b 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/DatasetLocalResource.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/DatasetLocalResource.java
@@ -72,6 +72,10 @@
this.partition = partition;
}
+ public IResource getResource() {
+ return resource;
+ }
+
@Override
public IIndex createInstance(INCServiceContext ncServiceCtx) throws HyracksDataException {
return resource.createInstance(ncServiceCtx);
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
index 068c125..44471a2 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
@@ -85,6 +85,8 @@
PARQUET_DECIMAL_TO_DOUBLE_PRECISION_LOSS(55),
PARQUET_TIME_ZONE_ID_IS_NOT_SET(56),
PARQUET_CONTAINS_OVERFLOWED_BIGINT(57),
+ UNEXPECTED_ERROR_ENCOUNTERED(58),
+ INVALID_PARQUET_FILE(59),
UNSUPPORTED_JRE(100),
@@ -263,6 +265,8 @@
S3_REGION_NOT_SUPPORTED(1170),
COMPILATION_SET_OPERATION_ERROR(1171),
INVALID_TIMEZONE(1172),
+ INVALID_PARAM_VALUE_ALLOWED_VALUE(1173),
+ SAMPLE_HAS_ZERO_ROWS(1174),
// 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/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/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 f5ef79d..c9ab080 100644
--- a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
+++ b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
@@ -92,6 +92,8 @@
55 = Parquet decimal precision loss: precision '%1$s' is greater than the maximum supported precision '%2$s'
56 = Parquet file(s) contain values of the temporal type '%1$s' that are adjusted to UTC. Recreate the external dataset and set the option '%2$s' to get the local-adjusted '%1$s' value
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
100 = Unsupported JRE: %1$s
@@ -265,6 +267,8 @@
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
# Feed Errors
3001 = Illegal state.
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/input.component.scss b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/input.component.scss
index 94311f8..68c12f4 100755
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/input.component.scss
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/input.component.scss
@@ -46,6 +46,7 @@
.options {
display: flex;
flex-flow: row;
+ overflow: auto;
}
.codemirror-container {
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/input.component.ts b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/input.component.ts
index f885edb..cb46a6a 100755
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/input.component.ts
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/input.component.ts
@@ -538,7 +538,7 @@
this.queryString = '';
this.selected = 'Default';
} else {
- this.queryString = 'USE ' + this.selected + '; \n';
+ this.queryString = 'USE ' + this.selected + '; \n' + this.queryString;
}
this.editor.getDoc().setValue(this.queryString);
this.editor.execCommand('goDocEnd')
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-viewer.component.html b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-viewer.component.html
index a131353..4b6ecc5 100644
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-viewer.component.html
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-viewer.component.html
@@ -97,7 +97,7 @@
[links]="edgesArr"
[nodes]="nodesArr"
[draggingEnabled]="false"
- [zoomSpeed]="0.025"
+ [zoomSpeed]="0.015"
[update$]="update$"
[layoutSettings]="{
orientation: planOrientation,
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-viewer.component.scss b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-viewer.component.scss
index 6c40a68..9601675 100644
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-viewer.component.scss
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-viewer.component.scss
@@ -36,7 +36,6 @@
}
.panel {
- order: 2;
display: flex;
flex-flow: column;
justify-content: stretch;
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/tree-view.component.html b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/tree-view.component.html
index 0fbf6f6..c5754762 100644
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/tree-view.component.html
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/tree-view.component.html
@@ -30,7 +30,8 @@
<button *ngIf="isCSV" mat-button class='button export' (click)='openJSONExportPicker()' matTooltip="Export CSV file to Computer">EXPORT</button>
<button *ngIf="isCSV == false" mat-button class='button export' (click)='openJSONExportPicker()' matTooltip="Export JSON/JSONL file to Computer">EXPORT</button>
</span>
- <mat-paginator *ngIf="this.planVisible === false" [showFirstLastButtons]="true" [length]='metrics.resultCount' [pageSize]='pagedefaults.pageSize' [pageSizeOptions]='pageSizeOptions' (page)='showResults($event, false)'>
+
+ <mat-paginator *ngIf="this.planVisible === false" [showFirstLastButtons]="true" [length]='metrics.resultCount' [pageSize]='pageSize' [pageSizeOptions]='pageSizeOptions' (page)='showResults($event, false)'>
</mat-paginator>
</div>
<div *ngIf='treeVisible' class='navi-data' class='navi-data'>
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/tree-view.component.ts b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/tree-view.component.ts
index 5309991..66a6dd6 100644
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/tree-view.component.ts
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/tree-view.component.ts
@@ -60,7 +60,8 @@
currentRange: any;
/* see 10 records as initial set */
pagedefaults: any = { pageIndex: 0, pageSize:10, lenght: 0};
- pageSizeOptions = [5, 10, 25, 100, 200];
+ pageSize = 10;
+ pageSizeOptions = [5, 10, 25, 100, 200, 300, 400];
viewMode = 'JSON';
showGoTop = false;
showGoBottom = false;
@@ -141,6 +142,9 @@
}
showResults(range, expanded) {
+ // update pagesize
+ this.pageSize = range.pageSize;
+
this.currentRange = range;
this.currentIndex = this.currentRange.pageIndex;
diff --git a/asterixdb/asterix-doc/src/main/grammar/sqlpp.ebnf b/asterixdb/asterix-doc/src/main/grammar/sqlpp.ebnf
index b237f30..af67e33 100644
--- a/asterixdb/asterix-doc/src/main/grammar/sqlpp.ebnf
+++ b/asterixdb/asterix-doc/src/main/grammar/sqlpp.ebnf
@@ -59,7 +59,7 @@
StreamGenerator::= FromClause LetClause? WhereClause? (GroupByClause LetClause? HavingClause?)?
-SelectClause ::= "SELECT" ("DISTINCT" | "ALL")? ( "VALUE" Expr | Projection ("," Projection)*)
+SelectClause ::= "SELECT" ("DISTINCT" | "ALL")? ( "VALUE" Expr | Projection ("," Projection)* ( "EXCLUDE" Identifier (("," | ".") Identifier)* )? )
Projection ::= (Expr ("AS"? Identifier)?) | (VariableRef "." "*") | "*"
@@ -162,7 +162,7 @@
CreateType ::= "CREATE" "TYPE" QualifiedName ("IF" "NOT" "EXISTS")? "AS" ObjectTypeDef
-ObjectTypeDef ::= ("CLOSED" | "OPEN")? "{" ObjectField ("," ObjectField)* "}"
+ObjectTypeDef ::= ("CLOSED" | "OPEN")? "{" ( ObjectField ("," ObjectField)* )? "}"
ObjectField ::= Identifier ":" Identifier "?"?
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..b8dc3bf 100644
--- a/asterixdb/asterix-doc/src/main/markdown/sqlpp/3_query.md
+++ b/asterixdb/asterix-doc/src/main/markdown/sqlpp/3_query.md
@@ -248,7 +248,7 @@
##### Example
-(Q3.5) Returns all of the different cities in the `customers` dataset.
+(Q3.5a) Returns all of the different cities in the `customers` dataset.
FROM customers AS c
SELECT DISTINCT c.address.city;
@@ -270,6 +270,32 @@
}
]
+### <a id="Select_exclude">SELECT EXCLUDE</a>
+The `EXCLUDE` keyword is used to remove one or more fields that would otherwise be returned from the `SELECT` clause.
+Conceptually, the scope of the `EXCLUDE` clause is the output of the `SELECT` clause itself.
+In a Stream Generator with both `DISTINCT` and `EXCLUDE` clauses, the `DISTINCT` clause is applied after the `EXCLUDE` clause.
+
+##### Example
+
+(Q3.5b) For the customer with `custid = C13`, return their information _excluding_ the `zipcode` field inside the `address` object and the top-level `name` field.
+
+ FROM customers AS c
+ WHERE c.custid = "C13"
+ SELECT c.* EXCLUDE address.zipcode, name;
+
+Result:
+
+ [
+ {
+ "custid": "C13",
+ "address": {
+ "street": "201 Main St.",
+ "city": "St. Louis, MO"
+ },
+ "rating": 750
+ }
+ ]
+
### <a id="Unnamed_projections">Unnamed Projections</a>
Similar to standard SQL, the query language supports unnamed projections (a.k.a, unnamed `SELECT` clause items), for which names are generated rather than user-provided.
diff --git a/asterixdb/asterix-docker/pom.xml b/asterixdb/asterix-docker/pom.xml
deleted file mode 100644
index ed47c29..0000000
--- a/asterixdb/asterix-docker/pom.xml
+++ /dev/null
@@ -1,68 +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.
- !-->
-
-<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/xsd/maven-4.0.0.xsd">
- <modelVersion>4.0.0</modelVersion>
- <parent>
- <artifactId>apache-asterixdb</artifactId>
- <groupId>org.apache.asterix</groupId>
- <version>0.9.9-SNAPSHOT</version>
- </parent>
- <artifactId>asterix-docker</artifactId>
-
- <properties>
- <root.dir>${basedir}/..</root.dir>
- </properties>
-
- <licenses>
- <license>
- <name>Apache License, Version 2.0</name>
- <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
- <distribution>repo</distribution>
- <comments>A business-friendly OSS license</comments>
- </license>
- </licenses>
-
- <profiles>
- <profile>
- <id>docker</id>
- <build>
- <plugins>
- <plugin>
- <groupId>com.spotify</groupId>
- <artifactId>docker-maven-plugin</artifactId>
- <version>0.2.11</version>
- <configuration>
- <imageName>asterixdb/demo</imageName>
- <dockerDirectory>docker</dockerDirectory>
- <resources>
- <resource>
- <targetPath>/</targetPath>
- <directory>../asterix-server/target/</directory>
- <include>asterix-server-${project.version}-binary-assembly.zip</include>
- </resource>
- </resources>
- </configuration>
- </plugin>
- </plugins>
- </build>
- </profile>
- </profiles>
-
-</project>
diff --git a/asterixdb/asterix-external-data/pom.xml b/asterixdb/asterix-external-data/pom.xml
index 0310700..14a3487 100644
--- a/asterixdb/asterix-external-data/pom.xml
+++ b/asterixdb/asterix-external-data/pom.xml
@@ -272,7 +272,6 @@
<dependency>
<groupId>com.e-movimento.tinytools</groupId>
<artifactId>privilegedaccessor</artifactId>
- <version>1.2.2</version>
<scope>test</scope>
</dependency>
<dependency>
@@ -518,6 +517,10 @@
</exclusion>
</exclusions>
</dependency>
+ <dependency>
+ <groupId>com.google.cloud.bigdataoss</groupId>
+ <artifactId>gcs-connector</artifactId>
+ </dependency>
<!-- apply patch for HADOOP-17225 to workaround CVE-2019-10172 -->
<dependency>
<groupId>org.codehaus.jackson</groupId>
@@ -542,6 +545,16 @@
<groupId>io.netty</groupId>
<artifactId>netty-transport-native-unix-common</artifactId>
</dependency>
+ <!-- TODO(htowaileb): removed from hadoop transitively and added separately to avoid CVEs, can
+ be removed once upgraded to hadoop 3.3.4 as it addresses the CVEs -->
+ <dependency>
+ <groupId>org.eclipse.jetty</groupId>
+ <artifactId>jetty-util</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.eclipse.jetty</groupId>
+ <artifactId>jetty-util-ajax</artifactId>
+ </dependency>
</dependencies>
<!-- apply patch for HADOOP-17225 to workaround CVE-2019-10172 -->
<repositories>
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IExternalLangIPCProto.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IExternalLangIPCProto.java
new file mode 100644
index 0000000..35e5961
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IExternalLangIPCProto.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.api;
+
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.external.ipc.MessageType;
+import org.apache.asterix.external.library.msgpack.MsgPackPointableVisitor;
+import org.apache.asterix.om.pointables.AFlatValuePointable;
+import org.apache.asterix.om.pointables.AListVisitablePointable;
+import org.apache.asterix.om.pointables.ARecordVisitablePointable;
+import org.apache.asterix.om.pointables.PointableAllocator;
+import org.apache.asterix.om.pointables.base.IVisitablePointable;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.types.TypeTagUtil;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+
+public interface IExternalLangIPCProto {
+ static void visitValueRef(IAType type, DataOutput out, IValueReference valueReference,
+ PointableAllocator pointableAllocator, MsgPackPointableVisitor pointableVisitor, boolean visitNull)
+ throws IOException {
+ IVisitablePointable pointable;
+ switch (type.getTypeTag()) {
+ case OBJECT:
+ pointable = pointableAllocator.allocateRecordValue(type);
+ pointable.set(valueReference);
+ pointableVisitor.visit((ARecordVisitablePointable) pointable, pointableVisitor.getTypeInfo(type, out));
+ break;
+ case ARRAY:
+ case MULTISET:
+ pointable = pointableAllocator.allocateListValue(type);
+ pointable.set(valueReference);
+ pointableVisitor.visit((AListVisitablePointable) pointable, pointableVisitor.getTypeInfo(type, out));
+ break;
+ case ANY:
+ ATypeTag rtTypeTag = EnumDeserializer.ATYPETAGDESERIALIZER
+ .deserialize(valueReference.getByteArray()[valueReference.getStartOffset()]);
+ IAType rtType = TypeTagUtil.getBuiltinTypeByTag(rtTypeTag);
+ visitValueRef(rtType, out, valueReference, pointableAllocator, pointableVisitor, visitNull);
+ break;
+ case MISSING:
+ case NULL:
+ if (!visitNull) {
+ return;
+ }
+ default:
+ pointable = pointableAllocator.allocateFieldValue(type);
+ pointable.set(valueReference);
+ pointableVisitor.visit((AFlatValuePointable) pointable, pointableVisitor.getTypeInfo(type, out));
+ break;
+ }
+ }
+
+ void start();
+
+ void helo() throws IOException, AsterixException;
+
+ long init(String module, String clazz, String fn) throws IOException, AsterixException;
+
+ ByteBuffer call(long functionId, IAType[] argTypes, IValueReference[] argValues, boolean nullCall)
+ throws IOException, AsterixException;
+
+ ByteBuffer callMulti(long key, ArrayBackedValueStorage args, int numTuples) throws IOException, AsterixException;
+
+ //For future use with interpreter reuse between jobs.
+ void quit() throws HyracksDataException;
+
+ void receiveMsg() throws IOException, AsterixException;
+
+ void sendHeader(long key, int msgLen) throws IOException;
+
+ void sendMsg(ArrayBackedValueStorage content) throws IOException;
+
+ void sendMsg() throws IOException;
+
+ MessageType getResponseType();
+
+ long getRouteId();
+
+ DataOutputStream getSockOut();
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/ILibraryEvaluator.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/ILibraryEvaluator.java
new file mode 100644
index 0000000..8c6538b
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/ILibraryEvaluator.java
@@ -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.
+ */
+package org.apache.asterix.external.api;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.om.functions.IExternalFunctionInfo;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.api.resources.IDeallocatable;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+
+public interface ILibraryEvaluator extends IDeallocatable {
+
+ void start() throws IOException, AsterixException;
+
+ long initialize(IExternalFunctionInfo finfo) throws IOException, AsterixException;
+
+ ByteBuffer call(long id, IAType[] argTypes, IValueReference[] valueReferences, boolean nullCall) throws IOException;
+
+ ByteBuffer callMulti(long id, ArrayBackedValueStorage arguments, int numTuples) throws IOException;
+}
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 8ea9ed4..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;
@@ -25,7 +29,8 @@
import java.util.Map;
import org.apache.asterix.common.api.IApplicationContext;
-import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.external.api.AsterixInputStream;
import org.apache.asterix.external.api.IExternalIndexer;
import org.apache.asterix.external.api.IIndexibleExternalDataSource;
@@ -100,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();
@@ -119,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();
@@ -131,7 +136,19 @@
this.recordClass = char[].class;
}
} catch (IOException e) {
- throw new AsterixException(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/aws/AwsS3InputStream.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStream.java
index f14af53..bbcf9cd 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStream.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStream.java
@@ -18,7 +18,6 @@
*/
package org.apache.asterix.external.input.record.reader.aws;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3;
import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
import java.io.IOException;
@@ -32,7 +31,7 @@
import org.apache.asterix.common.exceptions.RuntimeDataException;
import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStream;
import org.apache.asterix.external.util.ExternalDataConstants;
-import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.asterix.external.util.aws.s3.S3Utils;
import org.apache.commons.lang3.StringUtils;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.util.CleanupUtils;
@@ -109,7 +108,7 @@
}
private boolean shouldRetry(String errorCode, int currentRetry) {
- return currentRetry < MAX_RETRIES && AwsS3.isRetryableError(errorCode);
+ return currentRetry < MAX_RETRIES && S3Utils.isRetryableError(errorCode);
}
@Override
@@ -134,7 +133,7 @@
private S3Client buildAwsS3Client(Map<String, String> configuration) throws HyracksDataException {
try {
- return ExternalDataUtils.AwsS3.buildAwsS3Client(configuration);
+ return S3Utils.buildAwsS3Client(configuration);
} catch (CompilationException ex) {
throw HyracksDataException.create(ex);
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStreamFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStreamFactory.java
index 89ea39e..a241354 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStreamFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStreamFactory.java
@@ -26,6 +26,7 @@
import org.apache.asterix.external.api.AsterixInputStream;
import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStreamFactory;
import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.asterix.external.util.aws.s3.S3Utils;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.api.application.IServiceContext;
import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -53,8 +54,7 @@
IncludeExcludeMatcher includeExcludeMatcher = ExternalDataUtils.getIncludeExcludeMatchers(configuration);
//Get a list of S3 objects
- List<S3Object> filesOnly =
- ExternalDataUtils.AwsS3.listS3Objects(configuration, includeExcludeMatcher, warningCollector);
+ List<S3Object> filesOnly = S3Utils.listS3Objects(configuration, includeExcludeMatcher, warningCollector);
// Distribute work load amongst the partitions
distributeWorkLoad(filesOnly, getPartitionsCount());
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/parquet/AwsS3ParquetReaderFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/parquet/AwsS3ParquetReaderFactory.java
index 803e657..92b7a95 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/parquet/AwsS3ParquetReaderFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/parquet/AwsS3ParquetReaderFactory.java
@@ -18,22 +18,32 @@
*/
package org.apache.asterix.external.input.record.reader.aws.parquet;
+import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
+
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
import org.apache.asterix.external.input.HDFSDataSourceFactory;
import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStreamFactory.IncludeExcludeMatcher;
import org.apache.asterix.external.util.ExternalDataConstants;
import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.asterix.external.util.aws.s3.S3Constants;
+import org.apache.asterix.external.util.aws.s3.S3Utils;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.api.application.IServiceContext;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.util.ExceptionUtils;
+import com.amazonaws.SdkBaseException;
+
+import software.amazon.awssdk.core.exception.SdkException;
import software.amazon.awssdk.services.s3.model.S3Object;
public class AwsS3ParquetReaderFactory extends HDFSDataSourceFactory {
@@ -50,10 +60,20 @@
putS3ConfToHadoopConf(configuration, path);
//Configure Hadoop S3 input splits
- JobConf conf = createHdfsConf(serviceCtx, configuration);
- int numberOfPartitions = getPartitionConstraint().getLocations().length;
- ExternalDataUtils.AwsS3.configureAwsS3HdfsJobConf(conf, configuration, numberOfPartitions);
- configureHdfsConf(conf, configuration);
+ try {
+ JobConf conf = createHdfsConf(serviceCtx, configuration);
+ int numberOfPartitions = getPartitionConstraint().getLocations().length;
+ S3Utils.configureAwsS3HdfsJobConf(conf, configuration, numberOfPartitions);
+ configureHdfsConf(conf, configuration);
+ } catch (SdkException | SdkBaseException ex) {
+ throw new RuntimeDataException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+ } catch (AlgebricksException ex) {
+ Throwable root = ExceptionUtils.getRootCause(ex);
+ if (root instanceof SdkException || root instanceof SdkBaseException) {
+ throw new RuntimeDataException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(root));
+ }
+ throw ex;
+ }
}
@Override
@@ -89,8 +109,7 @@
throws CompilationException {
String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
IncludeExcludeMatcher includeExcludeMatcher = ExternalDataUtils.getIncludeExcludeMatchers(configuration);
- List<S3Object> filesOnly =
- ExternalDataUtils.AwsS3.listS3Objects(configuration, includeExcludeMatcher, warningCollector);
+ List<S3Object> filesOnly = S3Utils.listS3Objects(configuration, includeExcludeMatcher, warningCollector);
StringBuilder builder = new StringBuilder();
if (!filesOnly.isEmpty()) {
@@ -105,7 +124,7 @@
}
private static void appendFileURI(StringBuilder builder, String container, S3Object file) {
- builder.append(ExternalDataConstants.AwsS3.HADOOP_S3_PROTOCOL);
+ builder.append(S3Constants.HADOOP_S3_PROTOCOL);
builder.append("://");
builder.append(container);
builder.append('/');
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/blob/AzureBlobInputStream.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/blob/AzureBlobInputStream.java
index cdb3834..bbfece2 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/blob/AzureBlobInputStream.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/blob/AzureBlobInputStream.java
@@ -18,6 +18,7 @@
*/
package org.apache.asterix.external.input.record.reader.azure.blob;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.buildAzureBlobClient;
import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
import java.io.IOException;
@@ -31,7 +32,6 @@
import org.apache.asterix.common.exceptions.RuntimeDataException;
import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStream;
import org.apache.asterix.external.util.ExternalDataConstants;
-import org.apache.asterix.external.util.ExternalDataUtils;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.util.LogRedactionUtil;
@@ -86,7 +86,7 @@
private BlobServiceClient buildAzureClient(IApplicationContext appCtx, Map<String, String> configuration)
throws HyracksDataException {
try {
- return ExternalDataUtils.Azure.buildAzureBlobClient(appCtx, configuration);
+ return buildAzureBlobClient(appCtx, configuration);
} catch (CompilationException ex) {
throw HyracksDataException.create(ex);
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/blob/AzureBlobInputStreamFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/blob/AzureBlobInputStreamFactory.java
index 064b319..55c0521 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/blob/AzureBlobInputStreamFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/blob/AzureBlobInputStreamFactory.java
@@ -18,6 +18,9 @@
*/
package org.apache.asterix.external.input.record.reader.azure.blob;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.buildAzureBlobClient;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.listBlobItems;
+
import java.util.Comparator;
import java.util.List;
import java.util.Map;
@@ -57,9 +60,9 @@
// Ensure the validity of include/exclude
ExternalDataUtils.validateIncludeExclude(configuration);
IncludeExcludeMatcher includeExcludeMatcher = ExternalDataUtils.getIncludeExcludeMatchers(configuration);
- BlobServiceClient blobServiceClient = ExternalDataUtils.Azure.buildAzureBlobClient(appCtx, configuration);
- List<BlobItem> filesOnly = ExternalDataUtils.Azure.listBlobItems(blobServiceClient, configuration,
- includeExcludeMatcher, warningCollector);
+ BlobServiceClient blobServiceClient = buildAzureBlobClient(appCtx, configuration);
+ List<BlobItem> filesOnly =
+ listBlobItems(blobServiceClient, configuration, includeExcludeMatcher, warningCollector);
// Distribute work load amongst the partitions
distributeWorkLoad(filesOnly, getPartitionsCount());
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/datalake/AzureDataLakeInputStream.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/datalake/AzureDataLakeInputStream.java
index e34d188..7a95222 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/datalake/AzureDataLakeInputStream.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/datalake/AzureDataLakeInputStream.java
@@ -18,6 +18,7 @@
*/
package org.apache.asterix.external.input.record.reader.azure.datalake;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.buildAzureDatalakeClient;
import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
import java.io.IOException;
@@ -31,7 +32,6 @@
import org.apache.asterix.common.exceptions.RuntimeDataException;
import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStream;
import org.apache.asterix.external.util.ExternalDataConstants;
-import org.apache.asterix.external.util.ExternalDataUtils;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.util.LogRedactionUtil;
@@ -86,7 +86,7 @@
private DataLakeServiceClient buildAzureClient(IApplicationContext appCtx, Map<String, String> configuration)
throws HyracksDataException {
try {
- return ExternalDataUtils.Azure.buildAzureDatalakeClient(appCtx, configuration);
+ return buildAzureDatalakeClient(appCtx, configuration);
} catch (CompilationException ex) {
throw HyracksDataException.create(ex);
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/datalake/AzureDataLakeInputStreamFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/datalake/AzureDataLakeInputStreamFactory.java
index e9f8d4c..929cb6e 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/datalake/AzureDataLakeInputStreamFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/datalake/AzureDataLakeInputStreamFactory.java
@@ -18,6 +18,9 @@
*/
package org.apache.asterix.external.input.record.reader.azure.datalake;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.buildAzureDatalakeClient;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.listDatalakePathItems;
+
import java.util.Comparator;
import java.util.List;
import java.util.Map;
@@ -57,9 +60,9 @@
// Ensure the validity of include/exclude
ExternalDataUtils.validateIncludeExclude(configuration);
IncludeExcludeMatcher includeExcludeMatcher = ExternalDataUtils.getIncludeExcludeMatchers(configuration);
- DataLakeServiceClient client = ExternalDataUtils.Azure.buildAzureDatalakeClient(appCtx, configuration);
- List<PathItem> filesOnly = ExternalDataUtils.Azure.listDatalakePathItems(client, configuration,
- includeExcludeMatcher, warningCollector);
+ DataLakeServiceClient client = buildAzureDatalakeClient(appCtx, configuration);
+ List<PathItem> filesOnly =
+ listDatalakePathItems(client, configuration, includeExcludeMatcher, warningCollector);
// Distribute work load amongst the partitions
distributeWorkLoad(filesOnly, getPartitionsCount());
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/parquet/AzureBlobParquetReaderFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/parquet/AzureBlobParquetReaderFactory.java
index c2251df..e08013c 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/parquet/AzureBlobParquetReaderFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/parquet/AzureBlobParquetReaderFactory.java
@@ -18,6 +18,11 @@
*/
package org.apache.asterix.external.input.record.reader.azure.parquet;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.HADOOP_AZURE_BLOB_PROTOCOL;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.buildAzureBlobClient;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.configureAzureHdfsJobConf;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.listBlobItems;
+
import java.util.Collections;
import java.util.List;
import java.util.Map;
@@ -47,7 +52,7 @@
public void configure(IServiceContext serviceCtx, Map<String, String> configuration,
IWarningCollector warningCollector) throws AlgebricksException, HyracksDataException {
IApplicationContext appCtx = (IApplicationContext) serviceCtx.getApplicationContext();
- BlobServiceClient blobServiceClient = ExternalDataUtils.Azure.buildAzureBlobClient(appCtx, configuration);
+ BlobServiceClient blobServiceClient = buildAzureBlobClient(appCtx, configuration);
//Get endpoint
String endPoint = extractEndPoint(blobServiceClient.getAccountUrl());
//Get path
@@ -57,7 +62,7 @@
//Configure Hadoop Azure input splits
JobConf conf = createHdfsConf(serviceCtx, configuration);
- ExternalDataUtils.Azure.configureAzureHdfsJobConf(conf, configuration, endPoint);
+ configureAzureHdfsJobConf(conf, configuration, endPoint);
configureHdfsConf(conf, configuration);
}
@@ -94,8 +99,8 @@
private static String buildPathURIs(Map<String, String> configuration, IWarningCollector warningCollector,
BlobServiceClient blobServiceClient, String endPoint) throws CompilationException {
IncludeExcludeMatcher includeExcludeMatcher = ExternalDataUtils.getIncludeExcludeMatchers(configuration);
- List<BlobItem> filesOnly = ExternalDataUtils.Azure.listBlobItems(blobServiceClient, configuration,
- includeExcludeMatcher, warningCollector);
+ List<BlobItem> filesOnly =
+ listBlobItems(blobServiceClient, configuration, includeExcludeMatcher, warningCollector);
StringBuilder builder = new StringBuilder();
String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
@@ -118,7 +123,7 @@
}
private static void appendFileURI(StringBuilder builder, String container, String endPoint, BlobItem file) {
- builder.append(ExternalDataConstants.Azure.HADOOP_AZURE_BLOB_PROTOCOL);
+ builder.append(HADOOP_AZURE_BLOB_PROTOCOL);
builder.append("://");
builder.append(container);
builder.append('@');
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/parquet/AzureDataLakeParquetReaderFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/parquet/AzureDataLakeParquetReaderFactory.java
index db87868..c98fc8b 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/parquet/AzureDataLakeParquetReaderFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/parquet/AzureDataLakeParquetReaderFactory.java
@@ -18,6 +18,11 @@
*/
package org.apache.asterix.external.input.record.reader.azure.parquet;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.HADOOP_AZURE_DATALAKE_PROTOCOL;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.buildAzureDatalakeClient;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.configureAzureHdfsJobConf;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.listDatalakePathItems;
+
import java.util.Collections;
import java.util.List;
import java.util.Map;
@@ -47,8 +52,7 @@
public void configure(IServiceContext serviceCtx, Map<String, String> configuration,
IWarningCollector warningCollector) throws AlgebricksException, HyracksDataException {
IApplicationContext appCtx = (IApplicationContext) serviceCtx.getApplicationContext();
- DataLakeServiceClient dataLakeServiceClient =
- ExternalDataUtils.Azure.buildAzureDatalakeClient(appCtx, configuration);
+ DataLakeServiceClient dataLakeServiceClient = buildAzureDatalakeClient(appCtx, configuration);
//Get endpoint
String endPoint = extractEndPoint(dataLakeServiceClient.getAccountUrl());
@@ -61,7 +65,7 @@
//Configure Hadoop Azure input splits
JobConf conf = createHdfsConf(serviceCtx, configuration);
- ExternalDataUtils.Azure.configureAzureHdfsJobConf(conf, configuration, endPoint);
+ configureAzureHdfsJobConf(conf, configuration, endPoint);
configureHdfsConf(conf, configuration);
}
@@ -98,8 +102,8 @@
private static String buildPathURIs(Map<String, String> configuration, IWarningCollector warningCollector,
DataLakeServiceClient dataLakeServiceClient, String endPoint) throws CompilationException {
IncludeExcludeMatcher includeExcludeMatcher = ExternalDataUtils.getIncludeExcludeMatchers(configuration);
- List<PathItem> filesOnly = ExternalDataUtils.Azure.listDatalakePathItems(dataLakeServiceClient, configuration,
- includeExcludeMatcher, warningCollector);
+ List<PathItem> filesOnly =
+ listDatalakePathItems(dataLakeServiceClient, configuration, includeExcludeMatcher, warningCollector);
StringBuilder builder = new StringBuilder();
String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
@@ -122,7 +126,7 @@
}
private static void appendFileURI(StringBuilder builder, String container, String endPoint, PathItem file) {
- builder.append(ExternalDataConstants.Azure.HADOOP_AZURE_DATALAKE_PROTOCOL);
+ builder.append(HADOOP_AZURE_DATALAKE_PROTOCOL);
builder.append("://");
builder.append(container);
builder.append('@');
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/GCSInputStream.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/GCSInputStream.java
index 652fa3e..007e8be 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/GCSInputStream.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/GCSInputStream.java
@@ -32,7 +32,7 @@
import org.apache.asterix.common.exceptions.RuntimeDataException;
import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStream;
import org.apache.asterix.external.util.ExternalDataConstants;
-import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.asterix.external.util.google.gcs.GCSUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.util.CleanupUtils;
@@ -134,7 +134,7 @@
private Storage buildClient(Map<String, String> configuration) throws HyracksDataException {
try {
- return ExternalDataUtils.GCS.buildClient(configuration);
+ return GCSUtils.buildClient(configuration);
} catch (CompilationException ex) {
throw HyracksDataException.create(ex);
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/GCSInputStreamFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/GCSInputStreamFactory.java
index 0e7ea90..278c1ad 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/GCSInputStreamFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/GCSInputStreamFactory.java
@@ -19,33 +19,23 @@
package org.apache.asterix.external.input.record.reader.gcs;
import static org.apache.asterix.external.util.ExternalDataUtils.getIncludeExcludeMatchers;
-import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
-import java.util.ArrayList;
import java.util.Comparator;
import java.util.List;
import java.util.Map;
import java.util.PriorityQueue;
-import java.util.function.BiPredicate;
-import java.util.regex.Matcher;
-import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.external.api.AsterixInputStream;
import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStreamFactory;
-import org.apache.asterix.external.util.ExternalDataConstants;
import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.asterix.external.util.google.gcs.GCSUtils;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.api.application.IServiceContext;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.exceptions.IWarningCollector;
-import org.apache.hyracks.api.exceptions.Warning;
-import com.google.api.gax.paging.Page;
-import com.google.cloud.BaseServiceException;
import com.google.cloud.storage.Blob;
-import com.google.cloud.storage.Storage;
public class GCSInputStreamFactory extends AbstractExternalInputStreamFactory {
@@ -63,57 +53,16 @@
// Ensure the validity of include/exclude
ExternalDataUtils.validateIncludeExclude(configuration);
-
- // Prepare to retrieve the objects
- List<Blob> filesOnly = new ArrayList<>();
- String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
- Storage gcs = ExternalDataUtils.GCS.buildClient(configuration);
- Storage.BlobListOption options = Storage.BlobListOption.prefix(ExternalDataUtils.getPrefix(configuration));
- Page<Blob> items;
-
- try {
- items = gcs.list(container, options);
- } catch (BaseServiceException ex) {
- throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
- }
-
- // Collect the paths to files only
IncludeExcludeMatcher includeExcludeMatcher = getIncludeExcludeMatchers(configuration);
- collectAndFilterFiles(items, includeExcludeMatcher.getPredicate(), includeExcludeMatcher.getMatchersList(),
- filesOnly);
- // Warn if no files are returned
- if (filesOnly.isEmpty() && warningCollector.shouldWarn()) {
- Warning warning = Warning.of(null, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
- warningCollector.warn(warning);
- }
+ // get the items
+ List<Blob> filesOnly = GCSUtils.listItems(configuration, includeExcludeMatcher, warningCollector);
// Distribute work load amongst the partitions
distributeWorkLoad(filesOnly, getPartitionsCount());
}
/**
- * AWS S3 returns all the objects as paths, not differentiating between folder and files. The path is considered
- * a file if it does not end up with a "/" which is the separator in a folder structure.
- *
- * @param items List of returned objects
- */
- private void collectAndFilterFiles(Page<Blob> items, BiPredicate<List<Matcher>, String> predicate,
- List<Matcher> matchers, List<Blob> filesOnly) {
- for (Blob item : items.iterateAll()) {
- // skip folders
- if (item.getName().endsWith("/")) {
- continue;
- }
-
- // No filter, add file
- if (predicate.test(matchers, item.getName())) {
- filesOnly.add(item);
- }
- }
- }
-
- /**
* To efficiently utilize the parallelism, work load will be distributed amongst the partitions based on the file
* size.
*
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/parquet/GCSParquetReaderFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/parquet/GCSParquetReaderFactory.java
new file mode 100644
index 0000000..2887415
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/parquet/GCSParquetReaderFactory.java
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.input.record.reader.gcs.parquet;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.external.input.HDFSDataSourceFactory;
+import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStreamFactory.IncludeExcludeMatcher;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.asterix.external.util.google.gcs.GCSConstants;
+import org.apache.asterix.external.util.google.gcs.GCSUtils;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.application.IServiceContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+
+import com.google.cloud.storage.Blob;
+
+public class GCSParquetReaderFactory extends HDFSDataSourceFactory {
+ private static final long serialVersionUID = -6140824803254158253L;
+ private static final List<String> recordReaderNames =
+ Collections.singletonList(ExternalDataConstants.KEY_ADAPTER_NAME_GCS);
+
+ @Override
+ public void configure(IServiceContext serviceCtx, Map<String, String> configuration,
+ IWarningCollector warningCollector) throws AlgebricksException, HyracksDataException {
+ // get path
+ String path = buildPathURIs(configuration, warningCollector);
+
+ // put GCS configurations to AsterixDB's Hadoop configuration
+ putGCSConfToHadoopConf(configuration, path);
+
+ // configure hadoop input splits
+ JobConf conf = createHdfsConf(serviceCtx, configuration);
+ int numberOfPartitions = getPartitionConstraint().getLocations().length;
+ GCSUtils.configureHdfsJobConf(conf, configuration, numberOfPartitions);
+ configureHdfsConf(conf, configuration);
+ }
+
+ @Override
+ public List<String> getRecordReaderNames() {
+ return recordReaderNames;
+ }
+
+ @Override
+ public Set<String> getReaderSupportedFormats() {
+ return Collections.singleton(ExternalDataConstants.FORMAT_PARQUET);
+ }
+
+ /**
+ * Prepare Hadoop configurations to read parquet files
+ *
+ * @param path Comma-delimited paths
+ */
+ private static void putGCSConfToHadoopConf(Map<String, String> configuration, String path) {
+ configuration.put(ExternalDataConstants.KEY_PATH, path);
+ configuration.put(ExternalDataConstants.KEY_INPUT_FORMAT, ExternalDataConstants.INPUT_FORMAT_PARQUET);
+ configuration.put(ExternalDataConstants.KEY_PARSER, ExternalDataConstants.FORMAT_NOOP);
+ }
+
+ /**
+ * Build Google Cloud Storage path-style for the requested files
+ *
+ * @param configuration properties
+ * @param warningCollector warning collector
+ * @return Comma-delimited paths (e.g., "gs://bucket/file1.parquet,gs://bucket/file2.parquet")
+ * @throws CompilationException Compilation exception
+ */
+ private static String buildPathURIs(Map<String, String> configuration, IWarningCollector warningCollector)
+ throws CompilationException {
+ String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
+
+ // Ensure the validity of include/exclude
+ ExternalDataUtils.validateIncludeExclude(configuration);
+ IncludeExcludeMatcher includeExcludeMatcher = ExternalDataUtils.getIncludeExcludeMatchers(configuration);
+
+ // get the items
+ List<Blob> filesOnly = GCSUtils.listItems(configuration, includeExcludeMatcher, warningCollector);
+ StringBuilder builder = new StringBuilder();
+
+ if (!filesOnly.isEmpty()) {
+ appendFileURI(builder, container, filesOnly.get(0));
+ for (int i = 1; i < filesOnly.size(); i++) {
+ builder.append(',');
+ appendFileURI(builder, container, filesOnly.get(i));
+ }
+ }
+
+ return builder.toString();
+ }
+
+ private static void appendFileURI(StringBuilder builder, String container, Blob file) {
+ builder.append(GCSConstants.HADOOP_GCS_PROTOCOL);
+ builder.append("://");
+ builder.append(container);
+ builder.append('/');
+ builder.append(file.getName());
+ }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/AsterixTypeToParquetTypeVisitor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/AsterixTypeToParquetTypeVisitor.java
index 7258359..68d5b86 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/AsterixTypeToParquetTypeVisitor.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/AsterixTypeToParquetTypeVisitor.java
@@ -174,7 +174,7 @@
//If no warning is created, then it means it has been reported
Warning warning = null;
if (actualType != ATypeTag.SYSTEM_NULL) {
- warning = info.createTypeMismatchWarning(expectedType, actualType);
+ warning = info.createWarning(expectedType, actualType);
}
if (warning != null) {
//New warning that we saw for the first time. We should report it.
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/MapredParquetInputFormat.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/MapredParquetInputFormat.java
index 328e09d..d3ad968 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/MapredParquetInputFormat.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/MapredParquetInputFormat.java
@@ -25,14 +25,18 @@
import java.io.IOException;
import java.util.List;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
import org.apache.hadoop.mapred.FileSplit;
import org.apache.hadoop.mapred.InputSplit;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.RecordReader;
import org.apache.hadoop.mapred.Reporter;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IValueReference;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.util.LogRedactionUtil;
import org.apache.parquet.hadoop.Footer;
import org.apache.parquet.hadoop.ParquetInputFormat;
import org.apache.parquet.hadoop.ParquetInputSplit;
@@ -103,8 +107,8 @@
} else if (oldSplit instanceof FileSplit) {
realReader.initialize((FileSplit) oldSplit, oldJobConf, reporter);
} else {
- throw new IllegalArgumentException(
- "Invalid split (not a FileSplit or ParquetInputSplitWrapper): " + oldSplit);
+ throw RuntimeDataException.create(ErrorCode.INVALID_PARQUET_FILE,
+ LogRedactionUtil.userData(oldSplit.toString()), "invalid file split");
}
valueContainer = new VoidPointable();
firstRecord = false;
@@ -119,6 +123,26 @@
}
} catch (InterruptedException e) {
throw new IOException(e);
+ } catch (HyracksDataException | AsterixParquetRuntimeException e) {
+ throw e;
+ } catch (Exception e) {
+ if (e.getMessage() != null && e.getMessage().contains("not a Parquet file")) {
+ throw RuntimeDataException.create(ErrorCode.INVALID_PARQUET_FILE,
+ LogRedactionUtil.userData(getPath(oldSplit)), "not a Parquet file");
+ }
+
+ throw RuntimeDataException.create(ErrorCode.UNEXPECTED_ERROR_ENCOUNTERED,
+ LogRedactionUtil.userData(e.toString()));
+ }
+ }
+
+ private String getPath(InputSplit split) {
+ if (split instanceof FileSplit) {
+ return ((FileSplit) split).getPath().toString();
+ } else if (split instanceof ParquetInputSplitWrapper) {
+ return ((ParquetInputSplitWrapper) split).realSplit.getPath().toString();
+ } else {
+ return split.toString();
}
}
@@ -215,5 +239,10 @@
public void write(DataOutput out) throws IOException {
realSplit.write(out);
}
+
+ @Override
+ public String toString() {
+ return realSplit.toString();
+ }
}
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/ipc/AbstractPythonIPCProto.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/ipc/AbstractPythonIPCProto.java
new file mode 100644
index 0000000..00d1dcc
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/ipc/AbstractPythonIPCProto.java
@@ -0,0 +1,183 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.ipc;
+
+import static org.apache.hyracks.ipc.impl.Message.HEADER_SIZE;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.external.api.IExternalLangIPCProto;
+import org.apache.asterix.external.library.msgpack.MsgPackPointableVisitor;
+import org.apache.asterix.om.pointables.PointableAllocator;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.ipc.impl.Message;
+import org.msgpack.core.MessagePack;
+import org.msgpack.core.MessageUnpacker;
+import org.msgpack.core.buffer.ArrayBufferInput;
+
+public abstract class AbstractPythonIPCProto {
+ public static final int HEADER_SIZE_LEN_INCLUSIVE = 21;
+ protected final PythonMessageBuilder messageBuilder;
+ protected final DataOutputStream sockOut;
+ protected final ArrayBufferInput unpackerInput;
+ protected final MessageUnpacker unpacker;
+ protected final ArrayBackedValueStorage argsStorage;
+ protected final PointableAllocator pointableAllocator;
+ protected final MsgPackPointableVisitor pointableVisitor;
+ private final ByteBuffer headerBuffer = ByteBuffer.allocate(HEADER_SIZE_LEN_INCLUSIVE);
+ protected ByteBuffer recvBuffer = ByteBuffer.allocate(32768);
+ protected long routeId;
+ protected Pair<ByteBuffer, Exception> bufferBox;
+ protected long maxFunctionId;
+
+ public AbstractPythonIPCProto(OutputStream sockOut) {
+ messageBuilder = new PythonMessageBuilder();
+ this.sockOut = new DataOutputStream(sockOut);
+ this.maxFunctionId = 0L;
+ unpackerInput = new ArrayBufferInput(new byte[0]);
+ unpacker = MessagePack.newDefaultUnpacker(unpackerInput);
+ this.argsStorage = new ArrayBackedValueStorage();
+ this.pointableAllocator = new PointableAllocator();
+ this.pointableVisitor = new MsgPackPointableVisitor();
+ }
+
+ public void helo() throws IOException, AsterixException {
+ recvBuffer.clear();
+ recvBuffer.position(0);
+ recvBuffer.limit(0);
+ messageBuilder.reset();
+ messageBuilder.hello();
+ sendHeader(routeId, messageBuilder.getLength());
+ sendMsg();
+ receiveMsg();
+ if (getResponseType() != MessageType.HELO) {
+ throw HyracksDataException.create(org.apache.hyracks.api.exceptions.ErrorCode.ILLEGAL_STATE,
+ "Expected HELO, recieved " + getResponseType().name());
+ }
+ }
+
+ public long init(String module, String clazz, String fn) throws IOException, AsterixException {
+ long functionId = maxFunctionId++;
+ recvBuffer.clear();
+ recvBuffer.position(0);
+ recvBuffer.limit(0);
+ messageBuilder.reset();
+ messageBuilder.init(module, clazz, fn);
+ sendHeader(functionId, messageBuilder.getLength());
+ sendMsg();
+ receiveMsg();
+ if (getResponseType() != MessageType.INIT_RSP) {
+ throw HyracksDataException.create(org.apache.hyracks.api.exceptions.ErrorCode.ILLEGAL_STATE,
+ "Expected INIT_RSP, recieved " + getResponseType().name());
+ }
+ return functionId;
+ }
+
+ public ByteBuffer call(long functionId, IAType[] argTypes, IValueReference[] argValues, boolean nullCall)
+ throws IOException, AsterixException {
+ recvBuffer.clear();
+ recvBuffer.position(0);
+ recvBuffer.limit(0);
+ messageBuilder.reset();
+ argsStorage.reset();
+ for (int i = 0; i < argTypes.length; i++) {
+ IExternalLangIPCProto.visitValueRef(argTypes[i], argsStorage.getDataOutput(), argValues[i],
+ pointableAllocator, pointableVisitor, nullCall);
+ }
+ int len = argsStorage.getLength() + 5;
+ sendHeader(functionId, len);
+ messageBuilder.call(argValues.length, len);
+ sendMsg(argsStorage);
+ receiveMsg();
+ if (getResponseType() != MessageType.CALL_RSP) {
+ throw HyracksDataException.create(org.apache.hyracks.api.exceptions.ErrorCode.ILLEGAL_STATE,
+ "Expected CALL_RSP, recieved " + getResponseType().name());
+ }
+ return recvBuffer;
+ }
+
+ public ByteBuffer callMulti(long key, ArrayBackedValueStorage args, int numTuples)
+ throws IOException, AsterixException {
+ recvBuffer.clear();
+ recvBuffer.position(0);
+ recvBuffer.limit(0);
+ messageBuilder.reset();
+ int len = args.getLength() + 4;
+ sendHeader(key, len);
+ messageBuilder.callMulti(0, numTuples);
+ sendMsg(args);
+ receiveMsg();
+ if (getResponseType() != MessageType.CALL_RSP) {
+ throw HyracksDataException.create(org.apache.hyracks.api.exceptions.ErrorCode.ILLEGAL_STATE,
+ "Expected CALL_RSP, recieved " + getResponseType().name());
+ }
+ return recvBuffer;
+ }
+
+ public void quit() throws HyracksDataException {
+ messageBuilder.quit();
+ }
+
+ public abstract void receiveMsg() throws IOException, AsterixException;
+
+ public void sendHeader(long key, int msgLen) throws IOException {
+ headerBuffer.clear();
+ headerBuffer.position(0);
+ headerBuffer.putInt(HEADER_SIZE + Integer.BYTES + msgLen);
+ headerBuffer.putLong(key);
+ headerBuffer.putLong(routeId);
+ headerBuffer.put(Message.NORMAL);
+ sockOut.write(headerBuffer.array(), 0, HEADER_SIZE + Integer.BYTES);
+ sockOut.flush();
+ }
+
+ public void sendMsg(ArrayBackedValueStorage content) throws IOException {
+ sockOut.write(messageBuilder.getBuf().array(), messageBuilder.getBuf().arrayOffset(),
+ messageBuilder.getBuf().position());
+ sockOut.write(content.getByteArray(), content.getStartOffset(), content.getLength());
+ sockOut.flush();
+ }
+
+ public void sendMsg() throws IOException {
+ sockOut.write(messageBuilder.getBuf().array(), messageBuilder.getBuf().arrayOffset(),
+ messageBuilder.getBuf().position());
+ sockOut.flush();
+ }
+
+ public MessageType getResponseType() {
+ return messageBuilder.type;
+ }
+
+ public long getRouteId() {
+ return routeId;
+ }
+
+ public DataOutputStream getSockOut() {
+ return sockOut;
+ }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/ipc/PythonDomainSocketProto.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/ipc/PythonDomainSocketProto.java
new file mode 100644
index 0000000..89f240a
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/ipc/PythonDomainSocketProto.java
@@ -0,0 +1,161 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.ipc;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.nio.channels.SocketChannel;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.external.api.IExternalLangIPCProto;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.msgpack.core.MessagePack;
+
+public class PythonDomainSocketProto extends AbstractPythonIPCProto implements IExternalLangIPCProto {
+ private final String wd;
+ SocketChannel chan;
+ private ByteBuffer headerBuffer;
+ private ProcessHandle pid;
+ public static final int HYR_HEADER_SIZE = 21; // 4 (sz) + 8 (mid) + 8 (rmid) + 1 (flags)
+ public static final int HYR_HEADER_SIZE_NOSZ = 17; // 8 + 8 + 1
+
+ public PythonDomainSocketProto(OutputStream sockOut, SocketChannel chan, String wd) {
+ super(sockOut);
+ this.chan = chan;
+ this.wd = wd;
+ headerBuffer = ByteBuffer.allocate(HYR_HEADER_SIZE);
+ }
+
+ @Override
+ public void start() {
+ }
+
+ @Override
+ public void helo() throws IOException, AsterixException {
+ recvBuffer.clear();
+ recvBuffer.position(0);
+ recvBuffer.limit(0);
+ messageBuilder.reset();
+ messageBuilder.helloDS(wd);
+ sendHeader(routeId, messageBuilder.getLength());
+ sendMsg(true);
+ receiveMsg(true);
+ byte pidType = recvBuffer.get();
+ if (pidType != MessagePack.Code.UINT32 && pidType != MessagePack.Code.UINT16) {
+ throw AsterixException.create(ErrorCode.EXTERNAL_UDF_EXCEPTION,
+ "Returned pid type is incorrect: " + pidType);
+ }
+ switch (pidType) {
+ case MessagePack.Code.UINT32:
+ pid = ProcessHandle.of(recvBuffer.getInt()).get();
+ break;
+ case MessagePack.Code.UINT16:
+ pid = ProcessHandle.of(recvBuffer.getShort()).get();
+ break;
+ case MessagePack.Code.UINT8:
+ pid = ProcessHandle.of(recvBuffer.get()).get();
+ break;
+ default:
+ throw AsterixException.create(ErrorCode.EXTERNAL_UDF_EXCEPTION,
+ "Returned pid type is incorrect: " + pidType);
+ }
+ if (getResponseType() != MessageType.HELO) {
+ throw HyracksDataException.create(org.apache.hyracks.api.exceptions.ErrorCode.ILLEGAL_STATE,
+ "Expected HELO, recieved " + getResponseType().name());
+ }
+ }
+
+ @Override
+ public void sendMsg() throws IOException {
+ sendMsg(false);
+ }
+
+ @Override
+ public void sendMsg(ArrayBackedValueStorage args) throws IOException {
+ sendMsg(false, args);
+ }
+
+ public void sendMsg(boolean sendIfDead) throws IOException {
+ if (!sendIfDead && (pid == null || !pid.isAlive())) {
+ return;
+ }
+ super.sendMsg();
+ }
+
+ public void sendMsg(boolean sendIfDead, ArrayBackedValueStorage args) throws IOException {
+ if (!sendIfDead && (pid == null || !pid.isAlive())) {
+ return;
+ }
+ super.sendMsg(args);
+ }
+
+ @Override
+ public void receiveMsg() throws IOException, AsterixException {
+ receiveMsg(false);
+ }
+
+ public void receiveMsg(boolean sendIfDead) throws IOException, AsterixException {
+ if (!sendIfDead && (pid == null || !pid.isAlive())) {
+ throw new AsterixException("Python process exited unexpectedly");
+ }
+ readFully(headerBuffer.capacity(), headerBuffer);
+ if (headerBuffer.remaining() < Integer.BYTES) {
+ recvBuffer.limit(0);
+ throw new AsterixException("Python process exited unexpectedly");
+ }
+ int msgSz = headerBuffer.getInt() - HYR_HEADER_SIZE_NOSZ;
+ if (recvBuffer.capacity() < msgSz) {
+ recvBuffer = ByteBuffer.allocate(((msgSz / 32768) + 1) * 32768);
+ }
+ readFully(msgSz, recvBuffer);
+ messageBuilder.readHead(recvBuffer);
+ if (messageBuilder.type == MessageType.ERROR) {
+ unpackerInput.reset(recvBuffer.array(), recvBuffer.position() + recvBuffer.arrayOffset(),
+ recvBuffer.remaining());
+ unpacker.reset(unpackerInput);
+ throw new AsterixException(unpacker.unpackString().replace('\0', ' '));
+ }
+ }
+
+ private void readFully(int msgSz, ByteBuffer buf) throws IOException, AsterixException {
+ buf.limit(msgSz);
+ buf.clear();
+ int read;
+ int size = msgSz;
+ while (size > 0) {
+ read = chan.read(buf);
+ if (read < 0) {
+ throw new AsterixException("Socket closed");
+ }
+ size -= read;
+ }
+ buf.flip();
+ }
+
+ @Override
+ public void quit() throws HyracksDataException {
+ messageBuilder.quit();
+ }
+
+ public ProcessHandle getPid() {
+ return pid;
+ }
+
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/ipc/PythonIPCProto.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/ipc/PythonIPCProto.java
deleted file mode 100644
index c803517..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/ipc/PythonIPCProto.java
+++ /dev/null
@@ -1,290 +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.external.ipc;
-
-import static org.apache.hyracks.ipc.impl.Message.HEADER_SIZE;
-
-import java.io.DataOutput;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.nio.ByteBuffer;
-
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.common.exceptions.ErrorCode;
-import org.apache.asterix.external.library.msgpack.MsgPackPointableVisitor;
-import org.apache.asterix.om.pointables.AFlatValuePointable;
-import org.apache.asterix.om.pointables.AListVisitablePointable;
-import org.apache.asterix.om.pointables.ARecordVisitablePointable;
-import org.apache.asterix.om.pointables.PointableAllocator;
-import org.apache.asterix.om.pointables.base.IVisitablePointable;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.EnumDeserializer;
-import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.types.TypeTagUtil;
-import org.apache.hyracks.algebricks.common.utils.Pair;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.api.IValueReference;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-import org.apache.hyracks.ipc.impl.Message;
-import org.msgpack.core.MessagePack;
-import org.msgpack.core.MessageUnpacker;
-import org.msgpack.core.buffer.ArrayBufferInput;
-
-public class PythonIPCProto {
-
- private final PythonMessageBuilder messageBuilder;
- private final DataOutputStream sockOut;
- private final ByteBuffer headerBuffer = ByteBuffer.allocate(21);
- private ByteBuffer recvBuffer = ByteBuffer.allocate(32768);
- private final ExternalFunctionResultRouter router;
- private long routeId;
- private Pair<ByteBuffer, Exception> bufferBox;
- private final Process pythonProc;
- private long maxFunctionId;
- private final ArrayBufferInput unpackerInput;
- private final MessageUnpacker unpacker;
- private final ArrayBackedValueStorage argsStorage;
- private final PointableAllocator pointableAllocator;
- private final MsgPackPointableVisitor pointableVisitor;
-
- public PythonIPCProto(OutputStream sockOut, ExternalFunctionResultRouter router, Process pythonProc) {
- this.sockOut = new DataOutputStream(sockOut);
- messageBuilder = new PythonMessageBuilder();
- this.router = router;
- this.pythonProc = pythonProc;
- this.maxFunctionId = 0L;
- unpackerInput = new ArrayBufferInput(new byte[0]);
- unpacker = MessagePack.newDefaultUnpacker(unpackerInput);
- this.argsStorage = new ArrayBackedValueStorage();
- this.pointableAllocator = new PointableAllocator();
- this.pointableVisitor = new MsgPackPointableVisitor();
- }
-
- public void start() {
- Pair<Long, Pair<ByteBuffer, Exception>> keyAndBufferBox = router.insertRoute(recvBuffer);
- this.routeId = keyAndBufferBox.getFirst();
- this.bufferBox = keyAndBufferBox.getSecond();
- }
-
- public void helo() throws IOException, AsterixException {
- recvBuffer.clear();
- recvBuffer.position(0);
- recvBuffer.limit(0);
- messageBuilder.reset();
- messageBuilder.hello();
- sendHeader(routeId, messageBuilder.getLength());
- sendMsg();
- receiveMsg();
- if (getResponseType() != MessageType.HELO) {
- throw HyracksDataException.create(org.apache.hyracks.api.exceptions.ErrorCode.ILLEGAL_STATE,
- "Expected HELO, recieved " + getResponseType().name());
- }
- }
-
- public long init(String module, String clazz, String fn) throws IOException, AsterixException {
- long functionId = maxFunctionId++;
- recvBuffer.clear();
- recvBuffer.position(0);
- recvBuffer.limit(0);
- messageBuilder.reset();
- messageBuilder.init(module, clazz, fn);
- sendHeader(functionId, messageBuilder.getLength());
- sendMsg();
- receiveMsg();
- if (getResponseType() != MessageType.INIT_RSP) {
- throw HyracksDataException.create(org.apache.hyracks.api.exceptions.ErrorCode.ILLEGAL_STATE,
- "Expected INIT_RSP, recieved " + getResponseType().name());
- }
- return functionId;
- }
-
- public ByteBuffer call(long functionId, IAType[] argTypes, IValueReference[] argValues, boolean nullCall)
- throws IOException, AsterixException {
- recvBuffer.clear();
- recvBuffer.position(0);
- recvBuffer.limit(0);
- messageBuilder.reset();
- argsStorage.reset();
- for (int i = 0; i < argTypes.length; i++) {
- visitValueRef(argTypes[i], argsStorage.getDataOutput(), argValues[i], pointableAllocator, pointableVisitor,
- nullCall);
- }
- int len = argsStorage.getLength() + 5;
- sendHeader(functionId, len);
- messageBuilder.call(argValues.length, len);
- sendMsg(argsStorage);
- receiveMsg();
- if (getResponseType() != MessageType.CALL_RSP) {
- throw HyracksDataException.create(org.apache.hyracks.api.exceptions.ErrorCode.ILLEGAL_STATE,
- "Expected CALL_RSP, recieved " + getResponseType().name());
- }
- return recvBuffer;
- }
-
- public ByteBuffer callMulti(long key, ArrayBackedValueStorage args, int numTuples)
- throws IOException, AsterixException {
- recvBuffer.clear();
- recvBuffer.position(0);
- recvBuffer.limit(0);
- messageBuilder.reset();
- int len = args.getLength() + 4;
- sendHeader(key, len);
- messageBuilder.callMulti(0, numTuples);
- sendMsg(args);
- receiveMsg();
- if (getResponseType() != MessageType.CALL_RSP) {
- throw HyracksDataException.create(org.apache.hyracks.api.exceptions.ErrorCode.ILLEGAL_STATE,
- "Expected CALL_RSP, recieved " + getResponseType().name());
- }
- return recvBuffer;
- }
-
- //For future use with interpreter reuse between jobs.
- public void quit() throws HyracksDataException {
- messageBuilder.quit();
- router.removeRoute(routeId);
- }
-
- public void receiveMsg() throws IOException, AsterixException {
- Exception except;
- try {
- synchronized (bufferBox) {
- while ((bufferBox.getFirst().limit() == 0 || bufferBox.getSecond() != null) && pythonProc.isAlive()) {
- bufferBox.wait(100);
- }
- }
- except = router.getAndRemoveException(routeId);
- if (!pythonProc.isAlive()) {
- except = new IOException("Python process exited with code: " + pythonProc.exitValue());
- }
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- throw new AsterixException(ErrorCode.EXTERNAL_UDF_EXCEPTION, e);
- }
- if (except != null) {
- throw new AsterixException(except);
- }
- if (bufferBox.getFirst() != recvBuffer) {
- recvBuffer = bufferBox.getFirst();
- }
- messageBuilder.readHead(recvBuffer);
- if (messageBuilder.type == MessageType.ERROR) {
- unpackerInput.reset(recvBuffer.array(), recvBuffer.position() + recvBuffer.arrayOffset(),
- recvBuffer.remaining());
- unpacker.reset(unpackerInput);
- throw new AsterixException(unpacker.unpackString());
- }
- }
-
- public void sendHeader(long key, int msgLen) throws IOException {
- headerBuffer.clear();
- headerBuffer.position(0);
- headerBuffer.putInt(HEADER_SIZE + Integer.BYTES + msgLen);
- headerBuffer.putLong(key);
- headerBuffer.putLong(routeId);
- headerBuffer.put(Message.NORMAL);
- sockOut.write(headerBuffer.array(), 0, HEADER_SIZE + Integer.BYTES);
- sockOut.flush();
- }
-
- public void sendMsg(ArrayBackedValueStorage content) throws IOException {
- sockOut.write(messageBuilder.getBuf().array(), messageBuilder.getBuf().arrayOffset(),
- messageBuilder.getBuf().position());
- sockOut.write(content.getByteArray(), content.getStartOffset(), content.getLength());
- sockOut.flush();
- }
-
- public void sendMsg() throws IOException {
- sockOut.write(messageBuilder.getBuf().array(), messageBuilder.getBuf().arrayOffset(),
- messageBuilder.getBuf().position());
- sockOut.flush();
- }
-
- public MessageType getResponseType() {
- return messageBuilder.type;
- }
-
- public long getRouteId() {
- return routeId;
- }
-
- public DataOutputStream getSockOut() {
- return sockOut;
- }
-
- public static void visitValueRef(IAType type, DataOutput out, IValueReference valueReference,
- PointableAllocator pointableAllocator, MsgPackPointableVisitor pointableVisitor, boolean visitNull)
- throws IOException {
- IVisitablePointable pointable;
- switch (type.getTypeTag()) {
- case OBJECT:
- pointable = pointableAllocator.allocateRecordValue(type);
- pointable.set(valueReference);
- pointableVisitor.visit((ARecordVisitablePointable) pointable, pointableVisitor.getTypeInfo(type, out));
- break;
- case ARRAY:
- case MULTISET:
- pointable = pointableAllocator.allocateListValue(type);
- pointable.set(valueReference);
- pointableVisitor.visit((AListVisitablePointable) pointable, pointableVisitor.getTypeInfo(type, out));
- break;
- case ANY:
- ATypeTag rtTypeTag = EnumDeserializer.ATYPETAGDESERIALIZER
- .deserialize(valueReference.getByteArray()[valueReference.getStartOffset()]);
- IAType rtType = TypeTagUtil.getBuiltinTypeByTag(rtTypeTag);
- switch (rtTypeTag) {
- case OBJECT:
- pointable = pointableAllocator.allocateRecordValue(rtType);
- pointable.set(valueReference);
- pointableVisitor.visit((ARecordVisitablePointable) pointable,
- pointableVisitor.getTypeInfo(rtType, out));
- break;
- case ARRAY:
- case MULTISET:
- pointable = pointableAllocator.allocateListValue(rtType);
- pointable.set(valueReference);
- pointableVisitor.visit((AListVisitablePointable) pointable,
- pointableVisitor.getTypeInfo(rtType, out));
- break;
- case MISSING:
- case NULL:
- if (!visitNull) {
- return;
- }
- default:
- pointable = pointableAllocator.allocateFieldValue(rtType);
- pointable.set(valueReference);
- pointableVisitor.visit((AFlatValuePointable) pointable,
- pointableVisitor.getTypeInfo(rtType, out));
- break;
- }
- break;
- case MISSING:
- case NULL:
- if (!visitNull) {
- return;
- }
- default:
- pointable = pointableAllocator.allocateFieldValue(type);
- pointable.set(valueReference);
- pointableVisitor.visit((AFlatValuePointable) pointable, pointableVisitor.getTypeInfo(type, out));
- break;
- }
- }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/ipc/PythonMessageBuilder.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/ipc/PythonMessageBuilder.java
index 5429657..20f8306 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/ipc/PythonMessageBuilder.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/ipc/PythonMessageBuilder.java
@@ -82,6 +82,16 @@
buf.put(serAddr);
}
+ public void helloDS(String modulePath) throws IOException {
+ this.type = MessageType.HELO;
+ // sum(string lengths) + 2 from fix array tag and message type
+ dataLength = PythonMessageBuilder.getStringLength(modulePath) + 2;
+ packHeader();
+ MessagePackUtils.packFixArrayHeader(buf, (byte) 2);
+ MessagePackUtils.packStr(buf, "HELLO");
+ MessagePackUtils.packStr(buf, modulePath);
+ }
+
public void quit() throws HyracksDataException {
this.type = MessageType.QUIT;
dataLength = getStringLength("QUIT");
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/ipc/PythonTCPSocketProto.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/ipc/PythonTCPSocketProto.java
new file mode 100644
index 0000000..7fd3de4
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/ipc/PythonTCPSocketProto.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.external.ipc;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.external.api.IExternalLangIPCProto;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class PythonTCPSocketProto extends AbstractPythonIPCProto
+ implements org.apache.asterix.external.api.IExternalLangIPCProto {
+
+ private final ExternalFunctionResultRouter router;
+ private final Process proc;
+
+ public PythonTCPSocketProto(OutputStream sockOut, ExternalFunctionResultRouter router, Process pythonProc) {
+ super(sockOut);
+ this.router = router;
+ this.proc = pythonProc;
+ }
+
+ @Override
+ public void start() {
+ Pair<Long, Pair<ByteBuffer, Exception>> keyAndBufferBox = router.insertRoute(recvBuffer);
+ this.routeId = keyAndBufferBox.getFirst();
+ this.bufferBox = keyAndBufferBox.getSecond();
+ }
+
+ @Override
+ public void quit() throws HyracksDataException {
+ messageBuilder.quit();
+ router.removeRoute(routeId);
+ }
+
+ @Override
+ public void receiveMsg() throws IOException, AsterixException {
+ Exception except;
+ try {
+ synchronized (bufferBox) {
+ while ((bufferBox.getFirst().limit() == 0 || bufferBox.getSecond() != null) && proc.isAlive()) {
+ bufferBox.wait(100);
+ }
+ }
+ except = router.getAndRemoveException(routeId);
+ if (!proc.isAlive()) {
+ except = new IOException("Python process exited with code: " + proc.exitValue());
+ }
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ throw new AsterixException(ErrorCode.EXTERNAL_UDF_EXCEPTION, e);
+ }
+ if (except != null) {
+ throw new AsterixException(except);
+ }
+ if (bufferBox.getFirst() != recvBuffer) {
+ recvBuffer = bufferBox.getFirst();
+ }
+ messageBuilder.readHead(recvBuffer);
+ if (messageBuilder.type == MessageType.ERROR) {
+ unpackerInput.reset(recvBuffer.array(), recvBuffer.position() + recvBuffer.arrayOffset(),
+ recvBuffer.remaining());
+ unpacker.reset(unpackerInput);
+ throw new AsterixException(unpacker.unpackString());
+ }
+ }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/AbstractLibrarySocketEvaluator.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/AbstractLibrarySocketEvaluator.java
new file mode 100644
index 0000000..6fcfdcf
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/AbstractLibrarySocketEvaluator.java
@@ -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.
+ */
+package org.apache.asterix.external.library;
+
+import static org.apache.asterix.common.exceptions.ErrorCode.EXTERNAL_UDF_EXCEPTION;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.external.api.IExternalLangIPCProto;
+import org.apache.asterix.external.api.ILibraryEvaluator;
+import org.apache.asterix.om.functions.IExternalFunctionInfo;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.api.dataflow.TaskAttemptId;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.api.exceptions.Warning;
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.std.base.AbstractStateObject;
+
+public abstract class AbstractLibrarySocketEvaluator extends AbstractStateObject implements ILibraryEvaluator {
+
+ protected IExternalLangIPCProto proto;
+ protected TaskAttemptId task;
+ protected IWarningCollector warningCollector;
+ protected SourceLocation sourceLoc;
+
+ public AbstractLibrarySocketEvaluator(JobId jobId, PythonLibraryEvaluatorId evaluatorId, TaskAttemptId task,
+ IWarningCollector warningCollector, SourceLocation sourceLoc) {
+ super(jobId, evaluatorId);
+ this.task = task;
+ this.warningCollector = warningCollector;
+ this.sourceLoc = sourceLoc;
+ }
+
+ @Override
+ public long initialize(IExternalFunctionInfo finfo) throws IOException, AsterixException {
+ List<String> externalIdents = finfo.getExternalIdentifier();
+ String packageModule = externalIdents.get(0);
+ String clazz;
+ String fn;
+ String externalIdent1 = externalIdents.get(1);
+ int idx = externalIdent1.lastIndexOf('.');
+ if (idx >= 0) {
+ clazz = externalIdent1.substring(0, idx);
+ fn = externalIdent1.substring(idx + 1);
+ } else {
+ clazz = null;
+ fn = externalIdent1;
+ }
+ return proto.init(packageModule, clazz, fn);
+ }
+
+ @Override
+ public ByteBuffer call(long id, IAType[] argTypes, IValueReference[] valueReferences, boolean nullCall)
+ throws IOException {
+ ByteBuffer ret = null;
+ try {
+ ret = proto.call(id, argTypes, valueReferences, nullCall);
+ } catch (AsterixException e) {
+ if (warningCollector.shouldWarn()) {
+ warningCollector.warn(Warning.of(sourceLoc, EXTERNAL_UDF_EXCEPTION, e.getMessage()));
+ }
+ }
+ return ret;
+ }
+
+ @Override
+ public ByteBuffer callMulti(long id, ArrayBackedValueStorage arguments, int numTuples) throws IOException {
+ ByteBuffer ret = null;
+ try {
+ ret = proto.callMulti(id, arguments, numTuples);
+ } catch (AsterixException e) {
+ if (warningCollector.shouldWarn()) {
+ warningCollector.warn(Warning.of(sourceLoc, EXTERNAL_UDF_EXCEPTION, e.getMessage()));
+ }
+ }
+ return ret;
+ }
+}
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/library/ExternalScalarPythonFunctionEvaluator.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalScalarPythonFunctionEvaluator.java
index 94a4dd2..fb8d761 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalScalarPythonFunctionEvaluator.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalScalarPythonFunctionEvaluator.java
@@ -28,6 +28,7 @@
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.external.api.ILibraryEvaluator;
import org.apache.asterix.external.library.msgpack.MessageUnpackerToADM;
import org.apache.asterix.external.util.ExternalDataUtils;
import org.apache.asterix.om.functions.IExternalFunctionInfo;
@@ -49,7 +50,7 @@
class ExternalScalarPythonFunctionEvaluator extends ExternalScalarFunctionEvaluator {
- private final PythonLibraryEvaluator libraryEvaluator;
+ private final ILibraryEvaluator libraryEvaluator;
private final ArrayBackedValueStorage resultBuffer = new ArrayBackedValueStorage();
private final ByteBuffer argHolder;
@@ -115,7 +116,7 @@
return;
}
try {
- ByteBuffer res = libraryEvaluator.callPython(fnId, argTypes, argValues, nullCall);
+ ByteBuffer res = libraryEvaluator.call(fnId, argTypes, argValues, nullCall);
resultBuffer.reset();
wrap(res, resultBuffer.getDataOutput());
} catch (Exception e) {
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/PythonLibraryDomainSocketEvaluator.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/PythonLibraryDomainSocketEvaluator.java
new file mode 100644
index 0000000..056aa9a
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/PythonLibraryDomainSocketEvaluator.java
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.library;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandle;
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.MethodType;
+import java.lang.invoke.VarHandle;
+import java.net.ProtocolFamily;
+import java.net.SocketAddress;
+import java.net.StandardProtocolFamily;
+import java.nio.channels.Channels;
+import java.nio.channels.SocketChannel;
+import java.nio.file.Path;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.library.ILibraryManager;
+import org.apache.asterix.external.ipc.PythonDomainSocketProto;
+import org.apache.asterix.om.functions.IExternalFunctionInfo;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.TaskAttemptId;
+import org.apache.hyracks.api.exceptions.ErrorCode;
+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.job.JobId;
+import org.apache.hyracks.control.common.controllers.NCConfig;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+public class PythonLibraryDomainSocketEvaluator extends AbstractLibrarySocketEvaluator {
+
+ private final ILibraryManager libMgr;
+ private final Path sockPath;
+ SocketChannel chan;
+ ProcessHandle pid;
+ private static final Logger LOGGER = LogManager.getLogger(ExternalLibraryManager.class);
+
+ public PythonLibraryDomainSocketEvaluator(JobId jobId, PythonLibraryEvaluatorId evaluatorId, ILibraryManager libMgr,
+ TaskAttemptId task, IWarningCollector warningCollector, SourceLocation sourceLoc, Path sockPath) {
+ super(jobId, evaluatorId, task, warningCollector, sourceLoc);
+ this.libMgr = libMgr;
+ this.sockPath = sockPath;
+ }
+
+ public void start() throws IOException, AsterixException {
+ PythonLibraryEvaluatorId fnId = (PythonLibraryEvaluatorId) id;
+ PythonLibrary library =
+ (PythonLibrary) libMgr.getLibrary(fnId.getLibraryDataverseName(), fnId.getLibraryName());
+ String wd = library.getFile().getAbsolutePath();
+ MethodHandles.Lookup lookup = MethodHandles.lookup();
+ SocketAddress sockAddr;
+ try {
+ VarHandle sockEnum = lookup.in(StandardProtocolFamily.class)
+ .findStaticVarHandle(StandardProtocolFamily.class, "UNIX", StandardProtocolFamily.class);
+ Class domainSock = Class.forName("java.net.UnixDomainSocketAddress");
+ MethodType unixDomainSockAddrType = MethodType.methodType(domainSock, Path.class);
+ MethodHandle unixDomainSockAddr = lookup.findStatic(domainSock, "of", unixDomainSockAddrType);
+ MethodType sockChanMethodType = MethodType.methodType(SocketChannel.class, ProtocolFamily.class);
+ MethodHandle sockChanOpen = lookup.findStatic(SocketChannel.class, "open", sockChanMethodType);
+ sockAddr = ((SocketAddress) unixDomainSockAddr.invoke(sockPath));
+ chan = (SocketChannel) sockChanOpen.invoke(sockEnum.get());
+ } catch (Throwable e) {
+ throw HyracksDataException.create(ErrorCode.LOCAL_NETWORK_ERROR, e);
+ }
+ chan.connect(sockAddr);
+ proto = new PythonDomainSocketProto(Channels.newOutputStream(chan), chan, wd);
+ proto.start();
+ proto.helo();
+ this.pid = ((PythonDomainSocketProto) proto).getPid();
+ }
+
+ @Override
+ public void deallocate() {
+ try {
+ if (proto != null) {
+ proto.quit();
+ }
+ if (chan != null) {
+ chan.close();
+ }
+ } catch (IOException e) {
+ LOGGER.error("Caught exception exiting Python UDF:", e);
+ }
+ if (pid != null && pid.isAlive()) {
+ LOGGER.error("Python UDF " + pid.pid() + " did not exit as expected.");
+ }
+ }
+
+ static PythonLibraryDomainSocketEvaluator getInstance(IExternalFunctionInfo finfo, ILibraryManager libMgr,
+ IHyracksTaskContext ctx, IWarningCollector warningCollector, SourceLocation sourceLoc)
+ throws IOException, AsterixException {
+ PythonLibraryEvaluatorId evaluatorId = new PythonLibraryEvaluatorId(finfo.getLibraryDataverseName(),
+ finfo.getLibraryName(), Thread.currentThread());
+ PythonLibraryDomainSocketEvaluator evaluator =
+ (PythonLibraryDomainSocketEvaluator) ctx.getStateObject(evaluatorId);
+ if (evaluator == null) {
+ Path sockPath = Path.of(ctx.getJobletContext().getServiceContext().getAppConfig()
+ .getString(NCConfig.Option.PYTHON_DS_PATH));
+ evaluator = new PythonLibraryDomainSocketEvaluator(ctx.getJobletContext().getJobId(), evaluatorId, libMgr,
+ ctx.getTaskAttemptId(), warningCollector, sourceLoc, sockPath);
+ ctx.getJobletContext().registerDeallocatable(evaluator);
+ evaluator.start();
+ ctx.setStateObject(evaluator);
+ }
+ return evaluator;
+ }
+
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/PythonLibraryEvaluator.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/PythonLibraryEvaluator.java
deleted file mode 100644
index f82b30d..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/PythonLibraryEvaluator.java
+++ /dev/null
@@ -1,209 +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.external.library;
-
-import static org.apache.asterix.common.exceptions.ErrorCode.EXTERNAL_UDF_EXCEPTION;
-import static org.msgpack.core.MessagePack.Code.ARRAY16;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.InetAddress;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.common.library.ILibraryManager;
-import org.apache.asterix.external.ipc.ExternalFunctionResultRouter;
-import org.apache.asterix.external.ipc.PythonIPCProto;
-import org.apache.asterix.external.library.msgpack.MessagePackUtils;
-import org.apache.asterix.om.functions.IExternalFunctionInfo;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.EnumDeserializer;
-import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.types.TypeTagUtil;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.TaskAttemptId;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.exceptions.IWarningCollector;
-import org.apache.hyracks.api.exceptions.SourceLocation;
-import org.apache.hyracks.api.exceptions.Warning;
-import org.apache.hyracks.api.job.JobId;
-import org.apache.hyracks.api.resources.IDeallocatable;
-import org.apache.hyracks.data.std.api.IValueReference;
-import org.apache.hyracks.data.std.primitive.TaggedValuePointable;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-import org.apache.hyracks.dataflow.std.base.AbstractStateObject;
-import org.apache.hyracks.ipc.impl.IPCSystem;
-
-public class PythonLibraryEvaluator extends AbstractStateObject implements IDeallocatable {
-
- public static final String ENTRYPOINT = "entrypoint.py";
- public static final String SITE_PACKAGES = "site-packages";
-
- private Process p;
- private ILibraryManager libMgr;
- private File pythonHome;
- private PythonIPCProto proto;
- private ExternalFunctionResultRouter router;
- private IPCSystem ipcSys;
- private String sitePkgs;
- private List<String> pythonArgs;
- private Map<String, String> pythonEnv;
- private TaskAttemptId task;
- private IWarningCollector warningCollector;
- private SourceLocation sourceLoc;
-
- public PythonLibraryEvaluator(JobId jobId, PythonLibraryEvaluatorId evaluatorId, ILibraryManager libMgr,
- File pythonHome, String sitePkgs, List<String> pythonArgs, Map<String, String> pythonEnv,
- ExternalFunctionResultRouter router, IPCSystem ipcSys, TaskAttemptId task,
- IWarningCollector warningCollector, SourceLocation sourceLoc) {
- super(jobId, evaluatorId);
- this.libMgr = libMgr;
- this.pythonHome = pythonHome;
- this.sitePkgs = sitePkgs;
- this.pythonArgs = pythonArgs;
- this.pythonEnv = pythonEnv;
- this.router = router;
- this.task = task;
- this.ipcSys = ipcSys;
- this.warningCollector = warningCollector;
- this.sourceLoc = sourceLoc;
- }
-
- private void initialize() throws IOException, AsterixException {
- PythonLibraryEvaluatorId fnId = (PythonLibraryEvaluatorId) id;
- PythonLibrary library =
- (PythonLibrary) libMgr.getLibrary(fnId.getLibraryDataverseName(), fnId.getLibraryName());
- String wd = library.getFile().getAbsolutePath();
- int port = ipcSys.getSocketAddress().getPort();
- List<String> args = new ArrayList<>();
- args.add(pythonHome.getAbsolutePath());
- args.addAll(pythonArgs);
- args.add(ENTRYPOINT);
- args.add(InetAddress.getLoopbackAddress().getHostAddress());
- args.add(Integer.toString(port));
- args.add(sitePkgs);
- ProcessBuilder pb = new ProcessBuilder(args.toArray(new String[0]));
- pb.environment().putAll(pythonEnv);
- pb.directory(new File(wd));
- p = pb.start();
- proto = new PythonIPCProto(p.getOutputStream(), router, p);
- proto.start();
- proto.helo();
- }
-
- public long initialize(IExternalFunctionInfo finfo) throws IOException, AsterixException {
- List<String> externalIdents = finfo.getExternalIdentifier();
- String packageModule = externalIdents.get(0);
- String clazz;
- String fn;
- String externalIdent1 = externalIdents.get(1);
- int idx = externalIdent1.lastIndexOf('.');
- if (idx >= 0) {
- clazz = externalIdent1.substring(0, idx);
- fn = externalIdent1.substring(idx + 1);
- } else {
- clazz = null;
- fn = externalIdent1;
- }
- return proto.init(packageModule, clazz, fn);
- }
-
- public ByteBuffer callPython(long id, IAType[] argTypes, IValueReference[] valueReferences, boolean nullCall)
- throws IOException {
- ByteBuffer ret = null;
- try {
- ret = proto.call(id, argTypes, valueReferences, nullCall);
- } catch (AsterixException e) {
- if (warningCollector.shouldWarn()) {
- warningCollector.warn(Warning.of(sourceLoc, EXTERNAL_UDF_EXCEPTION, e.getMessage()));
- }
- }
- return ret;
- }
-
- public ByteBuffer callPythonMulti(long id, ArrayBackedValueStorage arguments, int numTuples) throws IOException {
- ByteBuffer ret = null;
- try {
- ret = proto.callMulti(id, arguments, numTuples);
- } catch (AsterixException e) {
- if (warningCollector.shouldWarn()) {
- warningCollector.warn(Warning.of(sourceLoc, EXTERNAL_UDF_EXCEPTION, e.getMessage()));
- }
- }
- return ret;
- }
-
- @Override
- public void deallocate() {
- if (p != null) {
- boolean dead = false;
- try {
- p.destroy();
- dead = p.waitFor(100, TimeUnit.MILLISECONDS);
- } catch (InterruptedException e) {
- //gonna kill it anyway
- }
- if (!dead) {
- p.destroyForcibly();
- }
- }
- router.removeRoute(proto.getRouteId());
- }
-
- public static ATypeTag peekArgument(IAType type, IValueReference valueReference) throws HyracksDataException {
- ATypeTag tag = type.getTypeTag();
- if (tag == ATypeTag.ANY) {
- TaggedValuePointable pointy = TaggedValuePointable.FACTORY.createPointable();
- pointy.set(valueReference);
- ATypeTag rtTypeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(pointy.getTag());
- IAType rtType = TypeTagUtil.getBuiltinTypeByTag(rtTypeTag);
- return MessagePackUtils.peekUnknown(rtType);
- } else {
- return MessagePackUtils.peekUnknown(type);
- }
- }
-
- public static void setVoidArgument(ArrayBackedValueStorage argHolder) throws IOException {
- argHolder.getDataOutput().writeByte(ARRAY16);
- argHolder.getDataOutput().writeShort((short) 0);
- }
-
- public static PythonLibraryEvaluator getInstance(IExternalFunctionInfo finfo, ILibraryManager libMgr,
- ExternalFunctionResultRouter router, IPCSystem ipcSys, File pythonHome, IHyracksTaskContext ctx,
- String sitePkgs, List<String> pythonArgs, Map<String, String> pythonEnv, IWarningCollector warningCollector,
- SourceLocation sourceLoc) throws IOException, AsterixException {
- PythonLibraryEvaluatorId evaluatorId = new PythonLibraryEvaluatorId(finfo.getLibraryDataverseName(),
- finfo.getLibraryName(), Thread.currentThread());
- PythonLibraryEvaluator evaluator = (PythonLibraryEvaluator) ctx.getStateObject(evaluatorId);
- if (evaluator == null) {
- evaluator = new PythonLibraryEvaluator(ctx.getJobletContext().getJobId(), evaluatorId, libMgr, pythonHome,
- sitePkgs, pythonArgs, pythonEnv, router, ipcSys, ctx.getTaskAttemptId(), warningCollector,
- sourceLoc);
- ctx.getJobletContext().registerDeallocatable(evaluator);
- evaluator.initialize();
- ctx.setStateObject(evaluator);
- }
- return evaluator;
- }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/PythonLibraryEvaluatorFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/PythonLibraryEvaluatorFactory.java
index 06c9bc9..63a6ec3 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/PythonLibraryEvaluatorFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/PythonLibraryEvaluatorFactory.java
@@ -18,10 +18,12 @@
*/
package org.apache.asterix.external.library;
-import static org.apache.asterix.external.library.PythonLibraryEvaluator.SITE_PACKAGES;
+import static org.apache.asterix.external.library.PythonLibraryTCPSocketEvaluator.SITE_PACKAGES;
import java.io.File;
import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
@@ -31,8 +33,10 @@
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.common.library.ILibraryManager;
+import org.apache.asterix.external.api.ILibraryEvaluator;
import org.apache.asterix.external.ipc.ExternalFunctionResultRouter;
import org.apache.asterix.om.functions.IExternalFunctionInfo;
+import org.apache.commons.lang3.SystemUtils;
import org.apache.hyracks.api.config.IApplicationConfig;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.exceptions.SourceLocation;
@@ -40,83 +44,116 @@
import org.apache.hyracks.ipc.impl.IPCSystem;
public class PythonLibraryEvaluatorFactory {
- private final ILibraryManager libraryManager;
- private final IPCSystem ipcSys;
- private final File pythonPath;
- private final IHyracksTaskContext ctx;
- private final ExternalFunctionResultRouter router;
- private final String sitePackagesPath;
- private final List<String> pythonArgs;
- private final Map<String, String> pythonEnv;
+
+ private ILibraryManager libraryManager;
+ private IPCSystem ipcSys;
+ private File pythonPath;
+ private IHyracksTaskContext ctx;
+ private ExternalFunctionResultRouter router;
+ private String sitePackagesPath;
+ private List<String> pythonArgs;
+ private Map<String, String> pythonEnv;
+
+ private boolean domainSockEnable;
public PythonLibraryEvaluatorFactory(IHyracksTaskContext ctx) throws AsterixException {
this.ctx = ctx;
+ String dsPath =
+ ctx.getJobletContext().getServiceContext().getAppConfig().getString(NCConfig.Option.PYTHON_DS_PATH);
+ config(dsPath == null ? null : Path.of(dsPath));
libraryManager = ((INcApplicationContext) ctx.getJobletContext().getServiceContext().getApplicationContext())
.getLibraryManager();
- router = libraryManager.getRouter();
- ipcSys = libraryManager.getIPCI();
- IApplicationConfig appCfg = ctx.getJobletContext().getServiceContext().getAppConfig();
- String pythonPathCmd = appCfg.getString(NCConfig.Option.PYTHON_CMD);
- boolean findPython = appCfg.getBoolean(NCConfig.Option.PYTHON_CMD_AUTOLOCATE);
- pythonArgs = new ArrayList<>();
- if (pythonPathCmd == null) {
- if (findPython) {
- //if absolute path to interpreter is not specified, try to use environmental python
- pythonPathCmd = "/usr/bin/env";
- pythonArgs.add("python3");
- } else {
- throw AsterixException.create(ErrorCode.EXTERNAL_UDF_EXCEPTION, "Python interpreter not specified, and "
- + NCConfig.Option.PYTHON_CMD_AUTOLOCATE.ini() + " is false");
- }
- }
- pythonEnv = new HashMap<>();
- String[] envRaw = appCfg.getStringArray((NCConfig.Option.PYTHON_ENV));
- if (envRaw != null) {
- for (String rawEnvArg : envRaw) {
- //TODO: i think equals is shared among all unixes and windows. but it needs verification
- if (rawEnvArg.length() < 1) {
- continue;
- }
- String[] rawArgSplit = rawEnvArg.split("(?<!\\\\)=", 2);
- if (rawArgSplit.length < 2) {
+ if (!domainSockEnable) {
+ router = libraryManager.getRouter();
+ ipcSys = libraryManager.getIPCI();
+ IApplicationConfig appCfg = ctx.getJobletContext().getServiceContext().getAppConfig();
+ String pythonPathCmd = appCfg.getString(NCConfig.Option.PYTHON_CMD);
+ boolean findPython = appCfg.getBoolean(NCConfig.Option.PYTHON_CMD_AUTOLOCATE);
+ pythonArgs = new ArrayList<>();
+ if (pythonPathCmd == null) {
+ if (findPython) {
+ //if absolute path to interpreter is not specified, try to use environmental python
+ pythonPathCmd = "/usr/bin/env";
+ pythonArgs.add("python3");
+ } else {
throw AsterixException.create(ErrorCode.EXTERNAL_UDF_EXCEPTION,
- "Invalid environment variable format detected.");
- }
- pythonEnv.put(rawArgSplit[0], rawArgSplit[1]);
- }
- }
- pythonPath = new File(pythonPathCmd);
- List<String> sitePkgs = new ArrayList<>();
- sitePkgs.add(SITE_PACKAGES);
- String[] addlSitePackages = appCfg.getStringArray((NCConfig.Option.PYTHON_ADDITIONAL_PACKAGES));
- for (String sitePkg : addlSitePackages) {
- if (sitePkg.length() > 0) {
- sitePkgs.add(sitePkg);
- }
- }
- if (appCfg.getBoolean(NCConfig.Option.PYTHON_USE_BUNDLED_MSGPACK)) {
- sitePkgs.add("ipc" + File.separator + SITE_PACKAGES + File.separator);
- }
- String[] pythonArgsRaw = appCfg.getStringArray(NCConfig.Option.PYTHON_ARGS);
- if (pythonArgsRaw != null) {
- for (String arg : pythonArgsRaw) {
- if (arg.length() > 0) {
- pythonArgs.add(arg);
+ "Python interpreter not specified or domain socket not found, and "
+ + NCConfig.Option.PYTHON_CMD_AUTOLOCATE.ini() + " is false");
}
}
+ pythonEnv = new HashMap<>();
+ String[] envRaw = appCfg.getStringArray((NCConfig.Option.PYTHON_ENV));
+ if (envRaw != null) {
+ for (String rawEnvArg : envRaw) {
+ //TODO: i think equals is shared among all unixes and windows. but it needs verification
+ if (rawEnvArg.length() < 1) {
+ continue;
+ }
+ String[] rawArgSplit = rawEnvArg.split("(?<!\\\\)=", 2);
+ if (rawArgSplit.length < 2) {
+ throw AsterixException.create(ErrorCode.EXTERNAL_UDF_EXCEPTION,
+ "Invalid environment variable format detected.");
+ }
+ pythonEnv.put(rawArgSplit[0], rawArgSplit[1]);
+ }
+ }
+ pythonPath = new File(pythonPathCmd);
+ List<String> sitePkgs = new ArrayList<>();
+ sitePkgs.add(SITE_PACKAGES);
+ String[] addlSitePackages = appCfg.getStringArray((NCConfig.Option.PYTHON_ADDITIONAL_PACKAGES));
+ for (String sitePkg : addlSitePackages) {
+ if (sitePkg.length() > 0) {
+ sitePkgs.add(sitePkg);
+ }
+ }
+ if (appCfg.getBoolean(NCConfig.Option.PYTHON_USE_BUNDLED_MSGPACK)) {
+ sitePkgs.add("ipc" + File.separator + SITE_PACKAGES + File.separator);
+ }
+ String[] pythonArgsRaw = appCfg.getStringArray(NCConfig.Option.PYTHON_ARGS);
+ if (pythonArgsRaw != null) {
+ for (String arg : pythonArgsRaw) {
+ if (arg.length() > 0) {
+ pythonArgs.add(arg);
+ }
+ }
+ }
+ StringBuilder sitePackagesPathBuilder = new StringBuilder();
+ for (int i = 0; i < sitePkgs.size() - 1; i++) {
+ sitePackagesPathBuilder.append(sitePkgs.get(i));
+ sitePackagesPathBuilder.append(File.pathSeparator);
+ }
+ sitePackagesPathBuilder.append(sitePkgs.get(sitePkgs.size() - 1));
+ sitePackagesPath = sitePackagesPathBuilder.toString();
}
- StringBuilder sitePackagesPathBuilder = new StringBuilder();
- for (int i = 0; i < sitePkgs.size() - 1; i++) {
- sitePackagesPathBuilder.append(sitePkgs.get(i));
- sitePackagesPathBuilder.append(File.pathSeparator);
- }
- sitePackagesPathBuilder.append(sitePkgs.get(sitePkgs.size() - 1));
- sitePackagesPath = sitePackagesPathBuilder.toString();
}
- public PythonLibraryEvaluator getEvaluator(IExternalFunctionInfo fnInfo, SourceLocation sourceLoc)
+ public ILibraryEvaluator getEvaluator(IExternalFunctionInfo fnInfo, SourceLocation sourceLoc)
throws IOException, AsterixException {
- return PythonLibraryEvaluator.getInstance(fnInfo, libraryManager, router, ipcSys, pythonPath, ctx,
- sitePackagesPath, pythonArgs, pythonEnv, ctx.getWarningCollector(), sourceLoc);
+ if (domainSockEnable) {
+ return PythonLibraryDomainSocketEvaluator.getInstance(fnInfo, libraryManager, ctx,
+ ctx.getWarningCollector(), sourceLoc);
+ } else {
+ return PythonLibraryTCPSocketEvaluator.getInstance(fnInfo, libraryManager, router, ipcSys, pythonPath, ctx,
+ sitePackagesPath, pythonArgs, pythonEnv, ctx.getWarningCollector(), sourceLoc);
+ }
+ }
+
+ private void config(Path sockPath) throws AsterixException {
+ if (sockPath == null) {
+ domainSockEnable = false;
+ return;
+ }
+ Runtime rt = Runtime.getRuntime();
+ if (rt.version().feature() >= 17 && SystemUtils.IS_OS_LINUX) {
+ if (Files.exists(sockPath)) {
+ domainSockEnable = true;
+ } else {
+ throw AsterixException.create(ErrorCode.EXTERNAL_UDF_EXCEPTION,
+ "Domain socket was not found at specified path");
+ }
+ } else {
+ throw AsterixException.create(ErrorCode.EXTERNAL_UDF_EXCEPTION,
+ "Domain socket path specified, but Java version is below 17 or OS is not Linux");
+ }
}
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/PythonLibraryTCPSocketEvaluator.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/PythonLibraryTCPSocketEvaluator.java
new file mode 100644
index 0000000..385d738
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/PythonLibraryTCPSocketEvaluator.java
@@ -0,0 +1,127 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.library;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.library.ILibraryManager;
+import org.apache.asterix.external.ipc.ExternalFunctionResultRouter;
+import org.apache.asterix.external.ipc.PythonTCPSocketProto;
+import org.apache.asterix.om.functions.IExternalFunctionInfo;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.TaskAttemptId;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.ipc.impl.IPCSystem;
+
+public class PythonLibraryTCPSocketEvaluator extends AbstractLibrarySocketEvaluator {
+
+ public static final String ENTRYPOINT = "entrypoint.py";
+ public static final String SITE_PACKAGES = "site-packages";
+
+ private Process p;
+ private ILibraryManager libMgr;
+ private File pythonHome;
+ private ExternalFunctionResultRouter router;
+ private IPCSystem ipcSys;
+ private String sitePkgs;
+ private List<String> pythonArgs;
+ private Map<String, String> pythonEnv;
+
+ public PythonLibraryTCPSocketEvaluator(JobId jobId, PythonLibraryEvaluatorId evaluatorId, ILibraryManager libMgr,
+ File pythonHome, String sitePkgs, List<String> pythonArgs, Map<String, String> pythonEnv,
+ ExternalFunctionResultRouter router, IPCSystem ipcSys, TaskAttemptId task,
+ IWarningCollector warningCollector, SourceLocation sourceLoc) {
+ super(jobId, evaluatorId, task, warningCollector, sourceLoc);
+ this.libMgr = libMgr;
+ this.pythonHome = pythonHome;
+ this.sitePkgs = sitePkgs;
+ this.pythonArgs = pythonArgs;
+ this.pythonEnv = pythonEnv;
+ this.router = router;
+ this.ipcSys = ipcSys;
+ }
+
+ @Override
+ public void start() throws IOException, AsterixException {
+ PythonLibraryEvaluatorId fnId = (PythonLibraryEvaluatorId) id;
+ PythonLibrary library =
+ (PythonLibrary) libMgr.getLibrary(fnId.getLibraryDataverseName(), fnId.getLibraryName());
+ String wd = library.getFile().getAbsolutePath();
+ int port = ipcSys.getSocketAddress().getPort();
+ List<String> args = new ArrayList<>();
+ args.add(pythonHome.getAbsolutePath());
+ args.addAll(pythonArgs);
+ args.add(ENTRYPOINT);
+ args.add(InetAddress.getLoopbackAddress().getHostAddress());
+ args.add(Integer.toString(port));
+ args.add(sitePkgs);
+ ProcessBuilder pb = new ProcessBuilder(args.toArray(new String[0]));
+ pb.environment().putAll(pythonEnv);
+ pb.directory(new File(wd));
+ p = pb.start();
+ proto = new PythonTCPSocketProto(p.getOutputStream(), router, p);
+ proto.start();
+ proto.helo();
+ }
+
+ @Override
+ public void deallocate() {
+ if (p != null) {
+ boolean dead = false;
+ try {
+ p.destroy();
+ dead = p.waitFor(100, TimeUnit.MILLISECONDS);
+ } catch (InterruptedException e) {
+ //gonna kill it anyway
+ }
+ if (!dead) {
+ p.destroyForcibly();
+ }
+ }
+ router.removeRoute(proto.getRouteId());
+ }
+
+ static PythonLibraryTCPSocketEvaluator getInstance(IExternalFunctionInfo finfo, ILibraryManager libMgr,
+ ExternalFunctionResultRouter router, IPCSystem ipcSys, File pythonHome, IHyracksTaskContext ctx,
+ String sitePkgs, List<String> pythonArgs, Map<String, String> pythonEnv, IWarningCollector warningCollector,
+ SourceLocation sourceLoc) throws IOException, AsterixException {
+ PythonLibraryEvaluatorId evaluatorId = new PythonLibraryEvaluatorId(finfo.getLibraryDataverseName(),
+ finfo.getLibraryName(), Thread.currentThread());
+ PythonLibraryTCPSocketEvaluator evaluator = (PythonLibraryTCPSocketEvaluator) ctx.getStateObject(evaluatorId);
+ if (evaluator == null) {
+ evaluator = new PythonLibraryTCPSocketEvaluator(ctx.getJobletContext().getJobId(), evaluatorId, libMgr,
+ pythonHome, sitePkgs, pythonArgs, pythonEnv, router, ipcSys, ctx.getTaskAttemptId(),
+ warningCollector, sourceLoc);
+ ctx.getJobletContext().registerDeallocatable(evaluator);
+ evaluator.start();
+ ctx.setStateObject(evaluator);
+ }
+ return evaluator;
+ }
+
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/msgpack/MsgPackAccessors.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/msgpack/MsgPackAccessors.java
index a90a183..6efbb6e 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/msgpack/MsgPackAccessors.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/msgpack/MsgPackAccessors.java
@@ -124,7 +124,7 @@
int s = pointable.getStartOffset();
int i = AInt32SerializerDeserializer.getInt(b, s + 1);
out.writeByte(INT32);
- out.writeByte(i);
+ out.writeInt(i);
return null;
}
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalAssignBatchRuntimeFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalAssignBatchRuntimeFactory.java
index 741dad2..5f8a3f0 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalAssignBatchRuntimeFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalAssignBatchRuntimeFactory.java
@@ -33,12 +33,13 @@
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.common.exceptions.RuntimeDataException;
-import org.apache.asterix.external.ipc.PythonIPCProto;
-import org.apache.asterix.external.library.PythonLibraryEvaluator;
+import org.apache.asterix.external.api.IExternalLangIPCProto;
+import org.apache.asterix.external.api.ILibraryEvaluator;
import org.apache.asterix.external.library.PythonLibraryEvaluatorFactory;
import org.apache.asterix.external.library.msgpack.MessageUnpackerToADM;
import org.apache.asterix.external.library.msgpack.MsgPackPointableVisitor;
import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.ExternalDataUtils;
import org.apache.asterix.om.functions.IExternalFunctionDescriptor;
import org.apache.asterix.om.pointables.PointableAllocator;
import org.apache.asterix.om.types.ATypeTag;
@@ -50,6 +51,7 @@
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.exceptions.Warning;
+import org.apache.hyracks.data.std.primitive.TaggedValuePointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
@@ -87,7 +89,7 @@
private ArrayBackedValueStorage outputWrapper;
private List<ArrayBackedValueStorage> argHolders;
ArrayTupleBuilder tupleBuilder;
- private List<Pair<Long, PythonLibraryEvaluator>> libraryEvaluators;
+ private List<Pair<Long, ILibraryEvaluator>> libraryEvaluators;
private ATypeTag[][] nullCalls;
private int[] numCalls;
private VoidPointable ref;
@@ -97,6 +99,7 @@
private MessageUnpackerToADM unpackerToADM;
private PointableAllocator pointableAllocator;
private MsgPackPointableVisitor pointableVisitor;
+ private TaggedValuePointable anyPointer;
@Override
public void open() throws HyracksDataException {
@@ -109,7 +112,7 @@
try {
PythonLibraryEvaluatorFactory evalFactory = new PythonLibraryEvaluatorFactory(ctx);
for (IExternalFunctionDescriptor fnDesc : fnDescs) {
- PythonLibraryEvaluator eval = evalFactory.getEvaluator(fnDesc.getFunctionInfo(), sourceLoc);
+ ILibraryEvaluator eval = evalFactory.getEvaluator(fnDesc.getFunctionInfo(), sourceLoc);
long id = eval.initialize(fnDesc.getFunctionInfo());
libraryEvaluators.add(new Pair<>(id, eval));
}
@@ -133,6 +136,7 @@
unpackerToADM = new MessageUnpackerToADM();
pointableAllocator = new PointableAllocator();
pointableVisitor = new MsgPackPointableVisitor();
+ anyPointer = TaggedValuePointable.FACTORY.createPointable();
}
private void resetBuffers(int numTuples, int[] numCalls) {
@@ -177,8 +181,12 @@
int numEntries = unpacker.unpackArrayHeader();
for (int j = 0; j < numEntries; j++) {
if (ctx.getWarningCollector().shouldWarn()) {
- ctx.getWarningCollector().warn(Warning.of(sourceLoc,
- ErrorCode.EXTERNAL_UDF_EXCEPTION, unpacker.unpackString()));
+ //TODO: in domain socket mode, a NUL can appear at the end of the stacktrace strings.
+ // this should probably not happen but warnings with control characters should
+ // also be properly escaped
+ ctx.getWarningCollector()
+ .warn(Warning.of(sourceLoc, ErrorCode.EXTERNAL_UDF_EXCEPTION,
+ unpacker.unpackString().replace('\0', ' ')));
}
}
} catch (MessagePackException e) {
@@ -211,8 +219,8 @@
for (int colIdx = 0; colIdx < cols.length; colIdx++) {
ref.set(buffer.array(), tRef.getFieldStart(cols[colIdx]),
tRef.getFieldLength(cols[colIdx]));
- ATypeTag argumentPresence = PythonLibraryEvaluator
- .peekArgument(fnDescs[func].getArgumentTypes()[colIdx], ref);
+ ATypeTag argumentPresence = ExternalDataUtils
+ .peekArgument(fnDescs[func].getArgumentTypes()[colIdx], ref, anyPointer);
argumentStatus = handleNullMatrix(func, t, argumentPresence, argumentStatus);
}
}
@@ -224,7 +232,7 @@
for (int colIdx = 0; colIdx < cols.length; colIdx++) {
ref.set(buffer.array(), tRef.getFieldStart(cols[colIdx]),
tRef.getFieldLength(cols[colIdx]));
- PythonIPCProto.visitValueRef(fnDescs[func].getArgumentTypes()[colIdx],
+ IExternalLangIPCProto.visitValueRef(fnDescs[func].getArgumentTypes()[colIdx],
argHolders.get(func).getDataOutput(), ref, pointableAllocator,
pointableVisitor, fnDescs[func].getFunctionInfo().getNullCall());
}
@@ -232,21 +240,25 @@
numCalls[func]--;
}
if (cols.length == 0) {
- PythonLibraryEvaluator.setVoidArgument(argHolders.get(func));
+ ExternalDataUtils.setVoidArgument(argHolders.get(func));
}
}
}
//TODO: maybe this could be done in parallel for each unique library evaluator?
for (int argHolderIdx = 0; argHolderIdx < argHolders.size(); argHolderIdx++) {
- Pair<Long, PythonLibraryEvaluator> fnEval = libraryEvaluators.get(argHolderIdx);
- ByteBuffer columnResult = fnEval.getSecond().callPythonMulti(fnEval.getFirst(),
+ Pair<Long, ILibraryEvaluator> fnEval = libraryEvaluators.get(argHolderIdx);
+ ByteBuffer columnResult = fnEval.getSecond().callMulti(fnEval.getFirst(),
argHolders.get(argHolderIdx), numCalls[argHolderIdx]);
if (columnResult != null) {
Pair<ByteBuffer, Counter> resultholder = batchResults.get(argHolderIdx);
- if (resultholder.getFirst().capacity() < columnResult.capacity()) {
- ByteBuffer realloc = ctx.reallocateFrame(resultholder.getFirst(),
- columnResult.capacity() * 2, false);
+ if (resultholder.getFirst().capacity() < columnResult.remaining()) {
+ ByteBuffer realloc =
+ ctx.reallocateFrame(resultholder.getFirst(),
+ ctx.getInitialFrameSize()
+ * ((columnResult.remaining() / ctx.getInitialFrameSize()) + 1),
+ false);
+ realloc.limit(columnResult.limit());
resultholder.setFirst(realloc);
}
ByteBuffer resultBuf = resultholder.getFirst();
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalScanOperatorDescriptor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalScanOperatorDescriptor.java
index 82b8113..0250ba8 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalScanOperatorDescriptor.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalScanOperatorDescriptor.java
@@ -27,6 +27,7 @@
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.job.JobSpecification;
import org.apache.hyracks.api.job.profiling.IOperatorStats;
+import org.apache.hyracks.api.job.profiling.NoOpOperatorStats;
import org.apache.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
import org.apache.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
import org.apache.hyracks.storage.am.common.api.ITupleFilter;
@@ -66,23 +67,18 @@
return new AbstractUnaryOutputSourceOperatorNodePushable() {
- private IOperatorStats stats;
+ private IOperatorStats stats = NoOpOperatorStats.INSTANCE;
@Override
public void initialize() throws HyracksDataException {
IDataSourceAdapter adapter;
- if (ctx.getStatsCollector() != null) {
- stats = ctx.getStatsCollector().getOrAddOperatorStats(getDisplayName());
- }
try {
writer.open();
ITupleFilter tupleFilter =
tupleFilterFactory != null ? tupleFilterFactory.createTupleFilter(ctx) : null;
adapter = adapterFactory.createAdapter(ctx, partition);
adapter.start(partition, writer, tupleFilter, outputLimit);
- if (stats != null) {
- stats.getTupleCounter().update(adapter.getProcessedTuples());
- }
+ stats.getInputTupleCounter().update(adapter.getProcessedTuples());
} catch (Exception e) {
writer.fail();
throw HyracksDataException.create(e);
@@ -90,6 +86,11 @@
writer.close();
}
}
+
+ @Override
+ public void setOperatorStats(IOperatorStats stats) {
+ this.stats = stats;
+ }
};
}
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..479679e 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() {
@@ -333,107 +334,4 @@
*/
public static final Set<String> VALID_TIME_ZONES = Set.of(TimeZone.getAvailableIDs());
}
-
- public static class AwsS3 {
- private AwsS3() {
- throw new AssertionError("do not instantiate");
- }
-
- public static final String REGION_FIELD_NAME = "region";
- public static final String ACCESS_KEY_ID_FIELD_NAME = "accessKeyId";
- public static final String SECRET_ACCESS_KEY_FIELD_NAME = "secretAccessKey";
- public static final String SESSION_TOKEN_FIELD_NAME = "sessionToken";
- public static final String SERVICE_END_POINT_FIELD_NAME = "serviceEndpoint";
-
- // AWS S3 specific error codes
- public static final String ERROR_INTERNAL_ERROR = "InternalError";
- public static final String ERROR_SLOW_DOWN = "SlowDown";
- public static final String ERROR_METHOD_NOT_IMPLEMENTED = "NotImplemented";
-
- public static boolean isRetryableError(String errorCode) {
- return errorCode.equals(ERROR_INTERNAL_ERROR) || errorCode.equals(ERROR_SLOW_DOWN);
- }
-
- /*
- * Hadoop-AWS
- * AWS connectors for s3 and s3n are deprecated.
- */
- public static final String HADOOP_ACCESS_KEY_ID = "fs.s3a.access.key";
- public static final String HADOOP_SECRET_ACCESS_KEY = "fs.s3a.secret.key";
- public static final String HADOOP_SESSION_TOKEN = "fs.s3a.session.token";
- public static final String HADOOP_REGION = "fs.s3a.region";
- public static final String HADOOP_SERVICE_END_POINT = "fs.s3a.endpoint";
-
- /*
- * Internal configurations
- */
- //Allows accessing directories as file system path
- public static final String HADOOP_PATH_STYLE_ACCESS = "fs.s3a.path.style.access";
- //The number of maximum HTTP connections in connection pool
- public static final String HADOOP_S3_CONNECTION_POOL_SIZE = "fs.s3a.connection.maximum";
- //S3 used protocol
- public static final String HADOOP_S3_PROTOCOL = "s3a";
-
- //Hadoop credentials provider key
- public static final String HADOOP_CREDENTIAL_PROVIDER_KEY = "fs.s3a.aws.credentials.provider";
- //Anonymous credential provider
- public static final String HADOOP_ANONYMOUS_ACCESS = "org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider";
- //Temporary credential provider
- public static final String HADOOP_TEMP_ACCESS = "org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider";
-
- }
-
- /*
- * Note: Azure Blob and Azure Datalake use identical authentication, so they are using the same properties.
- * If they end up diverging, then properties for AzureBlob and AzureDataLake need to be created.
- */
- public static class Azure {
- private Azure() {
- throw new AssertionError("do not instantiate");
- }
-
- /*
- * Asterix Configuration Keys
- */
- public static final String MANAGED_IDENTITY_ID_FIELD_NAME = "managedIdentityId";
- public static final String ACCOUNT_NAME_FIELD_NAME = "accountName";
- public static final String ACCOUNT_KEY_FIELD_NAME = "accountKey";
- public static final String SHARED_ACCESS_SIGNATURE_FIELD_NAME = "sharedAccessSignature";
- public static final String TENANT_ID_FIELD_NAME = "tenantId";
- public static final String CLIENT_ID_FIELD_NAME = "clientId";
- public static final String CLIENT_SECRET_FIELD_NAME = "clientSecret";
- public static final String CLIENT_CERTIFICATE_FIELD_NAME = "clientCertificate";
- public static final String CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME = "clientCertificatePassword";
- public static final String ENDPOINT_FIELD_NAME = "endpoint";
-
- // Specific Azure data lake property
- /*
- The behavior of Data Lake (true file system) is to read the files of the specified prefix only, example:
- storage/myData/personal/file1.json
- storage/myData/personal/file2.json
- storage/myData/file3.json
- If the prefix used is "myData", then only the file file3.json is read. However, if the property "recursive"
- is set to "true" when creating the external dataset, then it goes recursively overall the paths, and the result
- is file1.json, file2.json and file3.json.
- */
- public static final String RECURSIVE_FIELD_NAME = "recursive";
-
- /*
- * Hadoop-Azure
- */
- //Used when accountName and accessKey are provided
- public static final String HADOOP_AZURE_FS_ACCOUNT_KEY = "fs.azure.account.key";
- //Used when a connectionString is provided
- public static final String HADOOP_AZURE_FS_SAS = "fs.azure.sas";
- public static final String HADOOP_AZURE_BLOB_PROTOCOL = "wasbs";
- public static final String HADOOP_AZURE_DATALAKE_PROTOCOL = "abfss";
- }
-
- public static class GCS {
- private GCS() {
- throw new AssertionError("do not instantiate");
- }
-
- public static final String JSON_CREDENTIALS_FIELD_NAME = "jsonCredentials";
- }
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
index 8e38eed..d730898 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
@@ -18,65 +18,23 @@
*/
package org.apache.asterix.external.util;
-import static com.google.cloud.storage.Storage.BlobListOption;
-import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.apache.asterix.common.exceptions.ErrorCode.EXTERNAL_SOURCE_ERROR;
-import static org.apache.asterix.common.exceptions.ErrorCode.INVALID_REQ_PARAM_VAL;
-import static org.apache.asterix.common.exceptions.ErrorCode.PARAMETERS_NOT_ALLOWED_AT_SAME_TIME;
-import static org.apache.asterix.common.exceptions.ErrorCode.PARAMETERS_REQUIRED;
-import static org.apache.asterix.common.exceptions.ErrorCode.PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT;
-import static org.apache.asterix.common.exceptions.ErrorCode.REQUIRED_PARAM_IF_PARAM_IS_PRESENT;
-import static org.apache.asterix.common.exceptions.ErrorCode.REQUIRED_PARAM_OR_PARAM_IF_PARAM_IS_PRESENT;
-import static org.apache.asterix.common.exceptions.ErrorCode.S3_REGION_NOT_SUPPORTED;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.ACCESS_KEY_ID_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.ERROR_METHOD_NOT_IMPLEMENTED;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_ACCESS_KEY_ID;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_ANONYMOUS_ACCESS;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_CREDENTIAL_PROVIDER_KEY;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_PATH_STYLE_ACCESS;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_S3_CONNECTION_POOL_SIZE;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_S3_PROTOCOL;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_SECRET_ACCESS_KEY;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_SESSION_TOKEN;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_TEMP_ACCESS;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.SECRET_ACCESS_KEY_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.ACCOUNT_KEY_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.ACCOUNT_NAME_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.CLIENT_CERTIFICATE_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.CLIENT_ID_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.CLIENT_SECRET_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.ENDPOINT_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.HADOOP_AZURE_BLOB_PROTOCOL;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.HADOOP_AZURE_FS_ACCOUNT_KEY;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.HADOOP_AZURE_FS_SAS;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.MANAGED_IDENTITY_ID_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.RECURSIVE_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.SHARED_ACCESS_SIGNATURE_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.TENANT_ID_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.GCS.JSON_CREDENTIALS_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.KEY_ADAPTER_NAME_GCS;
import static org.apache.asterix.external.util.ExternalDataConstants.KEY_DELIMITER;
import static org.apache.asterix.external.util.ExternalDataConstants.KEY_ESCAPE;
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.azure.blob_storage.AzureUtils.validateAzureBlobProperties;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.validateAzureDataLakeProperties;
+import static org.apache.asterix.external.util.google.gcs.GCSUtils.validateProperties;
import static org.apache.asterix.runtime.evaluators.functions.StringEvaluatorUtils.RESERVED_REGEX_CHARS;
-import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
+import static org.msgpack.core.MessagePack.Code.ARRAY16;
-import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.DataOutputStream;
import java.io.IOException;
-import java.io.InputStream;
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
-import java.net.URI;
-import java.net.URISyntaxException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Base64;
@@ -85,7 +43,6 @@
import java.util.List;
import java.util.Map;
import java.util.Optional;
-import java.util.function.BiPredicate;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import java.util.regex.PatternSyntaxException;
@@ -105,21 +62,25 @@
import org.apache.asterix.external.api.IRecordReaderFactory;
import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStreamFactory.IncludeExcludeMatcher;
import org.apache.asterix.external.library.JavaLibrary;
+import org.apache.asterix.external.library.msgpack.MessagePackUtils;
import org.apache.asterix.external.util.ExternalDataConstants.ParquetOptions;
+import org.apache.asterix.external.util.aws.s3.S3Utils;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.types.TypeTagUtil;
import org.apache.asterix.runtime.evaluators.common.NumberUtils;
import org.apache.asterix.runtime.projection.DataProjectionInfo;
import org.apache.asterix.runtime.projection.FunctionCallInformation;
-import org.apache.hadoop.fs.s3a.Constants;
-import org.apache.hadoop.mapred.JobConf;
import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.exceptions.IWarningCollector;
import org.apache.hyracks.api.exceptions.SourceLocation;
-import org.apache.hyracks.api.exceptions.Warning;
-import org.apache.hyracks.api.util.CleanupUtils;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.data.std.primitive.TaggedValuePointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
import org.apache.hyracks.dataflow.common.data.parsers.BooleanParserFactory;
import org.apache.hyracks.dataflow.common.data.parsers.DoubleParserFactory;
import org.apache.hyracks.dataflow.common.data.parsers.FloatParserFactory;
@@ -129,46 +90,6 @@
import org.apache.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
import org.apache.hyracks.util.StorageUtil;
-import com.azure.core.credential.AzureSasCredential;
-import com.azure.core.http.rest.PagedIterable;
-import com.azure.identity.ClientCertificateCredentialBuilder;
-import com.azure.identity.ClientSecretCredentialBuilder;
-import com.azure.identity.ManagedIdentityCredentialBuilder;
-import com.azure.storage.blob.BlobContainerClient;
-import com.azure.storage.blob.BlobServiceClient;
-import com.azure.storage.blob.BlobServiceClientBuilder;
-import com.azure.storage.blob.models.BlobItem;
-import com.azure.storage.blob.models.ListBlobsOptions;
-import com.azure.storage.common.StorageSharedKeyCredential;
-import com.azure.storage.common.policy.RequestRetryOptions;
-import com.azure.storage.file.datalake.DataLakeFileSystemClient;
-import com.azure.storage.file.datalake.DataLakeServiceClient;
-import com.azure.storage.file.datalake.DataLakeServiceClientBuilder;
-import com.azure.storage.file.datalake.models.ListPathsOptions;
-import com.azure.storage.file.datalake.models.PathItem;
-import com.google.api.gax.paging.Page;
-import com.google.auth.oauth2.ServiceAccountCredentials;
-import com.google.cloud.storage.Blob;
-import com.google.cloud.storage.Storage;
-import com.google.cloud.storage.StorageOptions;
-
-import software.amazon.awssdk.auth.credentials.AnonymousCredentialsProvider;
-import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
-import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
-import software.amazon.awssdk.auth.credentials.AwsSessionCredentials;
-import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
-import software.amazon.awssdk.core.exception.SdkException;
-import software.amazon.awssdk.regions.Region;
-import software.amazon.awssdk.services.s3.S3Client;
-import software.amazon.awssdk.services.s3.S3ClientBuilder;
-import software.amazon.awssdk.services.s3.model.ListObjectsRequest;
-import software.amazon.awssdk.services.s3.model.ListObjectsResponse;
-import software.amazon.awssdk.services.s3.model.ListObjectsV2Request;
-import software.amazon.awssdk.services.s3.model.ListObjectsV2Response;
-import software.amazon.awssdk.services.s3.model.S3Exception;
-import software.amazon.awssdk.services.s3.model.S3Object;
-import software.amazon.awssdk.services.s3.model.S3Response;
-
public class ExternalDataUtils {
private static final Map<ATypeTag, IValueParserFactory> valueParserFactoryMap = new EnumMap<>(ATypeTag.class);
private static final int DEFAULT_MAX_ARGUMENT_SZ = 1024 * 1024;
@@ -468,7 +389,8 @@
/**
* Fills the configuration of the external dataset and its adapter with default values if not provided by user.
*
- * @param configuration external data configuration
+ * @param configuration
+ * external data configuration
*/
public static void defaultConfiguration(Map<String, String> configuration) {
String format = configuration.get(ExternalDataConstants.KEY_FORMAT);
@@ -490,8 +412,10 @@
* Prepares the configuration of the external data and its adapter by filling the information required by
* adapters and parsers.
*
- * @param adapterName adapter name
- * @param configuration external data configuration
+ * @param adapterName
+ * adapter name
+ * @param configuration
+ * external data configuration
*/
public static void prepare(String adapterName, Map<String, String> configuration) {
if (!configuration.containsKey(ExternalDataConstants.KEY_READER)) {
@@ -513,7 +437,8 @@
* Normalizes the values of certain parameters of the adapter configuration. This should happen before persisting
* the metadata (e.g. when creating external datasets or feeds) and when creating an adapter factory.
*
- * @param configuration external data configuration
+ * @param configuration
+ * external data configuration
*/
public static void normalize(Map<String, String> configuration) {
// normalize the "format" parameter
@@ -533,8 +458,10 @@
/**
* Validates the parameter values of the adapter configuration. This should happen after normalizing the values.
*
- * @param configuration external data configuration
- * @throws HyracksDataException HyracksDataException
+ * @param configuration
+ * external data configuration
+ * @throws HyracksDataException
+ * HyracksDataException
*/
public static void validate(Map<String, String> configuration) throws HyracksDataException {
String format = configuration.get(ExternalDataConstants.KEY_FORMAT);
@@ -596,7 +523,8 @@
* Validates adapter specific external dataset properties. Specific properties for different adapters should be
* validated here
*
- * @param configuration properties
+ * @param configuration
+ * properties
*/
public static void validateAdapterSpecificProperties(Map<String, String> configuration, SourceLocation srcLoc,
IWarningCollector collector, IApplicationContext appCtx) throws CompilationException {
@@ -604,16 +532,16 @@
switch (type) {
case ExternalDataConstants.KEY_ADAPTER_NAME_AWS_S3:
- AwsS3.validateProperties(configuration, srcLoc, collector);
+ S3Utils.validateProperties(configuration, srcLoc, collector);
break;
case ExternalDataConstants.KEY_ADAPTER_NAME_AZURE_BLOB:
- Azure.validateAzureBlobProperties(configuration, srcLoc, collector, appCtx);
+ validateAzureBlobProperties(configuration, srcLoc, collector, appCtx);
break;
case ExternalDataConstants.KEY_ADAPTER_NAME_AZURE_DATA_LAKE:
- Azure.validateAzureDataLakeProperties(configuration, srcLoc, collector, appCtx);
+ validateAzureDataLakeProperties(configuration, srcLoc, collector, appCtx);
break;
- case KEY_ADAPTER_NAME_GCS:
- GCS.validateProperties(configuration, srcLoc, collector);
+ case ExternalDataConstants.KEY_ADAPTER_NAME_GCS:
+ validateProperties(configuration, srcLoc, collector);
break;
default:
// Nothing needs to be done
@@ -624,7 +552,8 @@
/**
* Regex matches all the provided patterns against the provided path
*
- * @param path path to check against
+ * @param path
+ * path to check against
* @return {@code true} if all patterns match, {@code false} otherwise
*/
public static boolean matchPatterns(List<Matcher> matchers, String path) {
@@ -639,7 +568,8 @@
/**
* Converts the wildcard to proper regex
*
- * @param pattern wildcard pattern to convert
+ * @param pattern
+ * wildcard pattern to convert
* @return regex expression
*/
public static String patternToRegex(String pattern) {
@@ -728,7 +658,8 @@
/**
* Adjusts the prefix (if needed) and returns it
*
- * @param configuration configuration
+ * @param configuration
+ * configuration
*/
public static String getPrefix(Map<String, String> configuration) {
return getPrefix(configuration, true);
@@ -736,15 +667,35 @@
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;
}
/**
- * @param configuration configuration map
- * @throws CompilationException Compilation exception
+ * @param configuration
+ * configuration map
+ * @throws CompilationException
+ * Compilation exception
*/
public static void validateIncludeExclude(Map<String, String> configuration) throws CompilationException {
// Ensure that include and exclude are not provided at the same time + ensure valid format or property
@@ -828,8 +779,10 @@
/**
* Validate Parquet dataset's declared type and configuration
*
- * @param properties external dataset configuration
- * @param datasetRecordType dataset declared type
+ * @param properties
+ * external dataset configuration
+ * @param datasetRecordType
+ * dataset declared type
*/
public static void validateParquetTypeAndConfiguration(Map<String, String> properties,
ARecordType datasetRecordType) throws CompilationException {
@@ -844,7 +797,7 @@
}
}
- private static boolean isParquetFormat(Map<String, String> properties) {
+ public static boolean isParquetFormat(Map<String, String> properties) {
String inputFormat = properties.get(ExternalDataConstants.KEY_INPUT_FORMAT);
return ExternalDataConstants.CLASS_NAME_PARQUET_INPUT_FORMAT.equals(inputFormat)
|| ExternalDataConstants.INPUT_FORMAT_PARQUET.equals(inputFormat)
@@ -862,7 +815,8 @@
/**
* Serialize {@link ARecordType} as Base64 string to pass it to {@link org.apache.hadoop.conf.Configuration}
*
- * @param expectedType expected type
+ * @param expectedType
+ * expected type
* @return the expected type as Base64 string
*/
private static String serializeExpectedTypeToString(ARecordType expectedType) throws IOException {
@@ -881,7 +835,8 @@
* Serialize {@link FunctionCallInformation} map as Base64 string to pass it to
* {@link org.apache.hadoop.conf.Configuration}
*
- * @param functionCallInfoMap function information map
+ * @param functionCallInfoMap
+ * function information map
* @return function information map as Base64 string
*/
static String serializeFunctionCallInfoToString(Map<String, FunctionCallInformation> functionCallInfoMap)
@@ -893,1008 +848,6 @@
return encoder.encodeToString(byteArrayOutputStream.toByteArray());
}
- public static class AwsS3 {
- private AwsS3() {
- throw new AssertionError("do not instantiate");
- }
-
- /**
- * Builds the S3 client using the provided configuration
- *
- * @param configuration properties
- * @return S3 client
- * @throws CompilationException CompilationException
- */
- public static S3Client buildAwsS3Client(Map<String, String> configuration) throws CompilationException {
- // TODO(Hussain): Need to ensure that all required parameters are present in a previous step
- String accessKeyId = configuration.get(ACCESS_KEY_ID_FIELD_NAME);
- String secretAccessKey = configuration.get(SECRET_ACCESS_KEY_FIELD_NAME);
- String sessionToken = configuration.get(ExternalDataConstants.AwsS3.SESSION_TOKEN_FIELD_NAME);
- String regionId = configuration.get(ExternalDataConstants.AwsS3.REGION_FIELD_NAME);
- String serviceEndpoint = configuration.get(ExternalDataConstants.AwsS3.SERVICE_END_POINT_FIELD_NAME);
-
- S3ClientBuilder builder = S3Client.builder();
-
- // Credentials
- AwsCredentialsProvider credentialsProvider;
-
- // No auth required
- if (accessKeyId == null) {
- credentialsProvider = AnonymousCredentialsProvider.create();
- } else {
- // auth required, check for temporary or permanent credentials
- if (sessionToken != null) {
- credentialsProvider = StaticCredentialsProvider
- .create(AwsSessionCredentials.create(accessKeyId, secretAccessKey, sessionToken));
- } else {
- credentialsProvider =
- StaticCredentialsProvider.create(AwsBasicCredentials.create(accessKeyId, secretAccessKey));
- }
- }
-
- builder.credentialsProvider(credentialsProvider);
-
- // Validate the region
- List<Region> regions = S3Client.serviceMetadata().regions();
- Optional<Region> selectedRegion =
- regions.stream().filter(region -> region.id().equals(regionId)).findFirst();
-
- if (selectedRegion.isEmpty()) {
- throw new CompilationException(S3_REGION_NOT_SUPPORTED, regionId);
- }
- builder.region(selectedRegion.get());
-
- // Validate the service endpoint if present
- if (serviceEndpoint != null) {
- try {
- URI uri = new URI(serviceEndpoint);
- try {
- builder.endpointOverride(uri);
- } catch (NullPointerException ex) {
- throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
- }
- } catch (URISyntaxException ex) {
- throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR,
- String.format("Invalid service endpoint %s", serviceEndpoint));
- }
- }
-
- return builder.build();
- }
-
- /**
- * Builds the S3 client using the provided configuration
- *
- * @param configuration properties
- * @param numberOfPartitions number of partitions in the cluster
- */
- public static void configureAwsS3HdfsJobConf(JobConf conf, Map<String, String> configuration,
- int numberOfPartitions) {
- String accessKeyId = configuration.get(ExternalDataConstants.AwsS3.ACCESS_KEY_ID_FIELD_NAME);
- String secretAccessKey = configuration.get(ExternalDataConstants.AwsS3.SECRET_ACCESS_KEY_FIELD_NAME);
- String sessionToken = configuration.get(ExternalDataConstants.AwsS3.SESSION_TOKEN_FIELD_NAME);
- String serviceEndpoint = configuration.get(ExternalDataConstants.AwsS3.SERVICE_END_POINT_FIELD_NAME);
-
- //Disable caching S3 FileSystem
- HDFSUtils.disableHadoopFileSystemCache(conf, HADOOP_S3_PROTOCOL);
-
- /*
- * Authentication Methods:
- * 1- Anonymous: no accessKeyId and no secretAccessKey
- * 2- Temporary: has to provide accessKeyId, secretAccessKey and sessionToken
- * 3- Private: has to provide accessKeyId and secretAccessKey
- */
- if (accessKeyId == null) {
- //Tells hadoop-aws it is an anonymous access
- conf.set(HADOOP_CREDENTIAL_PROVIDER_KEY, HADOOP_ANONYMOUS_ACCESS);
- } else {
- conf.set(HADOOP_ACCESS_KEY_ID, accessKeyId);
- conf.set(HADOOP_SECRET_ACCESS_KEY, secretAccessKey);
- if (sessionToken != null) {
- conf.set(HADOOP_SESSION_TOKEN, sessionToken);
- //Tells hadoop-aws it is a temporary access
- conf.set(HADOOP_CREDENTIAL_PROVIDER_KEY, HADOOP_TEMP_ACCESS);
- }
- }
-
- /*
- * This is to allow S3 definition to have path-style form. Should always be true to match the current
- * way we access files in S3
- */
- conf.set(HADOOP_PATH_STYLE_ACCESS, ExternalDataConstants.TRUE);
-
- /*
- * Set the size of S3 connection pool to be the number of partitions
- */
- conf.set(HADOOP_S3_CONNECTION_POOL_SIZE, String.valueOf(numberOfPartitions));
-
- if (serviceEndpoint != null) {
- // Validation of the URL should be done at hadoop-aws level
- conf.set(ExternalDataConstants.AwsS3.HADOOP_SERVICE_END_POINT, serviceEndpoint);
- } else {
- //Region is ignored and buckets could be found by the central endpoint
- conf.set(ExternalDataConstants.AwsS3.HADOOP_SERVICE_END_POINT, Constants.CENTRAL_ENDPOINT);
- }
- }
-
- /**
- * Validate external dataset properties
- *
- * @param configuration properties
- * @throws CompilationException Compilation exception
- */
- public static void validateProperties(Map<String, String> configuration, SourceLocation srcLoc,
- IWarningCollector collector) throws CompilationException {
-
- // check if the format property is present
- if (configuration.get(ExternalDataConstants.KEY_FORMAT) == null) {
- throw new CompilationException(ErrorCode.PARAMETERS_REQUIRED, srcLoc, ExternalDataConstants.KEY_FORMAT);
- }
-
- // Both parameters should be passed, or neither should be passed (for anonymous/no auth)
- String accessKeyId = configuration.get(ACCESS_KEY_ID_FIELD_NAME);
- String secretAccessKey = configuration.get(SECRET_ACCESS_KEY_FIELD_NAME);
- if (accessKeyId == null || secretAccessKey == null) {
- // If one is passed, the other is required
- if (accessKeyId != null) {
- throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, SECRET_ACCESS_KEY_FIELD_NAME,
- ACCESS_KEY_ID_FIELD_NAME);
- } else if (secretAccessKey != null) {
- throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCESS_KEY_ID_FIELD_NAME,
- SECRET_ACCESS_KEY_FIELD_NAME);
- }
- }
-
- validateIncludeExclude(configuration);
-
- // Check if the bucket is present
- S3Client s3Client = buildAwsS3Client(configuration);
- S3Response response;
- boolean useOldApi = false;
- String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
- String prefix = getPrefix(configuration);
-
- try {
- response = isBucketEmpty(s3Client, container, prefix, false);
- } catch (S3Exception ex) {
- // Method not implemented, try falling back to old API
- try {
- // For error code, see https://docs.aws.amazon.com/AmazonS3/latest/API/ErrorResponses.html
- if (ex.awsErrorDetails().errorCode().equals(ERROR_METHOD_NOT_IMPLEMENTED)) {
- useOldApi = true;
- response = isBucketEmpty(s3Client, container, prefix, true);
- } else {
- throw ex;
- }
- } catch (SdkException ex2) {
- throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
- }
- } catch (SdkException ex) {
- throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
- } finally {
- if (s3Client != null) {
- CleanupUtils.close(s3Client, null);
- }
- }
-
- boolean isEmpty = useOldApi ? ((ListObjectsResponse) response).contents().isEmpty()
- : ((ListObjectsV2Response) response).contents().isEmpty();
- if (isEmpty && collector.shouldWarn()) {
- Warning warning = Warning.of(srcLoc, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
- collector.warn(warning);
- }
-
- // Returns 200 only in case the bucket exists, otherwise, throws an exception. However, to
- // ensure coverage, check if the result is successful as well and not only catch exceptions
- if (!response.sdkHttpResponse().isSuccessful()) {
- throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_CONTAINER_NOT_FOUND, container);
- }
- }
-
- /**
- * Checks for a single object in the specified bucket to determine if the bucket is empty or not.
- *
- * @param s3Client s3 client
- * @param container the container name
- * @param prefix Prefix to be used
- * @param useOldApi flag whether to use the old API or not
- * @return returns the S3 response
- */
- private static S3Response isBucketEmpty(S3Client s3Client, String container, String prefix, boolean useOldApi) {
- S3Response response;
- if (useOldApi) {
- ListObjectsRequest.Builder listObjectsBuilder = ListObjectsRequest.builder();
- listObjectsBuilder.prefix(prefix);
- response = s3Client.listObjects(listObjectsBuilder.bucket(container).maxKeys(1).build());
- } else {
- ListObjectsV2Request.Builder listObjectsBuilder = ListObjectsV2Request.builder();
- listObjectsBuilder.prefix(prefix);
- response = s3Client.listObjectsV2(listObjectsBuilder.bucket(container).maxKeys(1).build());
- }
- return response;
- }
-
- /**
- * Returns the lists of S3 objects.
- *
- * @param configuration properties
- * @param includeExcludeMatcher include/exclude matchers to apply
- */
- public static List<S3Object> listS3Objects(Map<String, String> configuration,
- IncludeExcludeMatcher includeExcludeMatcher, IWarningCollector warningCollector)
- throws CompilationException {
- // Prepare to retrieve the objects
- List<S3Object> filesOnly;
- String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
- S3Client s3Client = buildAwsS3Client(configuration);
- String prefix = getPrefix(configuration);
-
- try {
- filesOnly = listS3Objects(s3Client, container, prefix, includeExcludeMatcher);
- } catch (S3Exception ex) {
- // New API is not implemented, try falling back to old API
- try {
- // For error code, see https://docs.aws.amazon.com/AmazonS3/latest/API/ErrorResponses.html
- if (ex.awsErrorDetails().errorCode()
- .equals(ExternalDataConstants.AwsS3.ERROR_METHOD_NOT_IMPLEMENTED)) {
- filesOnly = oldApiListS3Objects(s3Client, container, prefix, includeExcludeMatcher);
- } else {
- throw ex;
- }
- } catch (SdkException ex2) {
- throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
- }
- } catch (SdkException ex) {
- throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
- } finally {
- if (s3Client != null) {
- CleanupUtils.close(s3Client, null);
- }
- }
-
- // Warn if no files are returned
- if (filesOnly.isEmpty() && warningCollector.shouldWarn()) {
- Warning warning = Warning.of(null, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
- warningCollector.warn(warning);
- }
-
- return filesOnly;
- }
-
- /**
- * Uses the latest API to retrieve the objects from the storage.
- *
- * @param s3Client S3 client
- * @param container container name
- * @param prefix definition prefix
- * @param includeExcludeMatcher include/exclude matchers to apply
- */
- private static List<S3Object> listS3Objects(S3Client s3Client, String container, String prefix,
- IncludeExcludeMatcher includeExcludeMatcher) {
- String newMarker = null;
- List<S3Object> filesOnly = new ArrayList<>();
-
- ListObjectsV2Response listObjectsResponse;
- ListObjectsV2Request.Builder listObjectsBuilder = ListObjectsV2Request.builder().bucket(container);
- listObjectsBuilder.prefix(prefix);
-
- while (true) {
- // List the objects from the start, or from the last marker in case of truncated result
- if (newMarker == null) {
- listObjectsResponse = s3Client.listObjectsV2(listObjectsBuilder.build());
- } else {
- listObjectsResponse =
- s3Client.listObjectsV2(listObjectsBuilder.continuationToken(newMarker).build());
- }
-
- // Collect the paths to files only
- collectAndFilterFiles(listObjectsResponse.contents(), includeExcludeMatcher.getPredicate(),
- includeExcludeMatcher.getMatchersList(), filesOnly);
-
- // Mark the flag as done if done, otherwise, get the marker of the previous response for the next request
- if (!listObjectsResponse.isTruncated()) {
- break;
- } else {
- newMarker = listObjectsResponse.nextContinuationToken();
- }
- }
-
- return filesOnly;
- }
-
- /**
- * Uses the old API (in case the new API is not implemented) to retrieve the objects from the storage
- *
- * @param s3Client S3 client
- * @param container container name
- * @param prefix definition prefix
- * @param includeExcludeMatcher include/exclude matchers to apply
- */
- private static List<S3Object> oldApiListS3Objects(S3Client s3Client, String container, String prefix,
- IncludeExcludeMatcher includeExcludeMatcher) {
- String newMarker = null;
- List<S3Object> filesOnly = new ArrayList<>();
-
- ListObjectsResponse listObjectsResponse;
- ListObjectsRequest.Builder listObjectsBuilder = ListObjectsRequest.builder().bucket(container);
- listObjectsBuilder.prefix(prefix);
-
- while (true) {
- // List the objects from the start, or from the last marker in case of truncated result
- if (newMarker == null) {
- listObjectsResponse = s3Client.listObjects(listObjectsBuilder.build());
- } else {
- listObjectsResponse = s3Client.listObjects(listObjectsBuilder.marker(newMarker).build());
- }
-
- // Collect the paths to files only
- collectAndFilterFiles(listObjectsResponse.contents(), includeExcludeMatcher.getPredicate(),
- includeExcludeMatcher.getMatchersList(), filesOnly);
-
- // Mark the flag as done if done, otherwise, get the marker of the previous response for the next request
- if (!listObjectsResponse.isTruncated()) {
- break;
- } else {
- newMarker = listObjectsResponse.nextMarker();
- }
- }
-
- return filesOnly;
- }
-
- /**
- * AWS S3 returns all the objects as paths, not differentiating between folder and files. The path is considered
- * a file if it does not end up with a "/" which is the separator in a folder structure.
- *
- * @param s3Objects List of returned objects
- */
- private static void collectAndFilterFiles(List<S3Object> s3Objects,
- BiPredicate<List<Matcher>, String> predicate, List<Matcher> matchers, List<S3Object> filesOnly) {
- for (S3Object object : s3Objects) {
- // skip folders
- if (object.key().endsWith("/")) {
- continue;
- }
-
- // No filter, add file
- if (predicate.test(matchers, object.key())) {
- filesOnly.add(object);
- }
- }
- }
- }
-
- /*
- * Note: Azure Blob and Azure Datalake use identical authentication, so they are using the same properties.
- * If they end up diverging, then properties for AzureBlob and AzureDataLake need to be created.
- */
- public static class Azure {
- private Azure() {
- throw new AssertionError("do not instantiate");
- }
-
- /**
- * Builds the Azure storage account using the provided configuration
- *
- * @param configuration properties
- * @return client
- */
- public static BlobServiceClient buildAzureBlobClient(IApplicationContext appCtx,
- Map<String, String> configuration) throws CompilationException {
- String managedIdentityId = configuration.get(MANAGED_IDENTITY_ID_FIELD_NAME);
- String accountName = configuration.get(ACCOUNT_NAME_FIELD_NAME);
- String accountKey = configuration.get(ACCOUNT_KEY_FIELD_NAME);
- String sharedAccessSignature = configuration.get(SHARED_ACCESS_SIGNATURE_FIELD_NAME);
- String tenantId = configuration.get(TENANT_ID_FIELD_NAME);
- String clientId = configuration.get(CLIENT_ID_FIELD_NAME);
- String clientSecret = configuration.get(CLIENT_SECRET_FIELD_NAME);
- String clientCertificate = configuration.get(CLIENT_CERTIFICATE_FIELD_NAME);
- String clientCertificatePassword = configuration.get(CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME);
- String endpoint = configuration.get(ENDPOINT_FIELD_NAME);
-
- // Client builder
- BlobServiceClientBuilder builder = new BlobServiceClientBuilder();
- int timeout = appCtx.getExternalProperties().getAzureRequestTimeout();
- RequestRetryOptions requestRetryOptions = new RequestRetryOptions(null, null, timeout, null, null, null);
- builder.retryOptions(requestRetryOptions);
-
- // Endpoint is required
- if (endpoint == null) {
- throw new CompilationException(PARAMETERS_REQUIRED, ENDPOINT_FIELD_NAME);
- }
- builder.endpoint(endpoint);
-
- // Shared Key
- if (accountName != null || accountKey != null) {
- if (accountName == null) {
- throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCOUNT_NAME_FIELD_NAME,
- ACCOUNT_KEY_FIELD_NAME);
- }
-
- if (accountKey == null) {
- throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCOUNT_KEY_FIELD_NAME,
- ACCOUNT_NAME_FIELD_NAME);
- }
-
- Optional<String> provided = getFirstNotNull(configuration, SHARED_ACCESS_SIGNATURE_FIELD_NAME,
- MANAGED_IDENTITY_ID_FIELD_NAME, CLIENT_ID_FIELD_NAME, CLIENT_SECRET_FIELD_NAME,
- CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
- if (provided.isPresent()) {
- throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
- ACCOUNT_KEY_FIELD_NAME);
- }
- StorageSharedKeyCredential credential = new StorageSharedKeyCredential(accountName, accountKey);
- builder.credential(credential);
- }
-
- // Shared access signature
- if (sharedAccessSignature != null) {
- Optional<String> provided = getFirstNotNull(configuration, MANAGED_IDENTITY_ID_FIELD_NAME,
- CLIENT_ID_FIELD_NAME, CLIENT_SECRET_FIELD_NAME, CLIENT_CERTIFICATE_FIELD_NAME,
- CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
- if (provided.isPresent()) {
- throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
- SHARED_ACCESS_SIGNATURE_FIELD_NAME);
- }
- AzureSasCredential credential = new AzureSasCredential(sharedAccessSignature);
- builder.credential(credential);
- }
-
- // Managed Identity auth
- if (managedIdentityId != null) {
- Optional<String> provided = getFirstNotNull(configuration, CLIENT_ID_FIELD_NAME,
- CLIENT_SECRET_FIELD_NAME, CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME,
- TENANT_ID_FIELD_NAME);
- if (provided.isPresent()) {
- throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
- MANAGED_IDENTITY_ID_FIELD_NAME);
- }
- builder.credential(new ManagedIdentityCredentialBuilder().clientId(managedIdentityId).build());
- }
-
- // Client secret & certificate auth
- if (clientId != null) {
- // Both (or neither) client secret and client secret were provided, only one is allowed
- if ((clientSecret == null) == (clientCertificate == null)) {
- if (clientSecret != null) {
- throw new CompilationException(PARAMETERS_NOT_ALLOWED_AT_SAME_TIME, CLIENT_SECRET_FIELD_NAME,
- CLIENT_CERTIFICATE_FIELD_NAME);
- } else {
- throw new CompilationException(REQUIRED_PARAM_OR_PARAM_IF_PARAM_IS_PRESENT,
- CLIENT_SECRET_FIELD_NAME, CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_ID_FIELD_NAME);
- }
- }
-
- // Tenant ID is required
- if (tenantId == null) {
- throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, TENANT_ID_FIELD_NAME,
- CLIENT_ID_FIELD_NAME);
- }
-
- // Client certificate password is not allowed if client secret is used
- if (clientCertificatePassword != null && clientSecret != null) {
- throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT,
- CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, CLIENT_SECRET_FIELD_NAME);
- }
-
- // Use AD authentication
- if (clientSecret != null) {
- ClientSecretCredentialBuilder secret = new ClientSecretCredentialBuilder();
- secret.clientId(clientId);
- secret.tenantId(tenantId);
- secret.clientSecret(clientSecret);
- builder.credential(secret.build());
- } else {
- // Certificate
- ClientCertificateCredentialBuilder certificate = new ClientCertificateCredentialBuilder();
- certificate.clientId(clientId);
- certificate.tenantId(tenantId);
- try {
- InputStream certificateContent = new ByteArrayInputStream(clientCertificate.getBytes(UTF_8));
- if (clientCertificatePassword == null) {
- Method pemCertificate = ClientCertificateCredentialBuilder.class
- .getDeclaredMethod("pemCertificate", InputStream.class);
- pemCertificate.setAccessible(true);
- pemCertificate.invoke(certificate, certificateContent);
- } else {
- Method pemCertificate = ClientCertificateCredentialBuilder.class
- .getDeclaredMethod("pfxCertificate", InputStream.class, String.class);
- pemCertificate.setAccessible(true);
- pemCertificate.invoke(certificate, certificateContent, clientCertificatePassword);
- }
- } catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException ex) {
- throw new CompilationException(EXTERNAL_SOURCE_ERROR, ex.getMessage());
- }
- builder.credential(certificate.build());
- }
- }
-
- // If client id is not present, ensure client secret, certificate, tenant id and client certificate
- // password are not present
- if (clientId == null) {
- Optional<String> provided = getFirstNotNull(configuration, CLIENT_SECRET_FIELD_NAME,
- CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
- if (provided.isPresent()) {
- throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
- SHARED_ACCESS_SIGNATURE_FIELD_NAME);
- }
- }
-
- try {
- return builder.buildClient();
- } catch (Exception ex) {
- throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
- }
- }
-
- /**
- * Builds the Azure data lake storage account using the provided configuration
- *
- * @param configuration properties
- * @return client
- */
- public static DataLakeServiceClient buildAzureDatalakeClient(IApplicationContext appCtx,
- Map<String, String> configuration) throws CompilationException {
- String managedIdentityId = configuration.get(MANAGED_IDENTITY_ID_FIELD_NAME);
- String accountName = configuration.get(ACCOUNT_NAME_FIELD_NAME);
- String accountKey = configuration.get(ACCOUNT_KEY_FIELD_NAME);
- String sharedAccessSignature = configuration.get(SHARED_ACCESS_SIGNATURE_FIELD_NAME);
- String tenantId = configuration.get(TENANT_ID_FIELD_NAME);
- String clientId = configuration.get(CLIENT_ID_FIELD_NAME);
- String clientSecret = configuration.get(CLIENT_SECRET_FIELD_NAME);
- String clientCertificate = configuration.get(CLIENT_CERTIFICATE_FIELD_NAME);
- String clientCertificatePassword = configuration.get(CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME);
- String endpoint = configuration.get(ENDPOINT_FIELD_NAME);
-
- // Client builder
- DataLakeServiceClientBuilder builder = new DataLakeServiceClientBuilder();
- int timeout = appCtx.getExternalProperties().getAzureRequestTimeout();
- RequestRetryOptions requestRetryOptions = new RequestRetryOptions(null, null, timeout, null, null, null);
- builder.retryOptions(requestRetryOptions);
-
- // Endpoint is required
- if (endpoint == null) {
- throw new CompilationException(PARAMETERS_REQUIRED, ENDPOINT_FIELD_NAME);
- }
- builder.endpoint(endpoint);
-
- // Shared Key
- if (accountName != null || accountKey != null) {
- if (accountName == null) {
- throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCOUNT_NAME_FIELD_NAME,
- ACCOUNT_KEY_FIELD_NAME);
- }
-
- if (accountKey == null) {
- throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCOUNT_KEY_FIELD_NAME,
- ACCOUNT_NAME_FIELD_NAME);
- }
-
- Optional<String> provided = getFirstNotNull(configuration, SHARED_ACCESS_SIGNATURE_FIELD_NAME,
- MANAGED_IDENTITY_ID_FIELD_NAME, CLIENT_ID_FIELD_NAME, CLIENT_SECRET_FIELD_NAME,
- CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
- if (provided.isPresent()) {
- throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
- ACCOUNT_KEY_FIELD_NAME);
- }
- StorageSharedKeyCredential credential = new StorageSharedKeyCredential(accountName, accountKey);
- builder.credential(credential);
- }
-
- // Shared access signature
- if (sharedAccessSignature != null) {
- Optional<String> provided = getFirstNotNull(configuration, MANAGED_IDENTITY_ID_FIELD_NAME,
- CLIENT_ID_FIELD_NAME, CLIENT_SECRET_FIELD_NAME, CLIENT_CERTIFICATE_FIELD_NAME,
- CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
- if (provided.isPresent()) {
- throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
- SHARED_ACCESS_SIGNATURE_FIELD_NAME);
- }
- AzureSasCredential credential = new AzureSasCredential(sharedAccessSignature);
- builder.credential(credential);
- }
-
- // Managed Identity auth
- if (managedIdentityId != null) {
- Optional<String> provided = getFirstNotNull(configuration, CLIENT_ID_FIELD_NAME,
- CLIENT_SECRET_FIELD_NAME, CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME,
- TENANT_ID_FIELD_NAME);
- if (provided.isPresent()) {
- throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
- MANAGED_IDENTITY_ID_FIELD_NAME);
- }
- builder.credential(new ManagedIdentityCredentialBuilder().clientId(managedIdentityId).build());
- }
-
- // Client secret & certificate auth
- if (clientId != null) {
- // Both (or neither) client secret and client secret were provided, only one is allowed
- if ((clientSecret == null) == (clientCertificate == null)) {
- if (clientSecret != null) {
- throw new CompilationException(PARAMETERS_NOT_ALLOWED_AT_SAME_TIME, CLIENT_SECRET_FIELD_NAME,
- CLIENT_CERTIFICATE_FIELD_NAME);
- } else {
- throw new CompilationException(REQUIRED_PARAM_OR_PARAM_IF_PARAM_IS_PRESENT,
- CLIENT_SECRET_FIELD_NAME, CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_ID_FIELD_NAME);
- }
- }
-
- // Tenant ID is required
- if (tenantId == null) {
- throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, TENANT_ID_FIELD_NAME,
- CLIENT_ID_FIELD_NAME);
- }
-
- // Client certificate password is not allowed if client secret is used
- if (clientCertificatePassword != null && clientSecret != null) {
- throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT,
- CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, CLIENT_SECRET_FIELD_NAME);
- }
-
- // Use AD authentication
- if (clientSecret != null) {
- ClientSecretCredentialBuilder secret = new ClientSecretCredentialBuilder();
- secret.clientId(clientId);
- secret.tenantId(tenantId);
- secret.clientSecret(clientSecret);
- builder.credential(secret.build());
- } else {
- // Certificate
- ClientCertificateCredentialBuilder certificate = new ClientCertificateCredentialBuilder();
- certificate.clientId(clientId);
- certificate.tenantId(tenantId);
- try {
- InputStream certificateContent = new ByteArrayInputStream(clientCertificate.getBytes(UTF_8));
- if (clientCertificatePassword == null) {
- Method pemCertificate = ClientCertificateCredentialBuilder.class
- .getDeclaredMethod("pemCertificate", InputStream.class);
- pemCertificate.setAccessible(true);
- pemCertificate.invoke(certificate, certificateContent);
- } else {
- Method pemCertificate = ClientCertificateCredentialBuilder.class
- .getDeclaredMethod("pfxCertificate", InputStream.class, String.class);
- pemCertificate.setAccessible(true);
- pemCertificate.invoke(certificate, certificateContent, clientCertificatePassword);
- }
- } catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException ex) {
- throw new CompilationException(EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
- }
- builder.credential(certificate.build());
- }
- }
-
- // If client id is not present, ensure client secret, certificate, tenant id and client certificate
- // password are not present
- if (clientId == null) {
- Optional<String> provided = getFirstNotNull(configuration, CLIENT_SECRET_FIELD_NAME,
- CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
- if (provided.isPresent()) {
- throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
- SHARED_ACCESS_SIGNATURE_FIELD_NAME);
- }
- }
-
- try {
- return builder.buildClient();
- } catch (Exception ex) {
- throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
- }
- }
-
- public static List<BlobItem> listBlobItems(BlobServiceClient blobServiceClient,
- Map<String, String> configuration, IncludeExcludeMatcher includeExcludeMatcher,
- IWarningCollector warningCollector) throws CompilationException {
- String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
-
- List<BlobItem> filesOnly = new ArrayList<>();
-
- // Ensure the validity of include/exclude
- ExternalDataUtils.validateIncludeExclude(configuration);
-
- BlobContainerClient blobContainer;
- try {
- blobContainer = blobServiceClient.getBlobContainerClient(container);
-
- // Get all objects in a container and extract the paths to files
- ListBlobsOptions listBlobsOptions = new ListBlobsOptions();
- listBlobsOptions.setPrefix(ExternalDataUtils.getPrefix(configuration));
- Iterable<BlobItem> blobItems = blobContainer.listBlobs(listBlobsOptions, null);
-
- // Collect the paths to files only
- collectAndFilterBlobFiles(blobItems, includeExcludeMatcher.getPredicate(),
- includeExcludeMatcher.getMatchersList(), filesOnly);
-
- // Warn if no files are returned
- if (filesOnly.isEmpty() && warningCollector.shouldWarn()) {
- Warning warning = Warning.of(null, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
- warningCollector.warn(warning);
- }
- } catch (Exception ex) {
- throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
- }
-
- return filesOnly;
- }
-
- /**
- * Collects and filters the files only, and excludes any folders
- *
- * @param items storage items
- * @param predicate predicate to test with for file filtration
- * @param matchers include/exclude matchers to test against
- * @param filesOnly List containing the files only (excluding folders)
- */
- private static void collectAndFilterBlobFiles(Iterable<BlobItem> items,
- BiPredicate<List<Matcher>, String> predicate, List<Matcher> matchers, List<BlobItem> filesOnly) {
- for (BlobItem item : items) {
- String uri = item.getName();
-
- // skip folders
- if (uri.endsWith("/")) {
- continue;
- }
-
- // No filter, add file
- if (predicate.test(matchers, uri)) {
- filesOnly.add(item);
- }
- }
- }
-
- public static List<PathItem> listDatalakePathItems(DataLakeServiceClient client,
- Map<String, String> configuration, IncludeExcludeMatcher includeExcludeMatcher,
- IWarningCollector warningCollector) throws CompilationException {
- String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
-
- List<PathItem> filesOnly = new ArrayList<>();
-
- // Ensure the validity of include/exclude
- ExternalDataUtils.validateIncludeExclude(configuration);
-
- DataLakeFileSystemClient fileSystemClient;
- try {
- fileSystemClient = client.getFileSystemClient(container);
-
- // Get all objects in a container and extract the paths to files
- ListPathsOptions listOptions = new ListPathsOptions();
- boolean recursive = Boolean.parseBoolean(configuration.get(RECURSIVE_FIELD_NAME));
- listOptions.setRecursive(recursive);
- listOptions.setPath(ExternalDataUtils.getPrefix(configuration, false));
- PagedIterable<PathItem> pathItems = fileSystemClient.listPaths(listOptions, null);
-
- // Collect the paths to files only
- collectAndFilterDatalakeFiles(pathItems, includeExcludeMatcher.getPredicate(),
- includeExcludeMatcher.getMatchersList(), filesOnly);
-
- // Warn if no files are returned
- if (filesOnly.isEmpty() && warningCollector.shouldWarn()) {
- Warning warning = Warning.of(null, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
- warningCollector.warn(warning);
- }
- } catch (Exception ex) {
- throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
- }
-
- return filesOnly;
- }
-
- /**
- * Collects and filters the files only, and excludes any folders
- *
- * @param items storage items
- * @param predicate predicate to test with for file filtration
- * @param matchers include/exclude matchers to test against
- * @param filesOnly List containing the files only (excluding folders)
- */
- private static void collectAndFilterDatalakeFiles(Iterable<PathItem> items,
- BiPredicate<List<Matcher>, String> predicate, List<Matcher> matchers, List<PathItem> filesOnly) {
- for (PathItem item : items) {
- String uri = item.getName();
-
- // skip folders
- if (uri.endsWith("/")) {
- continue;
- }
-
- // No filter, add file
- if (predicate.test(matchers, uri)) {
- filesOnly.add(item);
- }
- }
- }
-
- /**
- * Validate external dataset properties
- *
- * @param configuration properties
- * @throws CompilationException Compilation exception
- */
- public static void validateAzureBlobProperties(Map<String, String> configuration, SourceLocation srcLoc,
- IWarningCollector collector, IApplicationContext appCtx) throws CompilationException {
-
- // check if the format property is present
- if (configuration.get(ExternalDataConstants.KEY_FORMAT) == null) {
- throw new CompilationException(ErrorCode.PARAMETERS_REQUIRED, srcLoc, ExternalDataConstants.KEY_FORMAT);
- }
-
- validateIncludeExclude(configuration);
-
- // Check if the bucket is present
- BlobServiceClient blobServiceClient;
- try {
- String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
- blobServiceClient = buildAzureBlobClient(appCtx, configuration);
- BlobContainerClient blobContainer = blobServiceClient.getBlobContainerClient(container);
-
- // Get all objects in a container and extract the paths to files
- ListBlobsOptions listBlobsOptions = new ListBlobsOptions();
- listBlobsOptions.setPrefix(getPrefix(configuration));
- Iterable<BlobItem> blobItems = blobContainer.listBlobs(listBlobsOptions, null);
-
- if (!blobItems.iterator().hasNext() && collector.shouldWarn()) {
- Warning warning = Warning.of(srcLoc, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
- collector.warn(warning);
- }
- } catch (CompilationException ex) {
- throw ex;
- } catch (Exception ex) {
- throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
- }
- }
-
- /**
- * Validate external dataset properties
- *
- * @param configuration properties
- * @throws CompilationException Compilation exception
- */
- public static void validateAzureDataLakeProperties(Map<String, String> configuration, SourceLocation srcLoc,
- IWarningCollector collector, IApplicationContext appCtx) throws CompilationException {
-
- // check if the format property is present
- if (configuration.get(ExternalDataConstants.KEY_FORMAT) == null) {
- throw new CompilationException(ErrorCode.PARAMETERS_REQUIRED, srcLoc, ExternalDataConstants.KEY_FORMAT);
- }
-
- validateIncludeExclude(configuration);
-
- // Check if the bucket is present
- DataLakeServiceClient dataLakeServiceClient;
- try {
- String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
- dataLakeServiceClient = buildAzureDatalakeClient(appCtx, configuration);
- DataLakeFileSystemClient fileSystemClient = dataLakeServiceClient.getFileSystemClient(container);
-
- // Get all objects in a container and extract the paths to files
- ListPathsOptions listPathsOptions = new ListPathsOptions();
- listPathsOptions.setPath(getPrefix(configuration));
- Iterable<PathItem> blobItems = fileSystemClient.listPaths(listPathsOptions, null);
-
- if (!blobItems.iterator().hasNext() && collector.shouldWarn()) {
- Warning warning = Warning.of(srcLoc, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
- collector.warn(warning);
- }
- } catch (CompilationException ex) {
- throw ex;
- } catch (Exception ex) {
- throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
- }
- }
-
- /**
- * Builds the Azure Blob storage client using the provided configuration
- *
- * @param configuration properties
- * @see <a href="https://docs.microsoft.com/en-us/azure/databricks/data/data-sources/azure/azure-storage">Azure
- * Blob storage</a>
- */
- public static void configureAzureHdfsJobConf(JobConf conf, Map<String, String> configuration, String endPoint) {
- String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
- String accountKey = configuration.get(ACCOUNT_KEY_FIELD_NAME);
- String sharedAccessSignature = configuration.get(SHARED_ACCESS_SIGNATURE_FIELD_NAME);
-
- //Disable caching S3 FileSystem
- HDFSUtils.disableHadoopFileSystemCache(conf, HADOOP_AZURE_BLOB_PROTOCOL);
-
- //Key for Hadoop configuration
- StringBuilder hadoopKey = new StringBuilder();
- //Value for Hadoop configuration
- String hadoopValue;
- if (accountKey != null || sharedAccessSignature != null) {
- if (accountKey != null) {
- hadoopKey.append(HADOOP_AZURE_FS_ACCOUNT_KEY).append('.');
- //Set only the AccountKey
- hadoopValue = accountKey;
- } else {
- //Use SAS for Hadoop FS as connectionString is provided
- hadoopKey.append(HADOOP_AZURE_FS_SAS).append('.');
- //Setting the container is required for SAS
- hadoopKey.append(container).append('.');
- //Set the connection string for SAS
- hadoopValue = sharedAccessSignature;
- }
- //Set the endPoint, which includes the AccountName
- hadoopKey.append(endPoint);
- //Tells Hadoop we are reading from Blob Storage
- conf.set(hadoopKey.toString(), hadoopValue);
- }
- }
- }
-
- public static class GCS {
- private GCS() {
- throw new AssertionError("do not instantiate");
-
- }
-
- //TODO(htowaileb): Add validation step similar to other externals, which also checks if empty bucket
- //upon creating the external dataset
-
- /**
- * Builds the client using the provided configuration
- *
- * @param configuration properties
- * @return clientasterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
- * @throws CompilationException CompilationException
- */
- public static Storage buildClient(Map<String, String> configuration) throws CompilationException {
- String jsonCredentials = configuration.get(JSON_CREDENTIALS_FIELD_NAME);
-
- StorageOptions.Builder builder = StorageOptions.newBuilder();
-
- // Use credentials if available
- if (jsonCredentials != null) {
- try (InputStream credentialsStream = new ByteArrayInputStream(jsonCredentials.getBytes())) {
- builder.setCredentials(ServiceAccountCredentials.fromStream(credentialsStream));
- } catch (IOException ex) {
- throw new CompilationException(EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
- }
- }
-
- return builder.build().getService();
- }
-
- /**
- * Validate external dataset properties
- *
- * @param configuration properties
- * @throws CompilationException Compilation exception
- */
- public static void validateProperties(Map<String, String> configuration, SourceLocation srcLoc,
- IWarningCollector collector) throws CompilationException {
-
- // check if the format property is present
- if (configuration.get(ExternalDataConstants.KEY_FORMAT) == null) {
- throw new CompilationException(ErrorCode.PARAMETERS_REQUIRED, srcLoc, ExternalDataConstants.KEY_FORMAT);
- }
-
- // parquet is not supported for google cloud storage
- if (isParquetFormat(configuration)) {
- throw new CompilationException(INVALID_REQ_PARAM_VAL, srcLoc, KEY_FORMAT,
- configuration.get(KEY_FORMAT));
- }
-
- validateIncludeExclude(configuration);
- String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
-
- try {
- BlobListOption limitOption = BlobListOption.pageSize(1);
- BlobListOption prefixOption = BlobListOption.prefix(getPrefix(configuration));
- Storage storage = buildClient(configuration);
- Page<Blob> items = storage.list(container, limitOption, prefixOption);
-
- if (!items.iterateAll().iterator().hasNext() && collector.shouldWarn()) {
- Warning warning = Warning.of(srcLoc, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
- collector.warn(warning);
- }
- } catch (CompilationException ex) {
- throw ex;
- } catch (Exception ex) {
- throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
- }
- }
- }
-
public static int roundUpToNearestFrameSize(int size, int framesize) {
return ((size / framesize) + 1) * framesize;
}
@@ -1911,7 +864,25 @@
return maxArgSz;
}
- private static Optional<String> getFirstNotNull(Map<String, String> configuration, String... parameters) {
+ public static Optional<String> getFirstNotNull(Map<String, String> configuration, String... parameters) {
return Arrays.stream(parameters).filter(field -> configuration.get(field) != null).findFirst();
}
+
+ public static ATypeTag peekArgument(IAType type, IValueReference valueReference, TaggedValuePointable pointy)
+ throws HyracksDataException {
+ ATypeTag tag = type.getTypeTag();
+ if (tag == ATypeTag.ANY) {
+ pointy.set(valueReference);
+ ATypeTag rtTypeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(pointy.getTag());
+ IAType rtType = TypeTagUtil.getBuiltinTypeByTag(rtTypeTag);
+ return MessagePackUtils.peekUnknown(rtType);
+ } else {
+ return MessagePackUtils.peekUnknown(type);
+ }
+ }
+
+ public static void setVoidArgument(ArrayBackedValueStorage argHolder) throws IOException {
+ argHolder.getDataOutput().writeByte(ARRAY16);
+ argHolder.getDataOutput().writeShort((short) 0);
+ }
}
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 9e49d86..3506216 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
@@ -62,6 +62,7 @@
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.network.INetworkSecurityManager;
import org.apache.hyracks.hdfs.scheduler.Scheduler;
import org.apache.parquet.hadoop.ParquetInputFormat;
@@ -72,10 +73,12 @@
public static Scheduler initializeHDFSScheduler(ICCServiceContext serviceCtx) throws HyracksDataException {
ICCContext ccContext = serviceCtx.getCCContext();
+ INetworkSecurityManager networkSecurityManager = serviceCtx.getControllerService().getNetworkSecurityManager();
Scheduler scheduler = null;
try {
scheduler = new Scheduler(ccContext.getClusterControllerInfo().getClientNetAddress(),
- ccContext.getClusterControllerInfo().getClientNetPort());
+ ccContext.getClusterControllerInfo().getClientNetPort(),
+ networkSecurityManager.getSocketChannelFactory());
} catch (HyracksException e) {
throw new RuntimeDataException(ErrorCode.UTIL_HDFS_UTILS_CANNOT_OBTAIN_HDFS_SCHEDULER);
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/aws/s3/S3Constants.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/aws/s3/S3Constants.java
new file mode 100644
index 0000000..79bbbe2
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/aws/s3/S3Constants.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.util.aws.s3;
+
+public class S3Constants {
+ private S3Constants() {
+ throw new AssertionError("do not instantiate");
+ }
+
+ public static final String REGION_FIELD_NAME = "region";
+ public static final String INSTANCE_PROFILE_FIELD_NAME = "instanceProfile";
+ public static final String ACCESS_KEY_ID_FIELD_NAME = "accessKeyId";
+ public static final String SECRET_ACCESS_KEY_FIELD_NAME = "secretAccessKey";
+ public static final String SESSION_TOKEN_FIELD_NAME = "sessionToken";
+ public static final String SERVICE_END_POINT_FIELD_NAME = "serviceEndpoint";
+
+ // AWS S3 specific error codes
+ public static final String ERROR_INTERNAL_ERROR = "InternalError";
+ public static final String ERROR_SLOW_DOWN = "SlowDown";
+ public static final String ERROR_METHOD_NOT_IMPLEMENTED = "NotImplemented";
+
+ /*
+ * Hadoop-AWS
+ * AWS connectors for s3 and s3n are deprecated.
+ */
+ public static final String HADOOP_ACCESS_KEY_ID = "fs.s3a.access.key";
+ public static final String HADOOP_SECRET_ACCESS_KEY = "fs.s3a.secret.key";
+ public static final String HADOOP_SESSION_TOKEN = "fs.s3a.session.token";
+ public static final String HADOOP_REGION = "fs.s3a.region";
+ public static final String HADOOP_SERVICE_END_POINT = "fs.s3a.endpoint";
+
+ /*
+ * Internal configurations
+ */
+ //Allows accessing directories as file system path
+ public static final String HADOOP_PATH_STYLE_ACCESS = "fs.s3a.path.style.access";
+ //The number of maximum HTTP connections in connection pool
+ public static final String HADOOP_S3_CONNECTION_POOL_SIZE = "fs.s3a.connection.maximum";
+ //S3 used protocol
+ public static final String HADOOP_S3_PROTOCOL = "s3a";
+
+ //Hadoop credentials provider key
+ public static final String HADOOP_CREDENTIAL_PROVIDER_KEY = "fs.s3a.aws.credentials.provider";
+ //Anonymous credential provider
+ public static final String HADOOP_ANONYMOUS_ACCESS = "org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider";
+ //Temporary credential provider
+ public static final String HADOOP_TEMP_ACCESS = "org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider";
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/aws/s3/S3Utils.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/aws/s3/S3Utils.java
new file mode 100644
index 0000000..6775bf12b
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/aws/s3/S3Utils.java
@@ -0,0 +1,475 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.util.aws.s3;
+
+import static org.apache.asterix.common.exceptions.ErrorCode.INVALID_PARAM_VALUE_ALLOWED_VALUE;
+import static org.apache.asterix.common.exceptions.ErrorCode.PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT;
+import static org.apache.asterix.common.exceptions.ErrorCode.REQUIRED_PARAM_IF_PARAM_IS_PRESENT;
+import static org.apache.asterix.common.exceptions.ErrorCode.S3_REGION_NOT_SUPPORTED;
+import static org.apache.asterix.external.util.ExternalDataUtils.getPrefix;
+import static org.apache.asterix.external.util.ExternalDataUtils.validateIncludeExclude;
+import static org.apache.asterix.external.util.aws.s3.S3Constants.*;
+import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiPredicate;
+import java.util.regex.Matcher;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStreamFactory;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.HDFSUtils;
+import org.apache.hadoop.fs.s3a.Constants;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.api.exceptions.Warning;
+import org.apache.hyracks.api.util.CleanupUtils;
+
+import software.amazon.awssdk.auth.credentials.AnonymousCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.AwsSessionCredentials;
+import software.amazon.awssdk.auth.credentials.InstanceProfileCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.exception.SdkException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.s3.S3Client;
+import software.amazon.awssdk.services.s3.S3ClientBuilder;
+import software.amazon.awssdk.services.s3.model.ListObjectsRequest;
+import software.amazon.awssdk.services.s3.model.ListObjectsResponse;
+import software.amazon.awssdk.services.s3.model.ListObjectsV2Request;
+import software.amazon.awssdk.services.s3.model.ListObjectsV2Response;
+import software.amazon.awssdk.services.s3.model.S3Exception;
+import software.amazon.awssdk.services.s3.model.S3Object;
+import software.amazon.awssdk.services.s3.model.S3Response;
+
+public class S3Utils {
+ private S3Utils() {
+ throw new AssertionError("do not instantiate");
+ }
+
+ public static boolean isRetryableError(String errorCode) {
+ return errorCode.equals(ERROR_INTERNAL_ERROR) || errorCode.equals(ERROR_SLOW_DOWN);
+ }
+
+ /**
+ * Builds the S3 client using the provided configuration
+ *
+ * @param configuration properties
+ * @return S3 client
+ * @throws CompilationException CompilationException
+ */
+ public static S3Client buildAwsS3Client(Map<String, String> configuration) throws CompilationException {
+ // TODO(Hussain): Need to ensure that all required parameters are present in a previous step
+ String instanceProfile = configuration.get(INSTANCE_PROFILE_FIELD_NAME);
+ String accessKeyId = configuration.get(ACCESS_KEY_ID_FIELD_NAME);
+ String secretAccessKey = configuration.get(SECRET_ACCESS_KEY_FIELD_NAME);
+ String sessionToken = configuration.get(SESSION_TOKEN_FIELD_NAME);
+ String regionId = configuration.get(REGION_FIELD_NAME);
+ String serviceEndpoint = configuration.get(SERVICE_END_POINT_FIELD_NAME);
+
+ S3ClientBuilder builder = S3Client.builder();
+
+ // Credentials
+ AwsCredentialsProvider credentialsProvider;
+
+ // nothing provided, anonymous authentication
+ if (instanceProfile == null && accessKeyId == null && secretAccessKey == null && sessionToken == null) {
+ credentialsProvider = AnonymousCredentialsProvider.create();
+ } else if (instanceProfile != null) {
+
+ // only "true" value is allowed
+ if (!instanceProfile.equalsIgnoreCase("true")) {
+ throw new CompilationException(INVALID_PARAM_VALUE_ALLOWED_VALUE, INSTANCE_PROFILE_FIELD_NAME, "true");
+ }
+
+ // no other authentication parameters are allowed
+ if (accessKeyId != null) {
+ throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, ACCESS_KEY_ID_FIELD_NAME,
+ INSTANCE_PROFILE_FIELD_NAME);
+ }
+ if (secretAccessKey != null) {
+ throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, SECRET_ACCESS_KEY_FIELD_NAME,
+ INSTANCE_PROFILE_FIELD_NAME);
+ }
+ if (sessionToken != null) {
+ throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, SESSION_TOKEN_FIELD_NAME,
+ INSTANCE_PROFILE_FIELD_NAME);
+ }
+ credentialsProvider = InstanceProfileCredentialsProvider.create();
+ } else if (accessKeyId != null || secretAccessKey != null) {
+ // accessKeyId authentication
+ if (accessKeyId == null) {
+ throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCESS_KEY_ID_FIELD_NAME,
+ SECRET_ACCESS_KEY_FIELD_NAME);
+ }
+ if (secretAccessKey == null) {
+ throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, SECRET_ACCESS_KEY_FIELD_NAME,
+ ACCESS_KEY_ID_FIELD_NAME);
+ }
+
+ // use session token if provided
+ if (sessionToken != null) {
+ credentialsProvider = StaticCredentialsProvider
+ .create(AwsSessionCredentials.create(accessKeyId, secretAccessKey, sessionToken));
+ } else {
+ credentialsProvider =
+ StaticCredentialsProvider.create(AwsBasicCredentials.create(accessKeyId, secretAccessKey));
+ }
+ } else {
+ // if only session token is provided, accessKeyId is required
+ throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCESS_KEY_ID_FIELD_NAME,
+ SESSION_TOKEN_FIELD_NAME);
+ }
+
+ builder.credentialsProvider(credentialsProvider);
+
+ // Validate the region
+ List<Region> regions = S3Client.serviceMetadata().regions();
+ Optional<Region> selectedRegion = regions.stream().filter(region -> region.id().equals(regionId)).findFirst();
+
+ if (selectedRegion.isEmpty()) {
+ throw new CompilationException(S3_REGION_NOT_SUPPORTED, regionId);
+ }
+ builder.region(selectedRegion.get());
+
+ // Validate the service endpoint if present
+ if (serviceEndpoint != null) {
+ try {
+ URI uri = new URI(serviceEndpoint);
+ try {
+ builder.endpointOverride(uri);
+ } catch (NullPointerException ex) {
+ throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+ }
+ } catch (URISyntaxException ex) {
+ throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR,
+ String.format("Invalid service endpoint %s", serviceEndpoint));
+ }
+ }
+
+ return builder.build();
+ }
+
+ /**
+ * Builds the S3 client using the provided configuration
+ *
+ * @param configuration properties
+ * @param numberOfPartitions number of partitions in the cluster
+ */
+ public static void configureAwsS3HdfsJobConf(JobConf conf, Map<String, String> configuration,
+ int numberOfPartitions) {
+ String accessKeyId = configuration.get(ACCESS_KEY_ID_FIELD_NAME);
+ String secretAccessKey = configuration.get(SECRET_ACCESS_KEY_FIELD_NAME);
+ String sessionToken = configuration.get(SESSION_TOKEN_FIELD_NAME);
+ String serviceEndpoint = configuration.get(SERVICE_END_POINT_FIELD_NAME);
+
+ //Disable caching S3 FileSystem
+ HDFSUtils.disableHadoopFileSystemCache(conf, HADOOP_S3_PROTOCOL);
+
+ /*
+ * Authentication Methods:
+ * 1- Anonymous: no accessKeyId and no secretAccessKey
+ * 2- Temporary: has to provide accessKeyId, secretAccessKey and sessionToken
+ * 3- Private: has to provide accessKeyId and secretAccessKey
+ */
+ if (accessKeyId == null) {
+ //Tells hadoop-aws it is an anonymous access
+ conf.set(HADOOP_CREDENTIAL_PROVIDER_KEY, HADOOP_ANONYMOUS_ACCESS);
+ } else {
+ conf.set(HADOOP_ACCESS_KEY_ID, accessKeyId);
+ conf.set(HADOOP_SECRET_ACCESS_KEY, secretAccessKey);
+ if (sessionToken != null) {
+ conf.set(HADOOP_SESSION_TOKEN, sessionToken);
+ //Tells hadoop-aws it is a temporary access
+ conf.set(HADOOP_CREDENTIAL_PROVIDER_KEY, HADOOP_TEMP_ACCESS);
+ }
+ }
+
+ /*
+ * This is to allow S3 definition to have path-style form. Should always be true to match the current
+ * way we access files in S3
+ */
+ conf.set(HADOOP_PATH_STYLE_ACCESS, ExternalDataConstants.TRUE);
+
+ /*
+ * Set the size of S3 connection pool to be the number of partitions
+ */
+ conf.set(HADOOP_S3_CONNECTION_POOL_SIZE, String.valueOf(numberOfPartitions));
+
+ if (serviceEndpoint != null) {
+ // Validation of the URL should be done at hadoop-aws level
+ conf.set(HADOOP_SERVICE_END_POINT, serviceEndpoint);
+ } else {
+ //Region is ignored and buckets could be found by the central endpoint
+ conf.set(HADOOP_SERVICE_END_POINT, Constants.CENTRAL_ENDPOINT);
+ }
+ }
+
+ /**
+ * Validate external dataset properties
+ *
+ * @param configuration properties
+ * @throws CompilationException Compilation exception
+ */
+ public static void validateProperties(Map<String, String> configuration, SourceLocation srcLoc,
+ IWarningCollector collector) throws CompilationException {
+
+ // check if the format property is present
+ if (configuration.get(ExternalDataConstants.KEY_FORMAT) == null) {
+ throw new CompilationException(ErrorCode.PARAMETERS_REQUIRED, srcLoc, ExternalDataConstants.KEY_FORMAT);
+ }
+
+ // Both parameters should be passed, or neither should be passed (for anonymous/no auth)
+ String accessKeyId = configuration.get(ACCESS_KEY_ID_FIELD_NAME);
+ String secretAccessKey = configuration.get(SECRET_ACCESS_KEY_FIELD_NAME);
+ if (accessKeyId == null || secretAccessKey == null) {
+ // If one is passed, the other is required
+ if (accessKeyId != null) {
+ throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, SECRET_ACCESS_KEY_FIELD_NAME,
+ ACCESS_KEY_ID_FIELD_NAME);
+ } else if (secretAccessKey != null) {
+ throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCESS_KEY_ID_FIELD_NAME,
+ SECRET_ACCESS_KEY_FIELD_NAME);
+ }
+ }
+
+ validateIncludeExclude(configuration);
+
+ // Check if the bucket is present
+ S3Client s3Client = buildAwsS3Client(configuration);
+ S3Response response;
+ boolean useOldApi = false;
+ String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
+ String prefix = getPrefix(configuration);
+
+ try {
+ response = isBucketEmpty(s3Client, container, prefix, false);
+ } catch (S3Exception ex) {
+ // Method not implemented, try falling back to old API
+ try {
+ // For error code, see https://docs.aws.amazon.com/AmazonS3/latest/API/ErrorResponses.html
+ if (ex.awsErrorDetails().errorCode().equals(ERROR_METHOD_NOT_IMPLEMENTED)) {
+ useOldApi = true;
+ response = isBucketEmpty(s3Client, container, prefix, true);
+ } else {
+ throw ex;
+ }
+ } catch (SdkException ex2) {
+ throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+ }
+ } catch (SdkException ex) {
+ throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+ } finally {
+ if (s3Client != null) {
+ CleanupUtils.close(s3Client, null);
+ }
+ }
+
+ boolean isEmpty = useOldApi ? ((ListObjectsResponse) response).contents().isEmpty()
+ : ((ListObjectsV2Response) response).contents().isEmpty();
+ if (isEmpty && collector.shouldWarn()) {
+ Warning warning = Warning.of(srcLoc, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
+ collector.warn(warning);
+ }
+
+ // Returns 200 only in case the bucket exists, otherwise, throws an exception. However, to
+ // ensure coverage, check if the result is successful as well and not only catch exceptions
+ if (!response.sdkHttpResponse().isSuccessful()) {
+ throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_CONTAINER_NOT_FOUND, container);
+ }
+ }
+
+ /**
+ * Checks for a single object in the specified bucket to determine if the bucket is empty or not.
+ *
+ * @param s3Client s3 client
+ * @param container the container name
+ * @param prefix Prefix to be used
+ * @param useOldApi flag whether to use the old API or not
+ * @return returns the S3 response
+ */
+ private static S3Response isBucketEmpty(S3Client s3Client, String container, String prefix, boolean useOldApi) {
+ S3Response response;
+ if (useOldApi) {
+ ListObjectsRequest.Builder listObjectsBuilder = ListObjectsRequest.builder();
+ listObjectsBuilder.prefix(prefix);
+ response = s3Client.listObjects(listObjectsBuilder.bucket(container).maxKeys(1).build());
+ } else {
+ ListObjectsV2Request.Builder listObjectsBuilder = ListObjectsV2Request.builder();
+ listObjectsBuilder.prefix(prefix);
+ response = s3Client.listObjectsV2(listObjectsBuilder.bucket(container).maxKeys(1).build());
+ }
+ return response;
+ }
+
+ /**
+ * Returns the lists of S3 objects.
+ *
+ * @param configuration properties
+ * @param includeExcludeMatcher include/exclude matchers to apply
+ */
+ public static List<S3Object> listS3Objects(Map<String, String> configuration,
+ AbstractExternalInputStreamFactory.IncludeExcludeMatcher includeExcludeMatcher,
+ IWarningCollector warningCollector) throws CompilationException {
+ // Prepare to retrieve the objects
+ List<S3Object> filesOnly;
+ String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
+ S3Client s3Client = buildAwsS3Client(configuration);
+ String prefix = getPrefix(configuration);
+
+ try {
+ filesOnly = listS3Objects(s3Client, container, prefix, includeExcludeMatcher);
+ } catch (S3Exception ex) {
+ // New API is not implemented, try falling back to old API
+ try {
+ // For error code, see https://docs.aws.amazon.com/AmazonS3/latest/API/ErrorResponses.html
+ if (ex.awsErrorDetails().errorCode().equals(ERROR_METHOD_NOT_IMPLEMENTED)) {
+ filesOnly = oldApiListS3Objects(s3Client, container, prefix, includeExcludeMatcher);
+ } else {
+ throw ex;
+ }
+ } catch (SdkException ex2) {
+ throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+ }
+ } catch (SdkException ex) {
+ throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+ } finally {
+ if (s3Client != null) {
+ CleanupUtils.close(s3Client, null);
+ }
+ }
+
+ // Warn if no files are returned
+ if (filesOnly.isEmpty() && warningCollector.shouldWarn()) {
+ Warning warning = Warning.of(null, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
+ warningCollector.warn(warning);
+ }
+
+ return filesOnly;
+ }
+
+ /**
+ * Uses the latest API to retrieve the objects from the storage.
+ *
+ * @param s3Client S3 client
+ * @param container container name
+ * @param prefix definition prefix
+ * @param includeExcludeMatcher include/exclude matchers to apply
+ */
+ private static List<S3Object> listS3Objects(S3Client s3Client, String container, String prefix,
+ AbstractExternalInputStreamFactory.IncludeExcludeMatcher includeExcludeMatcher) {
+ String newMarker = null;
+ List<S3Object> filesOnly = new ArrayList<>();
+
+ ListObjectsV2Response listObjectsResponse;
+ ListObjectsV2Request.Builder listObjectsBuilder = ListObjectsV2Request.builder().bucket(container);
+ listObjectsBuilder.prefix(prefix);
+
+ while (true) {
+ // List the objects from the start, or from the last marker in case of truncated result
+ if (newMarker == null) {
+ listObjectsResponse = s3Client.listObjectsV2(listObjectsBuilder.build());
+ } else {
+ listObjectsResponse = s3Client.listObjectsV2(listObjectsBuilder.continuationToken(newMarker).build());
+ }
+
+ // Collect the paths to files only
+ collectAndFilterFiles(listObjectsResponse.contents(), includeExcludeMatcher.getPredicate(),
+ includeExcludeMatcher.getMatchersList(), filesOnly);
+
+ // Mark the flag as done if done, otherwise, get the marker of the previous response for the next request
+ if (!listObjectsResponse.isTruncated()) {
+ break;
+ } else {
+ newMarker = listObjectsResponse.nextContinuationToken();
+ }
+ }
+
+ return filesOnly;
+ }
+
+ /**
+ * Uses the old API (in case the new API is not implemented) to retrieve the objects from the storage
+ *
+ * @param s3Client S3 client
+ * @param container container name
+ * @param prefix definition prefix
+ * @param includeExcludeMatcher include/exclude matchers to apply
+ */
+ private static List<S3Object> oldApiListS3Objects(S3Client s3Client, String container, String prefix,
+ AbstractExternalInputStreamFactory.IncludeExcludeMatcher includeExcludeMatcher) {
+ String newMarker = null;
+ List<S3Object> filesOnly = new ArrayList<>();
+
+ ListObjectsResponse listObjectsResponse;
+ ListObjectsRequest.Builder listObjectsBuilder = ListObjectsRequest.builder().bucket(container);
+ listObjectsBuilder.prefix(prefix);
+
+ while (true) {
+ // List the objects from the start, or from the last marker in case of truncated result
+ if (newMarker == null) {
+ listObjectsResponse = s3Client.listObjects(listObjectsBuilder.build());
+ } else {
+ listObjectsResponse = s3Client.listObjects(listObjectsBuilder.marker(newMarker).build());
+ }
+
+ // Collect the paths to files only
+ collectAndFilterFiles(listObjectsResponse.contents(), includeExcludeMatcher.getPredicate(),
+ includeExcludeMatcher.getMatchersList(), filesOnly);
+
+ // Mark the flag as done if done, otherwise, get the marker of the previous response for the next request
+ if (!listObjectsResponse.isTruncated()) {
+ break;
+ } else {
+ newMarker = listObjectsResponse.nextMarker();
+ }
+ }
+
+ return filesOnly;
+ }
+
+ /**
+ * AWS S3 returns all the objects as paths, not differentiating between folder and files. The path is considered
+ * a file if it does not end up with a "/" which is the separator in a folder structure.
+ *
+ * @param s3Objects List of returned objects
+ */
+ private static void collectAndFilterFiles(List<S3Object> s3Objects, BiPredicate<List<Matcher>, String> predicate,
+ List<Matcher> matchers, List<S3Object> filesOnly) {
+ for (S3Object object : s3Objects) {
+ // skip folders
+ if (object.key().endsWith("/")) {
+ continue;
+ }
+
+ // No filter, add file
+ if (predicate.test(matchers, object.key())) {
+ filesOnly.add(object);
+ }
+ }
+ }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/azure/blob_storage/AzureConstants.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/azure/blob_storage/AzureConstants.java
new file mode 100644
index 0000000..9ade27b
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/azure/blob_storage/AzureConstants.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.util.azure.blob_storage;
+
+/*
+ * Note: Azure Blob and Azure Datalake use identical authentication, so they are using the same properties.
+ * If they end up diverging, then properties for AzureBlob and AzureDataLake need to be created.
+ */
+public class AzureConstants {
+ private AzureConstants() {
+ throw new AssertionError("do not instantiate");
+ }
+
+ /*
+ * Asterix Configuration Keys
+ */
+ public static final String MANAGED_IDENTITY_ID_FIELD_NAME = "managedIdentityId";
+ public static final String ACCOUNT_NAME_FIELD_NAME = "accountName";
+ public static final String ACCOUNT_KEY_FIELD_NAME = "accountKey";
+ public static final String SHARED_ACCESS_SIGNATURE_FIELD_NAME = "sharedAccessSignature";
+ public static final String TENANT_ID_FIELD_NAME = "tenantId";
+ public static final String CLIENT_ID_FIELD_NAME = "clientId";
+ public static final String CLIENT_SECRET_FIELD_NAME = "clientSecret";
+ public static final String CLIENT_CERTIFICATE_FIELD_NAME = "clientCertificate";
+ public static final String CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME = "clientCertificatePassword";
+ public static final String ENDPOINT_FIELD_NAME = "endpoint";
+
+ // Specific Azure data lake property
+ /*
+ The behavior of Data Lake (true file system) is to read the files of the specified prefix only, example:
+ storage/myData/personal/file1.json
+ storage/myData/personal/file2.json
+ storage/myData/file3.json
+ If the prefix used is "myData", then only the file file3.json is read. However, if the property "recursive"
+ is set to "true" when creating the external dataset, then it goes recursively overall the paths, and the result
+ is file1.json, file2.json and file3.json.
+ */
+ public static final String RECURSIVE_FIELD_NAME = "recursive";
+
+ /*
+ * Hadoop-Azure
+ */
+ //Used when accountName and accessKey are provided
+ public static final String HADOOP_AZURE_FS_ACCOUNT_KEY = "fs.azure.account.key";
+ //Used when a connectionString is provided
+ public static final String HADOOP_AZURE_FS_SAS = "fs.azure.sas";
+ public static final String HADOOP_AZURE_BLOB_PROTOCOL = "wasbs";
+ public static final String HADOOP_AZURE_DATALAKE_PROTOCOL = "abfss";
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/azure/blob_storage/AzureUtils.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/azure/blob_storage/AzureUtils.java
new file mode 100644
index 0000000..0dc9ad2
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/azure/blob_storage/AzureUtils.java
@@ -0,0 +1,636 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.util.azure.blob_storage;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.asterix.common.exceptions.ErrorCode.EXTERNAL_SOURCE_ERROR;
+import static org.apache.asterix.common.exceptions.ErrorCode.PARAMETERS_NOT_ALLOWED_AT_SAME_TIME;
+import static org.apache.asterix.common.exceptions.ErrorCode.PARAMETERS_REQUIRED;
+import static org.apache.asterix.common.exceptions.ErrorCode.PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT;
+import static org.apache.asterix.common.exceptions.ErrorCode.REQUIRED_PARAM_IF_PARAM_IS_PRESENT;
+import static org.apache.asterix.common.exceptions.ErrorCode.REQUIRED_PARAM_OR_PARAM_IF_PARAM_IS_PRESENT;
+import static org.apache.asterix.external.util.ExternalDataUtils.getFirstNotNull;
+import static org.apache.asterix.external.util.ExternalDataUtils.getPrefix;
+import static org.apache.asterix.external.util.ExternalDataUtils.validateIncludeExclude;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.ACCOUNT_KEY_FIELD_NAME;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.ACCOUNT_NAME_FIELD_NAME;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.CLIENT_CERTIFICATE_FIELD_NAME;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.CLIENT_ID_FIELD_NAME;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.CLIENT_SECRET_FIELD_NAME;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.ENDPOINT_FIELD_NAME;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.HADOOP_AZURE_BLOB_PROTOCOL;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.HADOOP_AZURE_FS_ACCOUNT_KEY;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.HADOOP_AZURE_FS_SAS;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.MANAGED_IDENTITY_ID_FIELD_NAME;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.RECURSIVE_FIELD_NAME;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.SHARED_ACCESS_SIGNATURE_FIELD_NAME;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.TENANT_ID_FIELD_NAME;
+import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
+
+import java.io.ByteArrayInputStream;
+import java.io.InputStream;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiPredicate;
+import java.util.regex.Matcher;
+
+import org.apache.asterix.common.api.IApplicationContext;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStreamFactory;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.HDFSUtils;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.api.exceptions.Warning;
+
+import com.azure.core.credential.AzureSasCredential;
+import com.azure.core.http.rest.PagedIterable;
+import com.azure.identity.ClientCertificateCredentialBuilder;
+import com.azure.identity.ClientSecretCredentialBuilder;
+import com.azure.identity.ManagedIdentityCredentialBuilder;
+import com.azure.storage.blob.BlobContainerClient;
+import com.azure.storage.blob.BlobServiceClient;
+import com.azure.storage.blob.BlobServiceClientBuilder;
+import com.azure.storage.blob.models.BlobItem;
+import com.azure.storage.blob.models.ListBlobsOptions;
+import com.azure.storage.common.StorageSharedKeyCredential;
+import com.azure.storage.common.policy.RequestRetryOptions;
+import com.azure.storage.file.datalake.DataLakeFileSystemClient;
+import com.azure.storage.file.datalake.DataLakeServiceClient;
+import com.azure.storage.file.datalake.DataLakeServiceClientBuilder;
+import com.azure.storage.file.datalake.models.ListPathsOptions;
+import com.azure.storage.file.datalake.models.PathItem;
+
+public class AzureUtils {
+ private AzureUtils() {
+ throw new AssertionError("do not instantiate");
+ }
+
+ /**
+ * Builds the Azure storage account using the provided configuration
+ *
+ * @param configuration properties
+ * @return client
+ */
+ public static BlobServiceClient buildAzureBlobClient(IApplicationContext appCtx, Map<String, String> configuration)
+ throws CompilationException {
+ String managedIdentityId = configuration.get(MANAGED_IDENTITY_ID_FIELD_NAME);
+ String accountName = configuration.get(ACCOUNT_NAME_FIELD_NAME);
+ String accountKey = configuration.get(ACCOUNT_KEY_FIELD_NAME);
+ String sharedAccessSignature = configuration.get(SHARED_ACCESS_SIGNATURE_FIELD_NAME);
+ String tenantId = configuration.get(TENANT_ID_FIELD_NAME);
+ String clientId = configuration.get(CLIENT_ID_FIELD_NAME);
+ String clientSecret = configuration.get(CLIENT_SECRET_FIELD_NAME);
+ String clientCertificate = configuration.get(CLIENT_CERTIFICATE_FIELD_NAME);
+ String clientCertificatePassword = configuration.get(CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME);
+ String endpoint = configuration.get(ENDPOINT_FIELD_NAME);
+
+ // Client builder
+ BlobServiceClientBuilder builder = new BlobServiceClientBuilder();
+ int timeout = appCtx.getExternalProperties().getAzureRequestTimeout();
+ RequestRetryOptions requestRetryOptions = new RequestRetryOptions(null, null, timeout, null, null, null);
+ builder.retryOptions(requestRetryOptions);
+
+ // Endpoint is required
+ if (endpoint == null) {
+ throw new CompilationException(PARAMETERS_REQUIRED, ENDPOINT_FIELD_NAME);
+ }
+ builder.endpoint(endpoint);
+
+ // Shared Key
+ if (accountName != null || accountKey != null) {
+ if (accountName == null) {
+ throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCOUNT_NAME_FIELD_NAME,
+ ACCOUNT_KEY_FIELD_NAME);
+ }
+
+ if (accountKey == null) {
+ throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCOUNT_KEY_FIELD_NAME,
+ ACCOUNT_NAME_FIELD_NAME);
+ }
+
+ Optional<String> provided = getFirstNotNull(configuration, SHARED_ACCESS_SIGNATURE_FIELD_NAME,
+ MANAGED_IDENTITY_ID_FIELD_NAME, CLIENT_ID_FIELD_NAME, CLIENT_SECRET_FIELD_NAME,
+ CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
+ if (provided.isPresent()) {
+ throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
+ ACCOUNT_KEY_FIELD_NAME);
+ }
+ StorageSharedKeyCredential credential = new StorageSharedKeyCredential(accountName, accountKey);
+ builder.credential(credential);
+ }
+
+ // Shared access signature
+ if (sharedAccessSignature != null) {
+ Optional<String> provided = getFirstNotNull(configuration, MANAGED_IDENTITY_ID_FIELD_NAME,
+ CLIENT_ID_FIELD_NAME, CLIENT_SECRET_FIELD_NAME, CLIENT_CERTIFICATE_FIELD_NAME,
+ CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
+ if (provided.isPresent()) {
+ throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
+ SHARED_ACCESS_SIGNATURE_FIELD_NAME);
+ }
+ AzureSasCredential credential = new AzureSasCredential(sharedAccessSignature);
+ builder.credential(credential);
+ }
+
+ // Managed Identity auth
+ if (managedIdentityId != null) {
+ Optional<String> provided = getFirstNotNull(configuration, CLIENT_ID_FIELD_NAME, CLIENT_SECRET_FIELD_NAME,
+ CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
+ if (provided.isPresent()) {
+ throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
+ MANAGED_IDENTITY_ID_FIELD_NAME);
+ }
+ builder.credential(new ManagedIdentityCredentialBuilder().clientId(managedIdentityId).build());
+ }
+
+ // Client secret & certificate auth
+ if (clientId != null) {
+ // Both (or neither) client secret and client secret were provided, only one is allowed
+ if ((clientSecret == null) == (clientCertificate == null)) {
+ if (clientSecret != null) {
+ throw new CompilationException(PARAMETERS_NOT_ALLOWED_AT_SAME_TIME, CLIENT_SECRET_FIELD_NAME,
+ CLIENT_CERTIFICATE_FIELD_NAME);
+ } else {
+ throw new CompilationException(REQUIRED_PARAM_OR_PARAM_IF_PARAM_IS_PRESENT,
+ CLIENT_SECRET_FIELD_NAME, CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_ID_FIELD_NAME);
+ }
+ }
+
+ // Tenant ID is required
+ if (tenantId == null) {
+ throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, TENANT_ID_FIELD_NAME,
+ CLIENT_ID_FIELD_NAME);
+ }
+
+ // Client certificate password is not allowed if client secret is used
+ if (clientCertificatePassword != null && clientSecret != null) {
+ throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT,
+ CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, CLIENT_SECRET_FIELD_NAME);
+ }
+
+ // Use AD authentication
+ if (clientSecret != null) {
+ ClientSecretCredentialBuilder secret = new ClientSecretCredentialBuilder();
+ secret.clientId(clientId);
+ secret.tenantId(tenantId);
+ secret.clientSecret(clientSecret);
+ builder.credential(secret.build());
+ } else {
+ // Certificate
+ ClientCertificateCredentialBuilder certificate = new ClientCertificateCredentialBuilder();
+ certificate.clientId(clientId);
+ certificate.tenantId(tenantId);
+ try {
+ InputStream certificateContent = new ByteArrayInputStream(clientCertificate.getBytes(UTF_8));
+ if (clientCertificatePassword == null) {
+ Method pemCertificate = ClientCertificateCredentialBuilder.class
+ .getDeclaredMethod("pemCertificate", InputStream.class);
+ pemCertificate.setAccessible(true);
+ pemCertificate.invoke(certificate, certificateContent);
+ } else {
+ Method pemCertificate = ClientCertificateCredentialBuilder.class
+ .getDeclaredMethod("pfxCertificate", InputStream.class, String.class);
+ pemCertificate.setAccessible(true);
+ pemCertificate.invoke(certificate, certificateContent, clientCertificatePassword);
+ }
+ } catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException ex) {
+ throw new CompilationException(EXTERNAL_SOURCE_ERROR, ex.getMessage());
+ }
+ builder.credential(certificate.build());
+ }
+ }
+
+ // If client id is not present, ensure client secret, certificate, tenant id and client certificate
+ // password are not present
+ if (clientId == null) {
+ Optional<String> provided = getFirstNotNull(configuration, CLIENT_SECRET_FIELD_NAME,
+ CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
+ if (provided.isPresent()) {
+ throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
+ SHARED_ACCESS_SIGNATURE_FIELD_NAME);
+ }
+ }
+
+ try {
+ return builder.buildClient();
+ } catch (Exception ex) {
+ throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+ }
+ }
+
+ /**
+ * Builds the Azure data lake storage account using the provided configuration
+ *
+ * @param configuration properties
+ * @return client
+ */
+ public static DataLakeServiceClient buildAzureDatalakeClient(IApplicationContext appCtx,
+ Map<String, String> configuration) throws CompilationException {
+ String managedIdentityId = configuration.get(MANAGED_IDENTITY_ID_FIELD_NAME);
+ String accountName = configuration.get(ACCOUNT_NAME_FIELD_NAME);
+ String accountKey = configuration.get(ACCOUNT_KEY_FIELD_NAME);
+ String sharedAccessSignature = configuration.get(SHARED_ACCESS_SIGNATURE_FIELD_NAME);
+ String tenantId = configuration.get(TENANT_ID_FIELD_NAME);
+ String clientId = configuration.get(CLIENT_ID_FIELD_NAME);
+ String clientSecret = configuration.get(CLIENT_SECRET_FIELD_NAME);
+ String clientCertificate = configuration.get(CLIENT_CERTIFICATE_FIELD_NAME);
+ String clientCertificatePassword = configuration.get(CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME);
+ String endpoint = configuration.get(ENDPOINT_FIELD_NAME);
+
+ // Client builder
+ DataLakeServiceClientBuilder builder = new DataLakeServiceClientBuilder();
+ int timeout = appCtx.getExternalProperties().getAzureRequestTimeout();
+ RequestRetryOptions requestRetryOptions = new RequestRetryOptions(null, null, timeout, null, null, null);
+ builder.retryOptions(requestRetryOptions);
+
+ // Endpoint is required
+ if (endpoint == null) {
+ throw new CompilationException(PARAMETERS_REQUIRED, ENDPOINT_FIELD_NAME);
+ }
+ builder.endpoint(endpoint);
+
+ // Shared Key
+ if (accountName != null || accountKey != null) {
+ if (accountName == null) {
+ throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCOUNT_NAME_FIELD_NAME,
+ ACCOUNT_KEY_FIELD_NAME);
+ }
+
+ if (accountKey == null) {
+ throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCOUNT_KEY_FIELD_NAME,
+ ACCOUNT_NAME_FIELD_NAME);
+ }
+
+ Optional<String> provided = getFirstNotNull(configuration, SHARED_ACCESS_SIGNATURE_FIELD_NAME,
+ MANAGED_IDENTITY_ID_FIELD_NAME, CLIENT_ID_FIELD_NAME, CLIENT_SECRET_FIELD_NAME,
+ CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
+ if (provided.isPresent()) {
+ throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
+ ACCOUNT_KEY_FIELD_NAME);
+ }
+ StorageSharedKeyCredential credential = new StorageSharedKeyCredential(accountName, accountKey);
+ builder.credential(credential);
+ }
+
+ // Shared access signature
+ if (sharedAccessSignature != null) {
+ Optional<String> provided = getFirstNotNull(configuration, MANAGED_IDENTITY_ID_FIELD_NAME,
+ CLIENT_ID_FIELD_NAME, CLIENT_SECRET_FIELD_NAME, CLIENT_CERTIFICATE_FIELD_NAME,
+ CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
+ if (provided.isPresent()) {
+ throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
+ SHARED_ACCESS_SIGNATURE_FIELD_NAME);
+ }
+ AzureSasCredential credential = new AzureSasCredential(sharedAccessSignature);
+ builder.credential(credential);
+ }
+
+ // Managed Identity auth
+ if (managedIdentityId != null) {
+ Optional<String> provided = getFirstNotNull(configuration, CLIENT_ID_FIELD_NAME, CLIENT_SECRET_FIELD_NAME,
+ CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
+ if (provided.isPresent()) {
+ throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
+ MANAGED_IDENTITY_ID_FIELD_NAME);
+ }
+ builder.credential(new ManagedIdentityCredentialBuilder().clientId(managedIdentityId).build());
+ }
+
+ // Client secret & certificate auth
+ if (clientId != null) {
+ // Both (or neither) client secret and client secret were provided, only one is allowed
+ if ((clientSecret == null) == (clientCertificate == null)) {
+ if (clientSecret != null) {
+ throw new CompilationException(PARAMETERS_NOT_ALLOWED_AT_SAME_TIME, CLIENT_SECRET_FIELD_NAME,
+ CLIENT_CERTIFICATE_FIELD_NAME);
+ } else {
+ throw new CompilationException(REQUIRED_PARAM_OR_PARAM_IF_PARAM_IS_PRESENT,
+ CLIENT_SECRET_FIELD_NAME, CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_ID_FIELD_NAME);
+ }
+ }
+
+ // Tenant ID is required
+ if (tenantId == null) {
+ throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, TENANT_ID_FIELD_NAME,
+ CLIENT_ID_FIELD_NAME);
+ }
+
+ // Client certificate password is not allowed if client secret is used
+ if (clientCertificatePassword != null && clientSecret != null) {
+ throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT,
+ CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, CLIENT_SECRET_FIELD_NAME);
+ }
+
+ // Use AD authentication
+ if (clientSecret != null) {
+ ClientSecretCredentialBuilder secret = new ClientSecretCredentialBuilder();
+ secret.clientId(clientId);
+ secret.tenantId(tenantId);
+ secret.clientSecret(clientSecret);
+ builder.credential(secret.build());
+ } else {
+ // Certificate
+ ClientCertificateCredentialBuilder certificate = new ClientCertificateCredentialBuilder();
+ certificate.clientId(clientId);
+ certificate.tenantId(tenantId);
+ try {
+ InputStream certificateContent = new ByteArrayInputStream(clientCertificate.getBytes(UTF_8));
+ if (clientCertificatePassword == null) {
+ Method pemCertificate = ClientCertificateCredentialBuilder.class
+ .getDeclaredMethod("pemCertificate", InputStream.class);
+ pemCertificate.setAccessible(true);
+ pemCertificate.invoke(certificate, certificateContent);
+ } else {
+ Method pemCertificate = ClientCertificateCredentialBuilder.class
+ .getDeclaredMethod("pfxCertificate", InputStream.class, String.class);
+ pemCertificate.setAccessible(true);
+ pemCertificate.invoke(certificate, certificateContent, clientCertificatePassword);
+ }
+ } catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException ex) {
+ throw new CompilationException(EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+ }
+ builder.credential(certificate.build());
+ }
+ }
+
+ // If client id is not present, ensure client secret, certificate, tenant id and client certificate
+ // password are not present
+ if (clientId == null) {
+ Optional<String> provided = getFirstNotNull(configuration, CLIENT_SECRET_FIELD_NAME,
+ CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
+ if (provided.isPresent()) {
+ throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
+ SHARED_ACCESS_SIGNATURE_FIELD_NAME);
+ }
+ }
+
+ try {
+ return builder.buildClient();
+ } catch (Exception ex) {
+ throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+ }
+ }
+
+ public static List<BlobItem> listBlobItems(BlobServiceClient blobServiceClient, Map<String, String> configuration,
+ AbstractExternalInputStreamFactory.IncludeExcludeMatcher includeExcludeMatcher,
+ IWarningCollector warningCollector) throws CompilationException {
+ String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
+
+ List<BlobItem> filesOnly = new ArrayList<>();
+
+ // Ensure the validity of include/exclude
+ validateIncludeExclude(configuration);
+
+ BlobContainerClient blobContainer;
+ try {
+ blobContainer = blobServiceClient.getBlobContainerClient(container);
+
+ // Get all objects in a container and extract the paths to files
+ ListBlobsOptions listBlobsOptions = new ListBlobsOptions();
+ listBlobsOptions.setPrefix(getPrefix(configuration));
+ Iterable<BlobItem> blobItems = blobContainer.listBlobs(listBlobsOptions, null);
+
+ // Collect the paths to files only
+ collectAndFilterBlobFiles(blobItems, includeExcludeMatcher.getPredicate(),
+ includeExcludeMatcher.getMatchersList(), filesOnly);
+
+ // Warn if no files are returned
+ if (filesOnly.isEmpty() && warningCollector.shouldWarn()) {
+ Warning warning = Warning.of(null, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
+ warningCollector.warn(warning);
+ }
+ } catch (Exception ex) {
+ throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+ }
+
+ return filesOnly;
+ }
+
+ /**
+ * Collects and filters the files only, and excludes any folders
+ *
+ * @param items storage items
+ * @param predicate predicate to test with for file filtration
+ * @param matchers include/exclude matchers to test against
+ * @param filesOnly List containing the files only (excluding folders)
+ */
+ private static void collectAndFilterBlobFiles(Iterable<BlobItem> items,
+ BiPredicate<List<Matcher>, String> predicate, List<Matcher> matchers, List<BlobItem> filesOnly) {
+ for (BlobItem item : items) {
+ String uri = item.getName();
+
+ // skip folders
+ if (uri.endsWith("/")) {
+ continue;
+ }
+
+ // No filter, add file
+ if (predicate.test(matchers, uri)) {
+ filesOnly.add(item);
+ }
+ }
+ }
+
+ public static List<PathItem> listDatalakePathItems(DataLakeServiceClient client, Map<String, String> configuration,
+ AbstractExternalInputStreamFactory.IncludeExcludeMatcher includeExcludeMatcher,
+ IWarningCollector warningCollector) throws CompilationException {
+ String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
+
+ List<PathItem> filesOnly = new ArrayList<>();
+
+ // Ensure the validity of include/exclude
+ validateIncludeExclude(configuration);
+
+ DataLakeFileSystemClient fileSystemClient;
+ try {
+ fileSystemClient = client.getFileSystemClient(container);
+
+ // Get all objects in a container and extract the paths to files
+ ListPathsOptions listOptions = new ListPathsOptions();
+ boolean recursive = Boolean.parseBoolean(configuration.get(RECURSIVE_FIELD_NAME));
+ listOptions.setRecursive(recursive);
+ listOptions.setPath(getPrefix(configuration, false));
+ PagedIterable<PathItem> pathItems = fileSystemClient.listPaths(listOptions, null);
+
+ // Collect the paths to files only
+ collectAndFilterDatalakeFiles(pathItems, includeExcludeMatcher.getPredicate(),
+ includeExcludeMatcher.getMatchersList(), filesOnly);
+
+ // Warn if no files are returned
+ if (filesOnly.isEmpty() && warningCollector.shouldWarn()) {
+ Warning warning = Warning.of(null, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
+ warningCollector.warn(warning);
+ }
+ } catch (Exception ex) {
+ throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+ }
+
+ return filesOnly;
+ }
+
+ /**
+ * Collects and filters the files only, and excludes any folders
+ *
+ * @param items storage items
+ * @param predicate predicate to test with for file filtration
+ * @param matchers include/exclude matchers to test against
+ * @param filesOnly List containing the files only (excluding folders)
+ */
+ private static void collectAndFilterDatalakeFiles(Iterable<PathItem> items,
+ BiPredicate<List<Matcher>, String> predicate, List<Matcher> matchers, List<PathItem> filesOnly) {
+ for (PathItem item : items) {
+ String uri = item.getName();
+
+ // skip folders
+ if (uri.endsWith("/")) {
+ continue;
+ }
+
+ // No filter, add file
+ if (predicate.test(matchers, uri)) {
+ filesOnly.add(item);
+ }
+ }
+ }
+
+ /**
+ * Validate external dataset properties
+ *
+ * @param configuration properties
+ * @throws CompilationException Compilation exception
+ */
+ public static void validateAzureBlobProperties(Map<String, String> configuration, SourceLocation srcLoc,
+ IWarningCollector collector, IApplicationContext appCtx) throws CompilationException {
+
+ // check if the format property is present
+ if (configuration.get(ExternalDataConstants.KEY_FORMAT) == null) {
+ throw new CompilationException(ErrorCode.PARAMETERS_REQUIRED, srcLoc, ExternalDataConstants.KEY_FORMAT);
+ }
+
+ validateIncludeExclude(configuration);
+
+ // Check if the bucket is present
+ BlobServiceClient blobServiceClient;
+ try {
+ String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
+ blobServiceClient = buildAzureBlobClient(appCtx, configuration);
+ BlobContainerClient blobContainer = blobServiceClient.getBlobContainerClient(container);
+
+ // Get all objects in a container and extract the paths to files
+ ListBlobsOptions listBlobsOptions = new ListBlobsOptions();
+ listBlobsOptions.setPrefix(getPrefix(configuration));
+ Iterable<BlobItem> blobItems = blobContainer.listBlobs(listBlobsOptions, null);
+
+ if (!blobItems.iterator().hasNext() && collector.shouldWarn()) {
+ Warning warning = Warning.of(srcLoc, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
+ collector.warn(warning);
+ }
+ } catch (CompilationException ex) {
+ throw ex;
+ } catch (Exception ex) {
+ throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+ }
+ }
+
+ /**
+ * Validate external dataset properties
+ *
+ * @param configuration properties
+ * @throws CompilationException Compilation exception
+ */
+ public static void validateAzureDataLakeProperties(Map<String, String> configuration, SourceLocation srcLoc,
+ IWarningCollector collector, IApplicationContext appCtx) throws CompilationException {
+
+ // check if the format property is present
+ if (configuration.get(ExternalDataConstants.KEY_FORMAT) == null) {
+ throw new CompilationException(ErrorCode.PARAMETERS_REQUIRED, srcLoc, ExternalDataConstants.KEY_FORMAT);
+ }
+
+ validateIncludeExclude(configuration);
+
+ // Check if the bucket is present
+ DataLakeServiceClient dataLakeServiceClient;
+ try {
+ String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
+ dataLakeServiceClient = buildAzureDatalakeClient(appCtx, configuration);
+ DataLakeFileSystemClient fileSystemClient = dataLakeServiceClient.getFileSystemClient(container);
+
+ // Get all objects in a container and extract the paths to files
+ ListPathsOptions listPathsOptions = new ListPathsOptions();
+ listPathsOptions.setPath(getPrefix(configuration));
+ Iterable<PathItem> blobItems = fileSystemClient.listPaths(listPathsOptions, null);
+
+ if (!blobItems.iterator().hasNext() && collector.shouldWarn()) {
+ Warning warning = Warning.of(srcLoc, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
+ collector.warn(warning);
+ }
+ } catch (CompilationException ex) {
+ throw ex;
+ } catch (Exception ex) {
+ throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+ }
+ }
+
+ /**
+ * Builds the Azure Blob storage client using the provided configuration
+ *
+ * @param configuration properties
+ * @see <a href="https://docs.microsoft.com/en-us/azure/databricks/data/data-sources/azure/azure-storage">Azure
+ * Blob storage</a>
+ */
+ public static void configureAzureHdfsJobConf(JobConf conf, Map<String, String> configuration, String endPoint) {
+ String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
+ String accountKey = configuration.get(ACCOUNT_KEY_FIELD_NAME);
+ String sharedAccessSignature = configuration.get(SHARED_ACCESS_SIGNATURE_FIELD_NAME);
+
+ //Disable caching S3 FileSystem
+ HDFSUtils.disableHadoopFileSystemCache(conf, HADOOP_AZURE_BLOB_PROTOCOL);
+
+ //Key for Hadoop configuration
+ StringBuilder hadoopKey = new StringBuilder();
+ //Value for Hadoop configuration
+ String hadoopValue;
+ if (accountKey != null || sharedAccessSignature != null) {
+ if (accountKey != null) {
+ hadoopKey.append(HADOOP_AZURE_FS_ACCOUNT_KEY).append('.');
+ //Set only the AccountKey
+ hadoopValue = accountKey;
+ } else {
+ //Use SAS for Hadoop FS as connectionString is provided
+ hadoopKey.append(HADOOP_AZURE_FS_SAS).append('.');
+ //Setting the container is required for SAS
+ hadoopKey.append(container).append('.');
+ //Set the connection string for SAS
+ hadoopValue = sharedAccessSignature;
+ }
+ //Set the endPoint, which includes the AccountName
+ hadoopKey.append(endPoint);
+ //Tells Hadoop we are reading from Blob Storage
+ conf.set(hadoopKey.toString(), hadoopValue);
+ }
+ }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/google/gcs/GCSConstants.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/google/gcs/GCSConstants.java
new file mode 100644
index 0000000..f2dbde7
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/google/gcs/GCSConstants.java
@@ -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.
+ */
+package org.apache.asterix.external.util.google.gcs;
+
+public class GCSConstants {
+ private GCSConstants() {
+ 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";
+ public static final String ENDPOINT_FIELD_NAME = "endpoint";
+
+ /*
+ * Hadoop internal configuration
+ */
+ public static final String HADOOP_GCS_PROTOCOL = "gs";
+
+ // hadoop credentials
+ public static final String HADOOP_AUTH_TYPE = "fs.gs.auth.type";
+ public static final String HADOOP_AUTH_UNAUTHENTICATED = "UNAUTHENTICATED";
+ public static final String HADOOP_AUTH_SERVICE_ACCOUNT_JSON_KEY_FILE = "SERVICE_ACCOUNT_JSON_KEYFILE";
+ public static final String HADOOP_AUTH_SERVICE_ACCOUNT_JSON_KEY_FILE_PATH =
+ "google.cloud.auth.service.account.json.keyfile";
+
+ // gs hadoop parameters
+ public static final String HADOOP_SUPPORT_COMPRESSED = "fs.gs.inputstream.support.gzip.encoding.enable";
+ public static final String HADOOP_ENDPOINT = "fs.gs.storage.root.url";
+ public static final String HADOOP_MAX_REQUESTS_PER_BATCH = "fs.gs.max.requests.per.batch";
+ public static final String HADOOP_BATCH_THREADS = "fs.gs.batch.threads";
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/google/gcs/GCSUtils.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/google/gcs/GCSUtils.java
new file mode 100644
index 0000000..3efb041
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/google/gcs/GCSUtils.java
@@ -0,0 +1,242 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.util.google.gcs;
+
+import static org.apache.asterix.common.exceptions.ErrorCode.EXTERNAL_SOURCE_ERROR;
+import static org.apache.asterix.common.exceptions.ErrorCode.INVALID_PARAM_VALUE_ALLOWED_VALUE;
+import static org.apache.asterix.common.exceptions.ErrorCode.PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT;
+import static org.apache.asterix.external.util.ExternalDataUtils.getPrefix;
+import static org.apache.asterix.external.util.ExternalDataUtils.validateIncludeExclude;
+import static org.apache.asterix.external.util.google.gcs.GCSConstants.APPLICATION_DEFAULT_CREDENTIALS_FIELD_NAME;
+import static org.apache.asterix.external.util.google.gcs.GCSConstants.ENDPOINT_FIELD_NAME;
+import static org.apache.asterix.external.util.google.gcs.GCSConstants.HADOOP_AUTH_SERVICE_ACCOUNT_JSON_KEY_FILE;
+import static org.apache.asterix.external.util.google.gcs.GCSConstants.HADOOP_AUTH_SERVICE_ACCOUNT_JSON_KEY_FILE_PATH;
+import static org.apache.asterix.external.util.google.gcs.GCSConstants.HADOOP_AUTH_TYPE;
+import static org.apache.asterix.external.util.google.gcs.GCSConstants.HADOOP_AUTH_UNAUTHENTICATED;
+import static org.apache.asterix.external.util.google.gcs.GCSConstants.HADOOP_ENDPOINT;
+import static org.apache.asterix.external.util.google.gcs.GCSConstants.HADOOP_GCS_PROTOCOL;
+import static org.apache.asterix.external.util.google.gcs.GCSConstants.JSON_CREDENTIALS_FIELD_NAME;
+import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.function.BiPredicate;
+import java.util.regex.Matcher;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStreamFactory.IncludeExcludeMatcher;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.asterix.external.util.HDFSUtils;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.api.exceptions.Warning;
+
+import com.google.api.gax.paging.Page;
+import com.google.auth.oauth2.GoogleCredentials;
+import com.google.cloud.BaseServiceException;
+import com.google.cloud.storage.Blob;
+import com.google.cloud.storage.Storage;
+import com.google.cloud.storage.StorageOptions;
+
+public class GCSUtils {
+ private GCSUtils() {
+ throw new AssertionError("do not instantiate");
+
+ }
+
+ /**
+ * Builds the client using the provided configuration
+ *
+ * @param configuration properties
+ * @return clientasterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
+ * @throws CompilationException CompilationException
+ */
+ public static Storage buildClient(Map<String, String> configuration) throws CompilationException {
+ String 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();
+
+ // 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 (IOException ex) {
+ throw CompilationException.create(EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+ }
+ }
+
+ // json credentials
+ if (jsonCredentials != null) {
+ try (InputStream credentialsStream = new ByteArrayInputStream(jsonCredentials.getBytes())) {
+ builder.setCredentials(GoogleCredentials.fromStream(credentialsStream));
+ } catch (IOException ex) {
+ throw new CompilationException(EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+ }
+ }
+
+ if (endpoint != null) {
+ builder.setHost(endpoint);
+ }
+
+ return builder.build().getService();
+ }
+
+ /**
+ * Validate external dataset properties
+ *
+ * @param configuration properties
+ * @throws CompilationException Compilation exception
+ */
+ public static void validateProperties(Map<String, String> configuration, SourceLocation srcLoc,
+ IWarningCollector collector) throws CompilationException {
+
+ // check if the format property is present
+ if (configuration.get(ExternalDataConstants.KEY_FORMAT) == null) {
+ throw new CompilationException(ErrorCode.PARAMETERS_REQUIRED, srcLoc, ExternalDataConstants.KEY_FORMAT);
+ }
+
+ validateIncludeExclude(configuration);
+ String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
+
+ try {
+ Storage.BlobListOption limitOption = Storage.BlobListOption.pageSize(1);
+ Storage.BlobListOption prefixOption = Storage.BlobListOption.prefix(getPrefix(configuration));
+ Storage storage = buildClient(configuration);
+ Page<Blob> items = storage.list(container, limitOption, prefixOption);
+
+ if (!items.iterateAll().iterator().hasNext() && collector.shouldWarn()) {
+ Warning warning = Warning.of(srcLoc, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
+ collector.warn(warning);
+ }
+ } catch (CompilationException ex) {
+ throw ex;
+ } catch (Exception ex) {
+ throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+ }
+ }
+
+ public static List<Blob> listItems(Map<String, String> configuration, IncludeExcludeMatcher includeExcludeMatcher,
+ IWarningCollector warningCollector) throws CompilationException {
+ // Prepare to retrieve the objects
+ List<Blob> filesOnly = new ArrayList<>();
+ String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
+ Storage gcs = buildClient(configuration);
+ Storage.BlobListOption options = Storage.BlobListOption.prefix(ExternalDataUtils.getPrefix(configuration));
+ Page<Blob> items;
+
+ try {
+ items = gcs.list(container, options);
+ } catch (BaseServiceException ex) {
+ throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+ }
+
+ // Collect the paths to files only
+ collectAndFilterFiles(items, includeExcludeMatcher.getPredicate(), includeExcludeMatcher.getMatchersList(),
+ filesOnly);
+
+ // Warn if no files are returned
+ if (filesOnly.isEmpty() && warningCollector.shouldWarn()) {
+ Warning warning = Warning.of(null, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
+ warningCollector.warn(warning);
+ }
+
+ return filesOnly;
+ }
+
+ /**
+ * Excludes paths ending with "/" as that's a directory indicator, we need to return the files only
+ *
+ * @param items List of returned objects
+ */
+ private static void collectAndFilterFiles(Page<Blob> items, BiPredicate<List<Matcher>, String> predicate,
+ List<Matcher> matchers, List<Blob> filesOnly) {
+ for (Blob item : items.iterateAll()) {
+ // skip folders
+ if (item.getName().endsWith("/")) {
+ continue;
+ }
+
+ // No filter, add file
+ if (predicate.test(matchers, item.getName())) {
+ filesOnly.add(item);
+ }
+ }
+ }
+
+ /**
+ * Builds the client using the provided configuration
+ *
+ * @param configuration properties
+ * @param numberOfPartitions number of partitions in the cluster
+ */
+ public static void configureHdfsJobConf(JobConf conf, Map<String, String> configuration, int numberOfPartitions) {
+ String jsonCredentials = configuration.get(JSON_CREDENTIALS_FIELD_NAME);
+ String endpoint = configuration.get(ENDPOINT_FIELD_NAME);
+
+ // disable caching FileSystem
+ HDFSUtils.disableHadoopFileSystemCache(conf, HADOOP_GCS_PROTOCOL);
+
+ // TODO(htowaileb): needs further testing, recommended to disable by gcs-hadoop team
+ conf.set(GCSConstants.HADOOP_SUPPORT_COMPRESSED, ExternalDataConstants.FALSE);
+
+ // TODO(htowaileb): needs further testing
+ // set number of threads
+ // conf.set(GCSConstants.HADOOP_MAX_REQUESTS_PER_BATCH, String.valueOf(numberOfPartitions));
+ // conf.set(GCSConstants.HADOOP_BATCH_THREADS, String.valueOf(numberOfPartitions));
+
+ // authentication method
+ // TODO(htowaileb): find a way to pass the content instead of the path to keyfile, this line is temporary
+ Path credentials = Path.of("credentials.json");
+ if (jsonCredentials == null) {
+ // anonymous access
+ conf.set(HADOOP_AUTH_TYPE, HADOOP_AUTH_UNAUTHENTICATED);
+ } else {
+ // TODO(htowaileb) need to pass the file content
+ conf.set(HADOOP_AUTH_TYPE, HADOOP_AUTH_SERVICE_ACCOUNT_JSON_KEY_FILE);
+ conf.set(HADOOP_AUTH_SERVICE_ACCOUNT_JSON_KEY_FILE_PATH, credentials.toAbsolutePath().toString());
+ }
+
+ // set endpoint if provided, default is https://storage.googleapis.com/
+ if (endpoint != null) {
+ conf.set(HADOOP_ENDPOINT, endpoint);
+ }
+ }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/checkerframework/checker/nullness/compatqual/NullableDecl.java b/asterixdb/asterix-external-data/src/main/java/org/checkerframework/checker/nullness/compatqual/NullableDecl.java
new file mode 100644
index 0000000..a4cb401
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/checkerframework/checker/nullness/compatqual/NullableDecl.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.checkerframework.checker.nullness.compatqual;
+
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+
+/*
+ * This is a clean room implementation of the NullableDecl interface based
+ * on 3 requirements:
+ * 1. shall be an @interface named NullableDecl
+ * 2. shall be in the org.checkerframework.checker.nullness.compatqual package
+ * 3. the rention policy for the interface shall be RUNTIME
+ */
+@Retention(RetentionPolicy.RUNTIME)
+public @interface NullableDecl {
+}
diff --git a/asterixdb/asterix-external-data/src/main/resources/META-INF/services/org.apache.asterix.external.api.IRecordReaderFactory b/asterixdb/asterix-external-data/src/main/resources/META-INF/services/org.apache.asterix.external.api.IRecordReaderFactory
index dceed82..88f3fcb 100644
--- a/asterixdb/asterix-external-data/src/main/resources/META-INF/services/org.apache.asterix.external.api.IRecordReaderFactory
+++ b/asterixdb/asterix-external-data/src/main/resources/META-INF/services/org.apache.asterix.external.api.IRecordReaderFactory
@@ -26,4 +26,5 @@
org.apache.asterix.external.input.record.reader.azure.blob.AzureBlobReaderFactory
org.apache.asterix.external.input.record.reader.azure.datalake.AzureDataLakeReaderFactory
org.apache.asterix.external.input.record.reader.azure.parquet.AzureBlobParquetReaderFactory
-org.apache.asterix.external.input.record.reader.azure.parquet.AzureDataLakeParquetReaderFactory
\ No newline at end of file
+org.apache.asterix.external.input.record.reader.azure.parquet.AzureDataLakeParquetReaderFactory
+org.apache.asterix.external.input.record.reader.gcs.parquet.GCSParquetReaderFactory
\ No newline at end of file
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/input/record/reader/awss3/AwsS3Test.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/input/record/reader/awss3/AwsS3Test.java
index 90ea04b..91afbd8 100644
--- a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/input/record/reader/awss3/AwsS3Test.java
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/input/record/reader/awss3/AwsS3Test.java
@@ -18,8 +18,8 @@
*/
package org.apache.asterix.external.input.record.reader.awss3;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.ERROR_INTERNAL_ERROR;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.ERROR_SLOW_DOWN;
+import static org.apache.asterix.external.util.aws.s3.S3Constants.ERROR_INTERNAL_ERROR;
+import static org.apache.asterix.external.util.aws.s3.S3Constants.ERROR_SLOW_DOWN;
import java.lang.reflect.Field;
import java.lang.reflect.Method;
diff --git a/asterixdb/asterix-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-lang-common/src/main/java/org/apache/asterix/lang/common/base/AbstractExtensionClause.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/AbstractExtensionClause.java
new file mode 100644
index 0000000..af9015b
--- /dev/null
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/AbstractExtensionClause.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.lang.common.base;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
+
+public abstract class AbstractExtensionClause extends AbstractClause {
+ /**
+ * Parent languages should handle extension functionality via a new method in {@link IVisitorExtension}.
+ */
+ @Override
+ public <R, T> R accept(ILangVisitor<R, T> visitor, T arg) throws CompilationException {
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, getSourceLocation(),
+ "Unhandled dispatch to an extension clause node!");
+ }
+
+ public abstract IVisitorExtension getVisitorExtension();
+
+ @Override
+ public final ClauseType getClauseType() {
+ return ClauseType.EXTENSION;
+ }
+}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Clause.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Clause.java
index 14f836b..95ed6df 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Clause.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Clause.java
@@ -43,7 +43,9 @@
SELECT_ELEMENT,
SELECT_REGULAR,
SELECT_SET_OPERATION,
- UNNEST_CLAUSE
+ UNNEST_CLAUSE,
+
+ EXTENSION
}
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IVisitorExtension.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IVisitorExtension.java
new file mode 100644
index 0000000..9007eb8
--- /dev/null
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IVisitorExtension.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.base;
+
+import java.util.Collection;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.expression.AbstractCallExpression;
+import org.apache.asterix.lang.common.expression.VariableExpr;
+import org.apache.asterix.lang.common.parser.ScopeChecker;
+import org.apache.asterix.lang.common.rewrites.VariableSubstitutionEnvironment;
+import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+
+/**
+ * Contract for all extension AST nodes to allow existing rewrites to support language extensions.
+ */
+public interface IVisitorExtension {
+ Expression simpleExpressionDispatch(ILangVisitor<Expression, ILangExpression> simpleExpressionVisitor,
+ ILangExpression argument) throws CompilationException;
+
+ Void freeVariableDispatch(ILangVisitor<Void, Collection<VariableExpr>> freeVariableVisitor,
+ Collection<VariableExpr> freeVariables) throws CompilationException;
+
+ Void bindingVariableDispatch(ILangVisitor<Void, Collection<VariableExpr>> bindingVariableVisitor,
+ Collection<VariableExpr> bindingVariables) throws CompilationException;
+
+ Expression variableScopeDispatch(ILangVisitor<Expression, ILangExpression> scopingVisitor, ILangExpression argument,
+ ScopeChecker scopeChecker) throws CompilationException;
+
+ ILangExpression deepCopyDispatch(ILangVisitor<ILangExpression, Void> deepCopyVisitor) throws CompilationException;
+
+ Pair<ILangExpression, VariableSubstitutionEnvironment> remapCloneDispatch(
+ ILangVisitor<Pair<ILangExpression, VariableSubstitutionEnvironment>, VariableSubstitutionEnvironment> remapCloneVisitor,
+ VariableSubstitutionEnvironment substitutionEnvironment) throws CompilationException;
+
+ Boolean inlineUDFsDispatch(ILangVisitor<Boolean, Void> inlineUDFsVisitor) throws CompilationException;
+
+ Void gatherFunctionsDispatch(ILangVisitor<Void, Void> gatherFunctionsVisitor,
+ Collection<? super AbstractCallExpression> functionCalls) throws CompilationException;
+
+ Boolean checkSubqueryDispatch(ILangVisitor<Boolean, ILangExpression> checkSubqueryVisitor, ILangExpression argument)
+ throws CompilationException;
+
+ Boolean check92AggregateDispatch(ILangVisitor<Boolean, ILangExpression> check92AggregateVisitor,
+ ILangExpression argument) throws CompilationException;
+
+ Boolean checkNonFunctionalDispatch(ILangVisitor<Boolean, Void> checkNonFunctionalVisitor)
+ throws CompilationException;
+
+ Boolean checkDatasetOnlyDispatch(ILangVisitor<Boolean, VariableExpr> checkDatasetOnlyVisitor,
+ VariableExpr datasetCandidate) throws CompilationException;
+}
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/rewrites/LangRewritingContext.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/rewrites/LangRewritingContext.java
index 95cccb0..19b3cfa 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/rewrites/LangRewritingContext.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/rewrites/LangRewritingContext.java
@@ -32,7 +32,7 @@
import org.apache.hyracks.algebricks.core.algebra.base.Counter;
import org.apache.hyracks.api.exceptions.IWarningCollector;
-public final class LangRewritingContext {
+public class LangRewritingContext {
private final MetadataProvider metadataProvider;
private final IWarningCollector warningCollector;
private final Map<FunctionSignature, FunctionDecl> declaredFunctions;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/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..cbf2c07
--- /dev/null
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/AnalyzeStatement.java
@@ -0,0 +1,170 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.lang.common.statement;
+
+import java.util.Locale;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.lang.common.base.AbstractStatement;
+import org.apache.asterix.lang.common.expression.RecordConstructor;
+import org.apache.asterix.lang.common.util.ExpressionUtils;
+import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
+import org.apache.asterix.object.base.AdmBigIntNode;
+import org.apache.asterix.object.base.AdmDoubleNode;
+import org.apache.asterix.object.base.AdmObjectNode;
+import org.apache.asterix.object.base.AdmStringNode;
+import org.apache.asterix.object.base.IAdmNode;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+
+public class AnalyzeStatement extends AbstractStatement {
+
+ private static final String SAMPLE_FIELD_NAME = "sample";
+ private static final String SAMPLE_LOW = "low";
+ private static final String SAMPLE_MEDIUM = "medium";
+ private static final String SAMPLE_HIGH = "high";
+ private static final int SAMPLE_LOW_SIZE = 1063;
+ private static final int SAMPLE_MEDIUM_SIZE = SAMPLE_LOW_SIZE * 4;
+ private static final int SAMPLE_HIGH_SIZE = SAMPLE_MEDIUM_SIZE * 4;
+ private static final int SAMPLE_DEFAULT_SIZE = SAMPLE_LOW_SIZE;
+
+ private static final String SAMPLE_SEED_FIELD_NAME = "sample-seed";
+
+ private final DataverseName dataverseName;
+ private final String datasetName;
+ private final AdmObjectNode options;
+
+ public AnalyzeStatement(DataverseName dataverseName, String datasetName, RecordConstructor options)
+ throws CompilationException {
+ this.dataverseName = dataverseName;
+ this.datasetName = datasetName;
+ this.options = options == null ? null : validateOptions(ExpressionUtils.toNode(options));
+ }
+
+ private static AdmObjectNode validateOptions(AdmObjectNode options) throws CompilationException {
+ for (String fieldName : options.getFieldNames()) {
+ switch (fieldName) {
+ case SAMPLE_FIELD_NAME:
+ case SAMPLE_SEED_FIELD_NAME:
+ break;
+ default:
+ throw new CompilationException(ErrorCode.INVALID_PARAM, fieldName);
+ }
+ }
+ return options;
+ }
+
+ @Override
+ public Kind getKind() {
+ return Kind.ANALYZE;
+ }
+
+ public DataverseName getDataverseName() {
+ return dataverseName;
+ }
+
+ public String getDatasetName() {
+ return datasetName;
+ }
+
+ public int getSampleSize() throws CompilationException {
+ IAdmNode n = getOption(SAMPLE_FIELD_NAME);
+ if (n == null) {
+ return SAMPLE_DEFAULT_SIZE;
+ }
+ switch (n.getType()) {
+ case STRING:
+ String s = ((AdmStringNode) n).get();
+ switch (s.toLowerCase(Locale.ROOT)) {
+ case SAMPLE_LOW:
+ return SAMPLE_LOW_SIZE;
+ case SAMPLE_MEDIUM:
+ return SAMPLE_MEDIUM_SIZE;
+ case SAMPLE_HIGH:
+ return SAMPLE_HIGH_SIZE;
+ default:
+ throw new CompilationException(ErrorCode.INVALID_PROPERTY_FORMAT, SAMPLE_FIELD_NAME);
+ }
+ case BIGINT:
+ int v = (int) ((AdmBigIntNode) n).get();
+ if (!isValidSampleSize(v)) {
+ throw new CompilationException(ErrorCode.INVALID_PROPERTY_FORMAT, SAMPLE_FIELD_NAME);
+ }
+ return v;
+ case DOUBLE:
+ v = (int) ((AdmDoubleNode) n).get();
+ if (!isValidSampleSize(v)) {
+ throw new CompilationException(ErrorCode.INVALID_PROPERTY_FORMAT, SAMPLE_FIELD_NAME);
+ }
+ return v;
+ default:
+ throw new CompilationException(ErrorCode.WITH_FIELD_MUST_BE_OF_TYPE, SAMPLE_FIELD_NAME,
+ BuiltinType.ASTRING.getTypeName(), n.getType().toString());
+ }
+ }
+
+ public long getOrCreateSampleSeed() throws AlgebricksException {
+ IAdmNode n = getOption(SAMPLE_SEED_FIELD_NAME);
+ return n != null ? getSampleSeed(n) : createSampleSeed();
+ }
+
+ private long getSampleSeed(IAdmNode n) throws CompilationException {
+ switch (n.getType()) {
+ case BIGINT:
+ return ((AdmBigIntNode) n).get();
+ case DOUBLE:
+ return (long) ((AdmDoubleNode) n).get();
+ case STRING:
+ String s = ((AdmStringNode) n).get();
+ try {
+ return Long.parseLong(s);
+ } catch (NumberFormatException e) {
+ throw new CompilationException(ErrorCode.INVALID_PROPERTY_FORMAT, SAMPLE_SEED_FIELD_NAME);
+ }
+ default:
+ throw new CompilationException(ErrorCode.WITH_FIELD_MUST_BE_OF_TYPE, SAMPLE_SEED_FIELD_NAME,
+ BuiltinType.AINT64.getTypeName(), n.getType().toString());
+ }
+ }
+
+ private long createSampleSeed() {
+ return System.nanoTime() + System.identityHashCode(this);
+ }
+
+ private boolean isValidSampleSize(int v) {
+ return v >= SAMPLE_LOW_SIZE && v <= SAMPLE_HIGH_SIZE;
+ }
+
+ private IAdmNode getOption(String optionName) {
+ return options != null ? options.get(optionName) : null;
+ }
+
+ @Override
+ public <R, T> R accept(ILangVisitor<R, T> visitor, T arg) throws CompilationException {
+ return visitor.visit(this, arg);
+ }
+
+ @Override
+ public byte getCategory() {
+ return Category.DDL;
+ }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/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/FunctionUtil.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/FunctionUtil.java
index 42a35f3..cce0bd8 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,81 @@
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);
+ }
+
+ return commutativeEquals(expr1, expr2, new BitSet());
+ }
+
+ private static boolean commutativeEquals(ILogicalExpression expr1, ILogicalExpression expr2, BitSet matched) {
+ if (expr1.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL
+ || expr2.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+ return expr1.equals(expr2);
+ }
+
+ AbstractFunctionCallExpression funcExpr1 = (AbstractFunctionCallExpression) expr1;
+ AbstractFunctionCallExpression funcExpr2 = (AbstractFunctionCallExpression) expr2;
+
+ List<Mutable<ILogicalExpression>> args1 = funcExpr1.getArguments();
+ List<Mutable<ILogicalExpression>> args2 = funcExpr2.getArguments();
+
+ BitSet childrenSet = 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);
+ childrenSet.clear();
+ if (!matched.get(i) && commutativeEquals(arg1.getValue(), arg2.getValue(), childrenSet)) {
+ 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..077748d 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
@@ -31,6 +31,7 @@
import org.apache.asterix.lang.common.base.Expression;
import org.apache.asterix.lang.common.base.Expression.Kind;
import org.apache.asterix.lang.common.base.ILangExpression;
+import org.apache.asterix.lang.common.base.IVisitorExtension;
import org.apache.asterix.lang.common.clause.GroupbyClause;
import org.apache.asterix.lang.common.clause.LetClause;
import org.apache.asterix.lang.common.clause.LimitClause;
@@ -264,6 +265,11 @@
}
@Override
+ public Boolean visit(IVisitorExtension ve, Void arg) throws CompilationException {
+ return ve.inlineUDFsDispatch(this);
+ }
+
+ @Override
public Boolean visit(InsertStatement insert, Void arg) throws CompilationException {
boolean changed = false;
Expression returnExpression = insert.getReturnExpression();
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/CloneAndSubstituteVariablesVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/CloneAndSubstituteVariablesVisitor.java
index d73c264..06f22b7 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/CloneAndSubstituteVariablesVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/CloneAndSubstituteVariablesVisitor.java
@@ -29,6 +29,7 @@
import org.apache.asterix.lang.common.base.Expression;
import org.apache.asterix.lang.common.base.Expression.Kind;
import org.apache.asterix.lang.common.base.ILangExpression;
+import org.apache.asterix.lang.common.base.IVisitorExtension;
import org.apache.asterix.lang.common.clause.GroupbyClause;
import org.apache.asterix.lang.common.clause.LetClause;
import org.apache.asterix.lang.common.clause.LimitClause;
@@ -339,6 +340,12 @@
}
@Override
+ public Pair<ILangExpression, VariableSubstitutionEnvironment> visit(IVisitorExtension ve,
+ VariableSubstitutionEnvironment arg) throws CompilationException {
+ return ve.remapCloneDispatch(this, arg);
+ }
+
+ @Override
public Pair<ILangExpression, VariableSubstitutionEnvironment> visit(FieldAccessor fa,
VariableSubstitutionEnvironment env) throws CompilationException {
Pair<ILangExpression, VariableSubstitutionEnvironment> p = fa.getExpr().accept(this, env);
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/GatherFunctionCallsVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/GatherFunctionCallsVisitor.java
index b7cf7af..6dcfb83 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/GatherFunctionCallsVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/GatherFunctionCallsVisitor.java
@@ -25,6 +25,7 @@
import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.lang.common.base.Expression;
+import org.apache.asterix.lang.common.base.IVisitorExtension;
import org.apache.asterix.lang.common.clause.GroupbyClause;
import org.apache.asterix.lang.common.clause.LetClause;
import org.apache.asterix.lang.common.clause.LimitClause;
@@ -227,6 +228,11 @@
}
@Override
+ public Void visit(IVisitorExtension ve, Void arg) throws CompilationException {
+ return ve.gatherFunctionsDispatch(this, calls);
+ }
+
+ @Override
public Void visit(WhereClause wc, Void arg) throws CompilationException {
wc.getWhereExpr().accept(this, arg);
return null;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/AbstractAstVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/AbstractAstVisitor.java
index a422ef1..a6092c6 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/AbstractAstVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/AbstractAstVisitor.java
@@ -19,6 +19,8 @@
package org.apache.asterix.lang.common.visitor.base;
import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.lang.common.base.IVisitorExtension;
import org.apache.asterix.lang.common.clause.GroupbyClause;
import org.apache.asterix.lang.common.clause.LetClause;
import org.apache.asterix.lang.common.clause.LimitClause;
@@ -130,4 +132,8 @@
return null;
}
+ @Override
+ public R visit(IVisitorExtension ve, T arg) throws CompilationException {
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, "Extension dispatch not implemented!");
+ }
}
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..8fda66e 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
@@ -19,6 +19,7 @@
package org.apache.asterix.lang.common.visitor.base;
import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.IVisitorExtension;
import org.apache.asterix.lang.common.clause.GroupbyClause;
import org.apache.asterix.lang.common.clause.LetClause;
import org.apache.asterix.lang.common.clause.LimitClause;
@@ -42,6 +43,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 +210,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;
@@ -216,4 +223,6 @@
R visit(ViewDropStatement vds, T arg) throws CompilationException;
R visit(ViewDecl vd, T arg) throws CompilationException;
+
+ R visit(IVisitorExtension ve, T arg) throws CompilationException;
}
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/SelectClause.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/SelectClause.java
index ea6bac9..c902dbe 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/SelectClause.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/SelectClause.java
@@ -19,23 +19,43 @@
package org.apache.asterix.lang.sqlpp.clause;
+import java.util.ArrayList;
+import java.util.List;
import java.util.Objects;
+import java.util.function.Function;
+import java.util.stream.Collectors;
import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.lang.common.base.AbstractClause;
import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
import org.apache.asterix.lang.sqlpp.visitor.base.ISqlppVisitor;
public class SelectClause extends AbstractClause {
-
+ private final List<List<String>> fieldExclusions = new ArrayList<>();
private SelectElement selectElement;
private SelectRegular selectRegular;
private boolean distinct;
- public SelectClause(SelectElement selectElement, SelectRegular selectRegular, boolean distinct) {
+ public SelectClause(SelectElement selectElement, SelectRegular selectRegular, List<List<String>> fieldExclusions,
+ boolean distinct) {
+ if (selectElement != null && selectRegular != null) {
+ throw new IllegalArgumentException("SELECT-ELEMENT and SELECT-REGULAR cannot both be specified.");
+ }
+ if (selectElement != null && fieldExclusions != null && !fieldExclusions.isEmpty()) {
+ throw new IllegalArgumentException("SELECT-ELEMENT and EXCLUDE cannot both be specified.");
+ }
+
this.selectElement = selectElement;
this.selectRegular = selectRegular;
this.distinct = distinct;
+ if (fieldExclusions != null) {
+ this.fieldExclusions.addAll(fieldExclusions);
+ }
+ }
+
+ public SelectClause(SelectElement selectElement, SelectRegular selectRegular, boolean distinct) {
+ this(selectElement, selectRegular, null, distinct);
}
@Override
@@ -48,7 +68,12 @@
return ClauseType.SELECT_CLAUSE;
}
- public void setSelectElement(SelectElement selectElement) {
+ public void setSelectElement(SelectElement selectElement) throws CompilationException {
+ if (!fieldExclusions.isEmpty() && selectElement != null) {
+ // We forbid SELECT VALUE and EXCLUDE at the parser, so we should never reach here.
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, getSourceLocation(),
+ "SELECT ELEMENT and EXCLUDE cannot coexist!");
+ }
this.selectElement = selectElement;
this.selectRegular = null;
}
@@ -82,15 +107,25 @@
this.distinct = distinct;
}
+ public List<List<String>> getFieldExclusions() {
+ return fieldExclusions;
+ }
+
@Override
public String toString() {
- return "select " + (distinct ? "distinct " : "")
- + (selectElement() ? "element " + selectElement : String.valueOf(selectRegular));
+ String distinctString = distinct ? "distinct " : "";
+ String valueString = selectElement() ? ("element " + selectElement) : String.valueOf(selectRegular);
+ String exceptString = "";
+ if (!fieldExclusions.isEmpty()) {
+ final Function<List<String>, String> fieldBuilder = f -> String.join(".", f);
+ exceptString = " exclude " + fieldExclusions.stream().map(fieldBuilder).collect(Collectors.joining(", "));
+ }
+ return String.format("select %s%s%s", distinctString, valueString, exceptString);
}
@Override
public int hashCode() {
- return Objects.hash(distinct, selectElement, selectRegular);
+ return Objects.hash(distinct, selectElement, selectRegular, fieldExclusions);
}
@Override
@@ -103,6 +138,7 @@
}
SelectClause target = (SelectClause) object;
return distinct == target.distinct && Objects.equals(selectElement, target.selectElement)
- && Objects.equals(selectRegular, target.selectRegular);
+ && Objects.equals(selectRegular, target.selectRegular)
+ && Objects.equals(fieldExclusions, target.fieldExclusions);
}
}
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/SqlppFunctionBodyRewriter.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppFunctionBodyRewriter.java
index 557f17d..32549d9 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppFunctionBodyRewriter.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppFunctionBodyRewriter.java
@@ -61,7 +61,7 @@
/**
* This rewriter is used to rewrite body expression of user defined functions and views
*/
-class SqlppFunctionBodyRewriter extends SqlppQueryRewriter {
+public class SqlppFunctionBodyRewriter extends SqlppQueryRewriter {
public SqlppFunctionBodyRewriter(IParserFactory parserFactory) {
super(parserFactory);
@@ -97,6 +97,9 @@
// Inlines column aliases.
inlineColumnAlias();
+ // Rewrite SELECT EXCLUDE to use OBJECT_REMOVE_FIELDS.
+ rewriteSelectExcludeSugar();
+
// Window expression core rewrites.
rewriteWindowExpressions();
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java
index 2f4fcc8..38177d8 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java
@@ -58,6 +58,7 @@
import org.apache.asterix.lang.sqlpp.rewrites.visitor.InlineColumnAliasVisitor;
import org.apache.asterix.lang.sqlpp.rewrites.visitor.InlineWithExpressionVisitor;
import org.apache.asterix.lang.sqlpp.rewrites.visitor.OperatorExpressionVisitor;
+import org.apache.asterix.lang.sqlpp.rewrites.visitor.SelectExcludeRewriteSugarVisitor;
import org.apache.asterix.lang.sqlpp.rewrites.visitor.SetOperationVisitor;
import org.apache.asterix.lang.sqlpp.rewrites.visitor.SqlCompatRewriteVisitor;
import org.apache.asterix.lang.sqlpp.rewrites.visitor.SqlppCaseAggregateExtractionVisitor;
@@ -162,6 +163,9 @@
// Inlines column aliases.
inlineColumnAlias();
+ // Rewrite SELECT EXCLUDE to use OBJECT_REMOVE_FIELDS.
+ rewriteSelectExcludeSugar();
+
// Window expression core rewrites.
rewriteWindowExpressions();
@@ -350,6 +354,12 @@
}
}
+ protected void rewriteSelectExcludeSugar() throws CompilationException {
+ SelectExcludeRewriteSugarVisitor selectExcludeRewriteSugarVisitor =
+ new SelectExcludeRewriteSugarVisitor(context);
+ rewriteTopExpr(selectExcludeRewriteSugarVisitor, null);
+ }
+
private <R, T> R rewriteTopExpr(ILangVisitor<R, T> visitor, T arg) throws CompilationException {
R result = topStatement.accept(visitor, arg);
logExpression(">>>> AST After", visitor.getClass().getSimpleName());
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/SelectExcludeRewriteSugarVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SelectExcludeRewriteSugarVisitor.java
new file mode 100644
index 0000000..e900c03
--- /dev/null
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SelectExcludeRewriteSugarVisitor.java
@@ -0,0 +1,230 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.sqlpp.rewrites.visitor;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.lang.common.base.Clause;
+import org.apache.asterix.lang.common.base.Expression;
+import org.apache.asterix.lang.common.base.ILangExpression;
+import org.apache.asterix.lang.common.context.Scope;
+import org.apache.asterix.lang.common.expression.CallExpr;
+import org.apache.asterix.lang.common.expression.ListConstructor;
+import org.apache.asterix.lang.common.expression.LiteralExpr;
+import org.apache.asterix.lang.common.expression.VariableExpr;
+import org.apache.asterix.lang.common.literal.StringLiteral;
+import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
+import org.apache.asterix.lang.common.struct.Identifier;
+import org.apache.asterix.lang.common.struct.VarIdentifier;
+import org.apache.asterix.lang.sqlpp.clause.FromClause;
+import org.apache.asterix.lang.sqlpp.clause.FromTerm;
+import org.apache.asterix.lang.sqlpp.clause.Projection;
+import org.apache.asterix.lang.sqlpp.clause.SelectBlock;
+import org.apache.asterix.lang.sqlpp.clause.SelectClause;
+import org.apache.asterix.lang.sqlpp.clause.SelectElement;
+import org.apache.asterix.lang.sqlpp.clause.SelectRegular;
+import org.apache.asterix.lang.sqlpp.clause.SelectSetOperation;
+import org.apache.asterix.lang.sqlpp.expression.SelectExpression;
+import org.apache.asterix.lang.sqlpp.struct.SetOperationInput;
+import org.apache.asterix.lang.sqlpp.struct.SetOperationRight;
+import org.apache.asterix.lang.sqlpp.util.SqlppRewriteUtil;
+import org.apache.asterix.lang.sqlpp.util.SqlppVariableUtil;
+import org.apache.asterix.lang.sqlpp.visitor.base.AbstractSqlppExpressionScopingVisitor;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+/**
+ * Rewrites the exclusion list of a SELECT clause into a OBJECT_REMOVE_FIELDS function application. This rewrite
+ * <b>MUST</b> run after {@link InlineColumnAliasVisitor}.
+ * <p>
+ * Input:
+ * <pre>
+ * FROM ...
+ * WHERE ...
+ * SELECT * EXCLUDE a, b.f
+ * ORDER BY c
+ * </pre>
+ * Output:
+ * <pre>
+ * FROM ( FROM ...
+ * WHERE ...
+ * SELECT *
+ * ORDER BY c ) TMP_1
+ * SELECT VALUE OBJECT_REMOVE_FIELDS(TMP_1, [ "a", [ "b", "f" ]])
+ * </pre>
+ * <p>
+ * There exists a special case with a single {@link FromTerm} node (with no other local bindings) and a SELECT * clause,
+ * where we qualify our field exclusion list with the {@link FromTerm} variable if we cannot anchor on the dataset
+ * variable. For example:
+ * <pre>
+ * FROM MyDataset D
+ * SELECT * EXCLUDE a, b.c, D.d
+ * </pre>
+ * Is conceptually processed as:
+ * <pre>
+ * FROM MyDataset D
+ * SELECT * EXCLUDE D.a, D.b.c, D.d
+ * </pre>
+ * For all other cases, our EXCLUDE will work solely with what our SELECT returns.
+ */
+public class SelectExcludeRewriteSugarVisitor extends AbstractSqlppExpressionScopingVisitor {
+ public SelectExcludeRewriteSugarVisitor(LangRewritingContext langRewritingContext) {
+ super(langRewritingContext);
+ }
+
+ @Override
+ public Expression visit(SelectBlock selectBlock, ILangExpression arg) throws CompilationException {
+ super.visit(selectBlock, arg);
+
+ // Proceed if we have field-exclusions.
+ SelectClause selectClause = selectBlock.getSelectClause();
+ if (selectClause.getFieldExclusions().isEmpty()) {
+ return null;
+ }
+ SelectExpression selectExpression = (SelectExpression) arg;
+
+ // If we have a single dataset in a FROM-CLAUSE (with no other variables in our local scope / grouping)...
+ if (selectBlock.hasFromClause() && selectBlock.getFromClause().getFromTerms().size() == 1) {
+ FromTerm fromTerm = selectBlock.getFromClause().getFromTerms().get(0);
+ if (!selectBlock.hasGroupbyClause() && !fromTerm.hasCorrelateClauses() && selectBlock.getLetWhereList()
+ .stream().noneMatch(c -> c.getClauseType() == Clause.ClauseType.LET_CLAUSE)) {
+ // ...and we have a 'SELECT *'...
+ SelectRegular selectRegular = selectClause.getSelectRegular();
+ if (selectClause.selectRegular() && selectRegular.getProjections().size() == 1
+ && selectRegular.getProjections().get(0).getKind() == Projection.Kind.STAR) {
+ // ...then qualify our field exclusions with our FROM-CLAUSE variable.
+ String fromTermName = fromTerm.getLeftVariable().getVar().getValue();
+ String qualifier = SqlppVariableUtil.toUserDefinedName(fromTermName);
+ selectClause.getFieldExclusions().stream().filter(e -> {
+ // Do not needlessly qualify names that are already bound to variables in our scope.
+ // Note: We use our local scope to include the single-dataset variable AND our outer scope.
+ // We already know that there are no other variables in our local scope.
+ Iterator<Pair<Identifier, Set<? extends Scope.SymbolAnnotation>>> liveSymbolIterator =
+ scopeChecker.getCurrentScope().liveSymbols(null);
+ while (liveSymbolIterator.hasNext()) {
+ Pair<Identifier, Set<? extends Scope.SymbolAnnotation>> symbol = liveSymbolIterator.next();
+ String symbolName = SqlppVariableUtil.toUserDefinedName(symbol.first.getValue());
+ if (symbolName.equals(e.get(0))) {
+ return false;
+ }
+ }
+ return true;
+ }).forEach(e -> e.add(0, qualifier));
+ }
+ }
+ }
+
+ // Find our parent SET-OP-INPUT.
+ SetOperationInput setOperationInput = null;
+ SelectSetOperation selectSetOperation = selectExpression.getSelectSetOperation();
+ if (selectBlock.equals(selectSetOperation.getLeftInput().getSelectBlock())) {
+ setOperationInput = selectSetOperation.getLeftInput();
+ } else {
+ for (SetOperationRight rightInput : selectSetOperation.getRightInputs()) {
+ SetOperationInput setOperationRightInput = rightInput.getSetOperationRightInput();
+ if (selectBlock.equals(setOperationRightInput.getSelectBlock())) {
+ setOperationInput = setOperationRightInput;
+ break;
+ }
+ }
+ }
+ if (setOperationInput == null) {
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, selectBlock.getSourceLocation(),
+ "Parent SET-OP-INPUT not found while rewriting SELECT-EXCLUDE!");
+ }
+
+ // Nest our original SELECT-BLOCK.
+ SourceLocation sourceLocation = selectBlock.getSourceLocation();
+ SetOperationInput innerSetOpInput = new SetOperationInput(selectBlock, null);
+ SelectSetOperation innerSelectSetOp = new SelectSetOperation(innerSetOpInput, null);
+ innerSelectSetOp.setSourceLocation(sourceLocation);
+ SelectExpression innerSelectExpr;
+ if (!selectSetOperation.hasRightInputs()) {
+ // We need to attach our LET / ORDER BY / LIMIT to our inner SELECT-EXPR.
+ SelectExpression selectExprCopy = (SelectExpression) SqlppRewriteUtil.deepCopy(selectExpression);
+ innerSelectExpr = new SelectExpression(selectExprCopy.getLetList(), innerSelectSetOp,
+ selectExprCopy.getOrderbyClause(), selectExprCopy.getLimitClause(), true);
+ selectExpression.getLetList().clear();
+ selectExpression.setOrderbyClause(null);
+ selectExpression.setLimitClause(null);
+ } else {
+ innerSelectExpr = new SelectExpression(null, innerSelectSetOp, null, null, true);
+ }
+ innerSelectExpr.setSourceLocation(sourceLocation);
+
+ // Build a new SELECT-BLOCK.
+ VarIdentifier fromTermVariable = context.newVariable();
+ VariableExpr fromTermVariableExpr = new VariableExpr(fromTermVariable);
+ SelectClause innerSelectClause = buildSelectClause(selectClause, fromTermVariable);
+ innerSelectClause.setSourceLocation(sourceLocation);
+ FromTerm innerFromTerm = new FromTerm(innerSelectExpr, fromTermVariableExpr, null, null);
+ innerFromTerm.setSourceLocation(sourceLocation);
+ FromClause innerFromClause = new FromClause(List.of(innerFromTerm));
+ innerFromClause.setSourceLocation(sourceLocation);
+ SelectBlock innerSelectBlock = new SelectBlock(innerSelectClause, innerFromClause, null, null, null);
+ setOperationInput.setSelectBlock(innerSelectBlock);
+ return null;
+ }
+
+ private SelectClause buildSelectClause(SelectClause originalSelectClause, VarIdentifier iterationVariable) {
+ // Convert our list of identifiers into a list of literals representing field names.
+ ListConstructor listConstructor = new ListConstructor();
+ listConstructor.setType(ListConstructor.Type.ORDERED_LIST_CONSTRUCTOR);
+ listConstructor.setExprList(new ArrayList<>());
+ for (List<String> nestedField : originalSelectClause.getFieldExclusions()) {
+ if (nestedField.size() == 1) {
+ // For non-nested fields, we do not wrap our name in a list.
+ listConstructor.getExprList().add(new LiteralExpr(new StringLiteral(nestedField.get(0))));
+ } else {
+ // Otherwise, build a list to insert into our list.
+ ListConstructor nestedFieldList = new ListConstructor();
+ nestedFieldList.setType(ListConstructor.Type.ORDERED_LIST_CONSTRUCTOR);
+ nestedFieldList.setExprList(nestedField.stream().map(f -> new LiteralExpr(new StringLiteral(f)))
+ .collect(Collectors.toList()));
+ listConstructor.getExprList().add(nestedFieldList);
+ }
+ }
+ List<Expression> objectRemoveFieldsArguments = new ArrayList<>();
+ objectRemoveFieldsArguments.add(new VariableExpr(iterationVariable));
+ objectRemoveFieldsArguments.add(listConstructor);
+ originalSelectClause.getFieldExclusions().clear();
+
+ // Remove the DISTINCT from our original SELECT-CLAUSE, if it exists.
+ boolean isDistinct = originalSelectClause.distinct();
+ if (isDistinct) {
+ originalSelectClause.setDistinct(false);
+ }
+
+ // Create the call to OBJECT_REMOVE_FIELDS.
+ FunctionSignature functionSignature = new FunctionSignature(BuiltinFunctions.REMOVE_FIELDS);
+ CallExpr callExpr = new CallExpr(functionSignature, objectRemoveFieldsArguments);
+ SelectElement selectElement = new SelectElement(callExpr);
+ SelectClause selectClause = new SelectClause(selectElement, null, isDistinct);
+ selectClause.setSourceLocation(originalSelectClause.getSourceLocation());
+ return selectClause;
+ }
+}
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/SqlppGroupByVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppGroupByVisitor.java
index d337de9..c4fb951 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppGroupByVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppGroupByVisitor.java
@@ -124,7 +124,8 @@
return expr.accept(visitor, selectBlock);
}
- private List<Pair<Expression, Identifier>> createGroupFieldList(SelectBlock selectBlock) {
+ private List<Pair<Expression, Identifier>> createGroupFieldList(SelectBlock selectBlock)
+ throws CompilationException {
List<Pair<Expression, Identifier>> groupFieldList = new ArrayList<>();
addToFieldList(groupFieldList, SqlppVariableUtil.getBindingVariables(selectBlock.getFromClause()));
addToFieldList(groupFieldList, SqlppVariableUtil.getLetBindingVariables(selectBlock.getLetWhereList()));
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppWindowAggregationSugarVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppWindowAggregationSugarVisitor.java
index bf3e227..aa548519 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppWindowAggregationSugarVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppWindowAggregationSugarVisitor.java
@@ -139,7 +139,8 @@
winExpr.setExprList(newExprList);
}
- private List<Pair<Expression, Identifier>> createWindowFieldList(SelectBlock selectBlock) {
+ private List<Pair<Expression, Identifier>> createWindowFieldList(SelectBlock selectBlock)
+ throws CompilationException {
List<Pair<Expression, Identifier>> fieldList = new ArrayList<>();
if (selectBlock != null) {
addToFieldList(fieldList, SqlppVariableUtil.getBindingVariables(selectBlock.getFromClause()));
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/java/org/apache/asterix/lang/sqlpp/util/SqlppVariableUtil.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/SqlppVariableUtil.java
index 9d50160..0a8e528 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/SqlppVariableUtil.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/SqlppVariableUtil.java
@@ -29,17 +29,11 @@
import org.apache.asterix.lang.common.base.Clause;
import org.apache.asterix.lang.common.base.Expression;
import org.apache.asterix.lang.common.base.ILangExpression;
-import org.apache.asterix.lang.common.clause.GroupbyClause;
import org.apache.asterix.lang.common.clause.LetClause;
-import org.apache.asterix.lang.common.expression.GbyVariableExpressionPair;
-import org.apache.asterix.lang.common.expression.QuantifiedExpression;
import org.apache.asterix.lang.common.expression.VariableExpr;
import org.apache.asterix.lang.common.struct.Identifier;
-import org.apache.asterix.lang.common.struct.QuantifiedPair;
import org.apache.asterix.lang.common.struct.VarIdentifier;
-import org.apache.asterix.lang.sqlpp.clause.AbstractBinaryCorrelateClause;
-import org.apache.asterix.lang.sqlpp.clause.FromClause;
-import org.apache.asterix.lang.sqlpp.clause.FromTerm;
+import org.apache.asterix.lang.sqlpp.visitor.BindingVariableVisitor;
import org.apache.asterix.lang.sqlpp.visitor.FreeVariableVisitor;
import org.apache.hyracks.algebricks.common.utils.Pair;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
@@ -131,61 +125,14 @@
return freeVars;
}
- public static List<VariableExpr> getBindingVariables(FromClause fromClause) {
- if (fromClause == null) {
+ public static List<VariableExpr> getBindingVariables(ILangExpression langExpr) throws CompilationException {
+ if (langExpr == null) {
return Collections.emptyList();
}
- List<VariableExpr> bindingVars = new ArrayList<>();
- for (FromTerm fromTerm : fromClause.getFromTerms()) {
- bindingVars.addAll(getBindingVariables(fromTerm));
- }
- return bindingVars;
- }
- public static List<VariableExpr> getBindingVariables(FromTerm fromTerm) {
+ final BindingVariableVisitor visitor = new BindingVariableVisitor();
List<VariableExpr> bindingVars = new ArrayList<>();
- if (fromTerm == null) {
- return bindingVars;
- }
- bindingVars.add(fromTerm.getLeftVariable());
- if (fromTerm.hasPositionalVariable()) {
- bindingVars.add(fromTerm.getPositionalVariable());
- }
- for (AbstractBinaryCorrelateClause correlateClause : fromTerm.getCorrelateClauses()) {
- bindingVars.add(correlateClause.getRightVariable());
- if (correlateClause.hasPositionalVariable()) {
- bindingVars.add(correlateClause.getPositionalVariable());
- }
- }
- return bindingVars;
- }
-
- public static List<VariableExpr> getBindingVariables(GroupbyClause gbyClause) {
- List<VariableExpr> bindingVars = new ArrayList<>();
- if (gbyClause == null) {
- return bindingVars;
- }
- Set<VariableExpr> gbyKeyVars = new HashSet<>();
- for (List<GbyVariableExpressionPair> gbyPairList : gbyClause.getGbyPairList()) {
- for (GbyVariableExpressionPair gbyKey : gbyPairList) {
- VariableExpr var = gbyKey.getVar();
- if (var != null && gbyKeyVars.add(var)) {
- bindingVars.add(var);
- }
- }
- }
- if (gbyClause.hasDecorList()) {
- for (GbyVariableExpressionPair gbyKey : gbyClause.getDecorPairList()) {
- VariableExpr var = gbyKey.getVar();
- if (var != null) {
- bindingVars.add(var);
- }
- }
- }
- if (gbyClause.hasWithMap()) {
- bindingVars.addAll(gbyClause.getWithVarMap().values());
- }
- bindingVars.add(gbyClause.getGroupVar());
+ langExpr.accept(visitor, bindingVars);
return bindingVars;
}
@@ -203,15 +150,6 @@
return bindingVars;
}
- public static List<VariableExpr> getBindingVariables(QuantifiedExpression qe) {
- List<QuantifiedPair> quantifiedList = qe.getQuantifiedList();
- List<VariableExpr> bindingVars = new ArrayList<>(quantifiedList.size());
- for (QuantifiedPair qp : quantifiedList) {
- bindingVars.add(qp.getVarExpr());
- }
- return bindingVars;
- }
-
public static void addToFieldVariableList(VariableExpr varExpr, List<Pair<Expression, Identifier>> outFieldList) {
VarIdentifier var = varExpr.getVar();
VariableExpr newVarExpr = new VariableExpr(var);
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/BindingVariableVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/BindingVariableVisitor.java
new file mode 100644
index 0000000..1b7dc24
--- /dev/null
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/BindingVariableVisitor.java
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.sqlpp.visitor;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.IVisitorExtension;
+import org.apache.asterix.lang.common.clause.GroupbyClause;
+import org.apache.asterix.lang.common.expression.GbyVariableExpressionPair;
+import org.apache.asterix.lang.common.expression.QuantifiedExpression;
+import org.apache.asterix.lang.common.expression.VariableExpr;
+import org.apache.asterix.lang.common.struct.QuantifiedPair;
+import org.apache.asterix.lang.sqlpp.clause.AbstractBinaryCorrelateClause;
+import org.apache.asterix.lang.sqlpp.clause.FromClause;
+import org.apache.asterix.lang.sqlpp.clause.FromTerm;
+import org.apache.asterix.lang.sqlpp.visitor.base.AbstractSqlppAstVisitor;
+
+public class BindingVariableVisitor extends AbstractSqlppAstVisitor<Void, Collection<VariableExpr>> {
+ @Override
+ public Void visit(FromClause fromClause, Collection<VariableExpr> bindingVars) throws CompilationException {
+ for (FromTerm fromTerm : fromClause.getFromTerms()) {
+ fromTerm.accept(this, bindingVars);
+ }
+ return null;
+ }
+
+ @Override
+ public Void visit(FromTerm fromTerm, Collection<VariableExpr> bindingVars) throws CompilationException {
+ bindingVars.add(fromTerm.getLeftVariable());
+ if (fromTerm.hasPositionalVariable()) {
+ bindingVars.add(fromTerm.getPositionalVariable());
+ }
+ for (AbstractBinaryCorrelateClause correlateClause : fromTerm.getCorrelateClauses()) {
+ bindingVars.add(correlateClause.getRightVariable());
+ if (correlateClause.hasPositionalVariable()) {
+ bindingVars.add(correlateClause.getPositionalVariable());
+ }
+ }
+ return null;
+ }
+
+ @Override
+ public Void visit(GroupbyClause groupbyClause, Collection<VariableExpr> bindingVars) throws CompilationException {
+ Set<VariableExpr> gbyKeyVars = new HashSet<>();
+ for (List<GbyVariableExpressionPair> gbyPairList : groupbyClause.getGbyPairList()) {
+ for (GbyVariableExpressionPair gbyKey : gbyPairList) {
+ VariableExpr var = gbyKey.getVar();
+ if (var != null && gbyKeyVars.add(var)) {
+ bindingVars.add(var);
+ }
+ }
+ }
+ if (groupbyClause.hasDecorList()) {
+ for (GbyVariableExpressionPair gbyKey : groupbyClause.getDecorPairList()) {
+ VariableExpr var = gbyKey.getVar();
+ if (var != null) {
+ bindingVars.add(var);
+ }
+ }
+ }
+ if (groupbyClause.hasWithMap()) {
+ bindingVars.addAll(groupbyClause.getWithVarMap().values());
+ }
+ bindingVars.add(groupbyClause.getGroupVar());
+ return null;
+ }
+
+ @Override
+ public Void visit(QuantifiedExpression qe, Collection<VariableExpr> bindingVars) throws CompilationException {
+ List<QuantifiedPair> quantifiedList = qe.getQuantifiedList();
+ for (QuantifiedPair qp : quantifiedList) {
+ bindingVars.add(qp.getVarExpr());
+ }
+ return null;
+ }
+
+ @Override
+ public Void visit(IVisitorExtension ve, Collection<VariableExpr> arg) throws CompilationException {
+ return ve.bindingVariableDispatch(this, arg);
+ }
+}
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckDatasetOnlyResolutionVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckDatasetOnlyResolutionVisitor.java
index e4ccef5..f700d21 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckDatasetOnlyResolutionVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckDatasetOnlyResolutionVisitor.java
@@ -21,6 +21,7 @@
import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.lang.common.base.ILangExpression;
+import org.apache.asterix.lang.common.base.IVisitorExtension;
import org.apache.asterix.lang.common.clause.GroupbyClause;
import org.apache.asterix.lang.common.clause.LetClause;
import org.apache.asterix.lang.common.clause.LimitClause;
@@ -160,6 +161,11 @@
}
@Override
+ public Boolean visit(IVisitorExtension ve, VariableExpr arg) throws CompilationException {
+ return ve.checkDatasetOnlyDispatch(this, arg);
+ }
+
+ @Override
public Boolean visit(IfExpr ifexpr, VariableExpr arg) throws CompilationException {
return false;
}
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckNonFunctionalExpressionVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckNonFunctionalExpressionVisitor.java
index d6ee1fc..d16322e 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckNonFunctionalExpressionVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckNonFunctionalExpressionVisitor.java
@@ -22,6 +22,7 @@
import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.lang.common.base.IVisitorExtension;
import org.apache.asterix.lang.common.expression.CallExpr;
import org.apache.asterix.lang.sqlpp.visitor.base.AbstractSqlppContainsExpressionVisitor;
import org.apache.asterix.metadata.declared.MetadataProvider;
@@ -68,4 +69,9 @@
}
return super.visit(callExpr, arg);
}
+
+ @Override
+ public Boolean visit(IVisitorExtension ve, Void arg) throws CompilationException {
+ return ve.checkNonFunctionalDispatch(this);
+ }
}
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckSql92AggregateVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckSql92AggregateVisitor.java
index 3192541..5e6a84a 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckSql92AggregateVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckSql92AggregateVisitor.java
@@ -24,6 +24,7 @@
import org.apache.asterix.common.functions.FunctionSignature;
import org.apache.asterix.lang.common.base.Expression;
import org.apache.asterix.lang.common.base.ILangExpression;
+import org.apache.asterix.lang.common.base.IVisitorExtension;
import org.apache.asterix.lang.common.clause.GroupbyClause;
import org.apache.asterix.lang.common.clause.LetClause;
import org.apache.asterix.lang.common.clause.LimitClause;
@@ -157,6 +158,11 @@
}
@Override
+ public Boolean visit(IVisitorExtension ve, ILangExpression arg) throws CompilationException {
+ return ve.check92AggregateDispatch(this, arg);
+ }
+
+ @Override
public Boolean visit(IfExpr ifexpr, ILangExpression parentSelectBlock) throws CompilationException {
if (ifexpr.getCondExpr().accept(this, parentSelectBlock)) {
return true;
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckSubqueryVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckSubqueryVisitor.java
index a903279..399e463 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckSubqueryVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckSubqueryVisitor.java
@@ -24,6 +24,7 @@
import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.lang.common.base.ILangExpression;
+import org.apache.asterix.lang.common.base.IVisitorExtension;
import org.apache.asterix.lang.common.clause.GroupbyClause;
import org.apache.asterix.lang.common.clause.LetClause;
import org.apache.asterix.lang.common.clause.LimitClause;
@@ -231,6 +232,11 @@
}
@Override
+ public Boolean visit(IVisitorExtension ve, ILangExpression arg) throws CompilationException {
+ return ve.checkSubqueryDispatch(this, arg);
+ }
+
+ @Override
public Boolean visit(IfExpr ifexpr, ILangExpression arg) throws CompilationException {
return visit(ifexpr.getCondExpr(), arg) || visit(ifexpr.getThenExpr(), arg) || visit(ifexpr.getElseExpr(), arg);
}
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/DeepCopyVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/DeepCopyVisitor.java
index d74d047..1d43d0b 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/DeepCopyVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/DeepCopyVisitor.java
@@ -28,6 +28,7 @@
import org.apache.asterix.lang.common.base.AbstractClause;
import org.apache.asterix.lang.common.base.Expression;
import org.apache.asterix.lang.common.base.ILangExpression;
+import org.apache.asterix.lang.common.base.IVisitorExtension;
import org.apache.asterix.lang.common.clause.GroupbyClause;
import org.apache.asterix.lang.common.clause.LetClause;
import org.apache.asterix.lang.common.clause.LimitClause;
@@ -193,7 +194,14 @@
if (selectClause.selectRegular()) {
selectRegular = (SelectRegular) selectClause.getSelectRegular().accept(this, arg);
}
- SelectClause copy = new SelectClause(selectElement, selectRegular, selectClause.distinct());
+ List<List<String>> fieldExclusions = new ArrayList<>();
+ if (!selectClause.getFieldExclusions().isEmpty()) {
+ for (List<String> fieldExclusion : selectClause.getFieldExclusions()) {
+ List<String> fieldExclusionCopy = new ArrayList<>(fieldExclusion);
+ fieldExclusions.add(fieldExclusionCopy);
+ }
+ }
+ SelectClause copy = new SelectClause(selectElement, selectRegular, fieldExclusions, selectClause.distinct());
copy.setSourceLocation(selectClause.getSourceLocation());
return copy;
}
@@ -507,6 +515,11 @@
}
@Override
+ public ILangExpression visit(IVisitorExtension ve, Void arg) throws CompilationException {
+ return ve.deepCopyDispatch(this);
+ }
+
+ @Override
public ILangExpression visit(CaseExpression caseExpr, Void arg) throws CompilationException {
Expression conditionExpr = (Expression) caseExpr.getConditionExpr().accept(this, arg);
List<Expression> whenExprList = copyExprList(caseExpr.getWhenExprs(), arg);
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/FreeVariableVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/FreeVariableVisitor.java
index 9115b1c..8ab87e7 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/FreeVariableVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/FreeVariableVisitor.java
@@ -27,6 +27,7 @@
import org.apache.asterix.lang.common.base.AbstractClause;
import org.apache.asterix.lang.common.base.Clause.ClauseType;
import org.apache.asterix.lang.common.base.Expression;
+import org.apache.asterix.lang.common.base.IVisitorExtension;
import org.apache.asterix.lang.common.clause.GroupbyClause;
import org.apache.asterix.lang.common.clause.LetClause;
import org.apache.asterix.lang.common.clause.LimitClause;
@@ -468,6 +469,12 @@
}
@Override
+ public Void visit(IVisitorExtension ve, Collection<VariableExpr> arg) throws CompilationException {
+ ve.freeVariableDispatch(this, arg);
+ return null;
+ }
+
+ @Override
public Void visit(CaseExpression caseExpr, Collection<VariableExpr> freeVars) throws CompilationException {
caseExpr.getConditionExpr().accept(this, freeVars);
visit(caseExpr.getWhenExprs(), freeVars);
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppAstPrintVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppAstPrintVisitor.java
index c803d83..cab6e87 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppAstPrintVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppAstPrintVisitor.java
@@ -22,12 +22,14 @@
import java.util.Collections;
import java.util.List;
import java.util.Map;
+import java.util.stream.Collectors;
import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.common.functions.FunctionSignature;
import org.apache.asterix.common.metadata.DatasetFullyQualifiedName;
import org.apache.asterix.lang.common.base.AbstractClause;
import org.apache.asterix.lang.common.base.Expression;
+import org.apache.asterix.lang.common.base.IVisitorExtension;
import org.apache.asterix.lang.common.base.Literal;
import org.apache.asterix.lang.common.clause.GroupbyClause;
import org.apache.asterix.lang.common.clause.LetClause;
@@ -178,6 +180,11 @@
public Void visit(SelectClause selectClause, Integer step) throws CompilationException {
if (selectClause.selectRegular()) {
selectClause.getSelectRegular().accept(this, step);
+ if (!selectClause.getFieldExclusions().isEmpty()) {
+ out.print(skip(step) + "EXCLUDE ");
+ out.println(selectClause.getFieldExclusions().stream().map(e -> String.join(".", e))
+ .collect(Collectors.joining(",")));
+ }
}
if (selectClause.selectElement()) {
selectClause.getSelectElement().accept(this, step);
@@ -373,6 +380,12 @@
}
@Override
+ public Void visit(IVisitorExtension ve, Integer arg) throws CompilationException {
+ // Language extensions should create a child of this class.
+ return null;
+ }
+
+ @Override
public Void visit(WindowExpression winExpr, Integer step) throws CompilationException {
out.print(skip(step) + "WINDOW ");
printFunctionSignature(out, winExpr.getFunctionSignature(), winExpr.getFunctionSignature().getArity());
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppCloneAndSubstituteVariablesVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppCloneAndSubstituteVariablesVisitor.java
index ce0d0a1..bff57e2 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppCloneAndSubstituteVariablesVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppCloneAndSubstituteVariablesVisitor.java
@@ -269,7 +269,15 @@
} else {
Pair<ILangExpression, VariableSubstitutionEnvironment> newSelectRegular =
selectClause.getSelectRegular().accept(this, env);
- SelectClause newSelectClause = new SelectClause(null, (SelectRegular) newSelectRegular.first, distinct);
+ List<List<String>> fieldExclusions = new ArrayList<>();
+ if (!selectClause.getFieldExclusions().isEmpty()) {
+ for (List<String> fieldExclusion : selectClause.getFieldExclusions()) {
+ List<String> fieldExclusionCopy = new ArrayList<>(fieldExclusion);
+ fieldExclusions.add(fieldExclusionCopy);
+ }
+ }
+ SelectClause newSelectClause =
+ new SelectClause(null, (SelectRegular) newSelectRegular.first, fieldExclusions, distinct);
newSelectClause.setSourceLocation(selectClause.getSourceLocation());
return new Pair<>(newSelectClause, newSelectRegular.second);
}
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppFormatPrintVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppFormatPrintVisitor.java
index 975379d..4fa6bdc 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppFormatPrintVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppFormatPrintVisitor.java
@@ -20,10 +20,12 @@
import java.io.PrintWriter;
import java.util.List;
+import java.util.stream.Collectors;
import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.lang.common.base.AbstractClause;
import org.apache.asterix.lang.common.base.Expression;
+import org.apache.asterix.lang.common.base.IVisitorExtension;
import org.apache.asterix.lang.common.clause.GroupbyClause;
import org.apache.asterix.lang.common.clause.LetClause;
import org.apache.asterix.lang.common.expression.GbyVariableExpressionPair;
@@ -171,6 +173,10 @@
public Void visit(SelectClause selectClause, Integer step) throws CompilationException {
if (selectClause.selectRegular()) {
selectClause.getSelectRegular().accept(this, step);
+ if (!selectClause.getFieldExclusions().isEmpty()) {
+ out.println("exclude " + selectClause.getFieldExclusions().stream().map(e -> String.join(".", e))
+ .collect(Collectors.joining(COMMA)));
+ }
}
if (selectClause.selectElement()) {
selectClause.getSelectElement().accept(this, step);
@@ -284,6 +290,12 @@
}
@Override
+ public Void visit(IVisitorExtension ve, Integer arg) throws CompilationException {
+ // Language extensions should create a child of this class.
+ return null;
+ }
+
+ @Override
public Void visit(InsertStatement insert, Integer step) throws CompilationException {
out.print(skip(step) + "insert into " + datasetSymbol
+ generateFullName(insert.getDataverseName(), insert.getDatasetName()) + "\n");
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppAstVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppAstVisitor.java
index e9c0e4c..7973841 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppAstVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppAstVisitor.java
@@ -19,6 +19,7 @@
package org.apache.asterix.lang.sqlpp.visitor.base;
import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.IVisitorExtension;
import org.apache.asterix.lang.common.expression.ListSliceExpression;
import org.apache.asterix.lang.common.visitor.base.AbstractAstVisitor;
import org.apache.asterix.lang.sqlpp.clause.FromClause;
@@ -121,4 +122,9 @@
public R visit(ListSliceExpression expression, T arg) throws CompilationException {
return null;
}
+
+ @Override
+ public R visit(IVisitorExtension ve, T arg) throws CompilationException {
+ return null;
+ }
}
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppExpressionScopingVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppExpressionScopingVisitor.java
index 7539046..765c2be 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppExpressionScopingVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppExpressionScopingVisitor.java
@@ -32,6 +32,7 @@
import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.lang.common.base.Expression;
import org.apache.asterix.lang.common.base.ILangExpression;
+import org.apache.asterix.lang.common.base.IVisitorExtension;
import org.apache.asterix.lang.common.clause.GroupbyClause;
import org.apache.asterix.lang.common.clause.LetClause;
import org.apache.asterix.lang.common.clause.LimitClause;
@@ -413,6 +414,11 @@
return winExpr;
}
+ @Override
+ public Expression visit(IVisitorExtension ve, ILangExpression arg) throws CompilationException {
+ return ve.variableScopeDispatch(this, arg, scopeChecker);
+ }
+
// Adds a new encountered alias identifier into a scope
private void addNewVarSymbolToScope(Scope scope, VarIdentifier var, SourceLocation sourceLoc,
SqlppVariableAnnotation... varAnnotations) throws CompilationException {
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppSimpleExpressionVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppSimpleExpressionVisitor.java
index ef8b43c..6bbb740 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppSimpleExpressionVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppSimpleExpressionVisitor.java
@@ -25,6 +25,7 @@
import org.apache.asterix.lang.common.base.AbstractClause;
import org.apache.asterix.lang.common.base.Expression;
import org.apache.asterix.lang.common.base.ILangExpression;
+import org.apache.asterix.lang.common.base.IVisitorExtension;
import org.apache.asterix.lang.common.clause.GroupbyClause;
import org.apache.asterix.lang.common.clause.LetClause;
import org.apache.asterix.lang.common.clause.LimitClause;
@@ -384,6 +385,11 @@
}
@Override
+ public Expression visit(IVisitorExtension ve, ILangExpression arg) throws CompilationException {
+ return ve.simpleExpressionDispatch(this, arg);
+ }
+
+ @Override
public Expression visit(CaseExpression caseExpr, ILangExpression arg) throws CompilationException {
caseExpr.setConditionExpr(visit(caseExpr.getConditionExpr(), arg));
caseExpr.setWhenExprs(visit(caseExpr.getWhenExprs(), arg));
@@ -403,7 +409,7 @@
return null;
}
- protected Expression visit(Expression expr, ILangExpression arg) throws CompilationException {
+ public Expression visit(Expression expr, ILangExpression arg) throws CompilationException {
return postVisit(preVisit(expr).accept(this, arg));
}
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
index 4e64944..fe82138 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,59 @@
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:
+ selectivity = 1.0; // uninitialized
+ if (hintToken.hintParams != null) {
+ Matcher mat = lessThanOnePat.matcher(hintToken.hintParams);
+ if (mat.find()) {
+ selectivity = Double.parseDouble (mat.group());
+ }
+ }
+
+ return new PredicateCardinalityAnnotation(selectivity);
+ case JOIN_PREDICATE_PRODUCTIVITY_HINT:
+ productivity = 1.0; // uninitialized
+ String leftSideDataSet = null;
+ if (hintToken.hintParams != 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();
+ Matcher numMat = number.matcher(matchedGroup);
+ if (numMat.find())
+ productivity = Double.parseDouble (numMat.group());
+ }
+ }
+ // 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 +921,7 @@
| stmt = UpsertStatement()
| stmt = ConnectionStatement()
| stmt = CompactStatement()
+ | stmt = AnalyzeStatement()
| stmt = Query()
| stmt = RefreshExternalDatasetStatement()
)
@@ -987,6 +1073,13 @@
(<DATASET>|<COLLECTION>)
}
+void DatasetToken() throws ParseException:
+{
+}
+{
+ Dataset()
+}
+
DatasetDecl CreateDatasetStatement(Token startStmtToken) throws ParseException:
{
DatasetDecl stmt = null;
@@ -1378,7 +1471,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 +2721,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 +3131,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 +3211,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 +3651,7 @@
Token opToken = null;
Expression operand = null;
IExpressionAnnotation annotation = null;
+ List<IExpressionAnnotation> annotationList = new ArrayList<IExpressionAnnotation>();
}
{
operand = BetweenExpr()
@@ -3494,10 +3665,17 @@
}
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);
+ 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 +3705,7 @@
{
if (annotation != null) {
- op.addHint(annotation);
+ op.addHints(annotationList);
}
return op==null? operand: op;
}
@@ -3547,8 +3725,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 +3810,7 @@
boolean not = false;
OperatorExpr op = null;
Expression operand = null;
+ IExpressionAnnotation annotation = null;
}
{
operand = ConcatExpr()
@@ -3639,6 +3818,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 +3836,9 @@
} catch (CompilationException e){
throw new SqlppParseException(getSourceLocation(token), e.getMessage());
}
+ if (annotation != null) {
+ op.addHint(annotation);
+ }
}
operand = ConcatExpr()
@@ -4675,12 +4861,33 @@
Token startToken = null;
SelectRegular selectRegular = null;
SelectElement selectElement = null;
+ List<List<String>> fieldExclusions = new ArrayList<List<String>>();
+ List<String> nestedField = new ArrayList<String>();
+ String identifier;
boolean distinct = false;
}
{
<SELECT> { startToken = token; } (<ALL>|<DISTINCT> { distinct = true; } )?
(
- selectRegular = SelectRegular()
+ (
+ selectRegular = SelectRegular()
+ ( LOOKAHEAD({laIdentifier(EXCLUDE)}) <IDENTIFIER>
+ identifier = Identifier() { nestedField.add(identifier); }
+ ( <DOT> identifier = Identifier() { nestedField.add(identifier); } )*
+ {
+ fieldExclusions.add(nestedField);
+ nestedField = new ArrayList<String>();
+ }
+ ( LOOKAHEAD(1) // Force <COMMA> to be recognized for a nested field in our EXCLUDE list.
+ <COMMA> identifier = Identifier() { nestedField.add(identifier); }
+ ( <DOT> identifier = Identifier() { nestedField.add(identifier); } )*
+ {
+ fieldExclusions.add(nestedField);
+ nestedField = new ArrayList<String>();
+ }
+ )*
+ )?
+ )
|
selectElement = SelectElement()
)?
@@ -4694,7 +4901,7 @@
selectRegular = new SelectRegular(projections);
selectRegular.setSourceLocation(sourceLoc);
}
- SelectClause selectClause = new SelectClause(selectElement, selectRegular, distinct);
+ SelectClause selectClause = new SelectClause(selectElement, selectRegular, fieldExclusions, distinct);
selectClause.setSourceLocation(sourceLoc);
return selectClause;
}
@@ -4752,7 +4959,12 @@
(
<MUL> { kind = Projection.Kind.STAR; startSrcLoc = getSourceLocation(token); }
| LOOKAHEAD(3) expr = VariableRef() <DOT> <MUL> { kind = Projection.Kind.VAR_STAR; }
- | expr = Expression() ((<AS>)? name = Identifier())?
+ | expr = Expression()
+ ( // EXCLUDE is a soft-keyword-- we want to avoid mistaking EXCLUDE as an identifier here.
+ LOOKAHEAD({ getToken(1).kind == AS || getToken(1).kind == QUOTED_STRING
+ || (getToken(1).kind == IDENTIFIER && !laIdentifier(1, EXCLUDE)) })
+ (<AS>)? name = Identifier()
+ )?
{
kind = Projection.Kind.NAMED_EXPR;
if (name == null) {
@@ -4797,7 +5009,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 +5089,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 +5534,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 +5549,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..06f5ba7
--- /dev/null
+++ b/asterixdb/asterix-lang-sqlpp/src/test/java/org/apache/asterix/lang/expression/CommutativeEqualsTest.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.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.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.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 testTwoOperands() {
+ // 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));
+ }
+
+ private void reset() {
+ varCounter = 0;
+ varNameToVarMap.clear();
+ }
+
+ private ILogicalExpression createExpression(FunctionIdentifier fid, char left, char right) {
+ List<Mutable<ILogicalExpression>> args = new ArrayList<>();
+
+ args.add(getVariableExpression(left));
+ args.add(getVariableExpression(right));
+
+ IFunctionInfo funcInfo = BuiltinFunctions.getBuiltinFunctionInfo(fid);
+ return new ScalarFunctionCallExpression(funcInfo, args);
+ }
+
+ private Mutable<ILogicalExpression> getVariableExpression(Character displayName) {
+ LogicalVariable variable = varNameToVarMap.computeIfAbsent(displayName,
+ k -> new LogicalVariable(varCounter++, displayName.toString()));
+ return new MutableObject<>(new VariableReferenceExpression(variable));
+ }
+}
diff --git a/asterixdb/asterix-metadata/pom.xml b/asterixdb/asterix-metadata/pom.xml
index 5729a06..e0b5387 100644
--- a/asterixdb/asterix-metadata/pom.xml
+++ b/asterixdb/asterix-metadata/pom.xml
@@ -180,5 +180,10 @@
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks-ipc</artifactId>
</dependency>
+ <dependency>
+ <groupId>org.apache.asterix</groupId>
+ <artifactId>asterix-column</artifactId>
+ <version>${project.version}</version>
+ </dependency>
</dependencies>
</project>
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
index fd0d30b..8880461 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
@@ -423,16 +423,16 @@
if (createMetadataDataset) {
final double bloomFilterFalsePositiveRate =
appContext.getStorageProperties().getBloomFilterFalsePositiveRate();
- LSMBTreeLocalResourceFactory lsmBtreeFactory =
- new LSMBTreeLocalResourceFactory(storageComponentProvider.getStorageManager(), typeTraits,
- cmpFactories, null, null, null, opTrackerFactory, ioOpCallbackFactory,
- pageWriteCallbackFactory, storageComponentProvider.getMetadataPageManagerFactory(),
- new AsterixVirtualBufferCacheProvider(datasetId),
- storageComponentProvider.getIoOperationSchedulerProvider(),
- appContext.getMetadataMergePolicyFactory(),
- StorageConstants.DEFAULT_COMPACTION_POLICY_PROPERTIES, true, bloomFilterKeyFields,
- bloomFilterFalsePositiveRate, true, null, NoOpCompressorDecompressorFactory.INSTANCE, true,
- TypeTraitProvider.INSTANCE.getTypeTrait(BuiltinType.ANULL), NullIntrospector.INSTANCE);
+ LSMBTreeLocalResourceFactory lsmBtreeFactory = new LSMBTreeLocalResourceFactory(
+ storageComponentProvider.getStorageManager(), typeTraits, cmpFactories, null, null, null,
+ opTrackerFactory, ioOpCallbackFactory, pageWriteCallbackFactory,
+ storageComponentProvider.getMetadataPageManagerFactory(),
+ new AsterixVirtualBufferCacheProvider(datasetId),
+ storageComponentProvider.getIoOperationSchedulerProvider(),
+ appContext.getMetadataMergePolicyFactory(), StorageConstants.DEFAULT_COMPACTION_POLICY_PROPERTIES,
+ true, bloomFilterKeyFields, bloomFilterFalsePositiveRate, true, null,
+ NoOpCompressorDecompressorFactory.INSTANCE, true,
+ TypeTraitProvider.INSTANCE.getTypeTrait(BuiltinType.ANULL), NullIntrospector.INSTANCE, false);
DatasetLocalResourceFactory dsLocalResourceFactory =
new DatasetLocalResourceFactory(datasetId, lsmBtreeFactory);
// TODO(amoudi) Creating the index should be done through the same code path as
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/ArrayBTreeResourceFactoryProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/ArrayBTreeResourceFactoryProvider.java
index c0bdc75..9a2821e 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/ArrayBTreeResourceFactoryProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/ArrayBTreeResourceFactoryProvider.java
@@ -100,7 +100,7 @@
pageWriteCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider,
mergePolicyFactory, mergePolicyProperties, true, null, bloomFilterFalsePositiveRate,
index.isPrimaryIndex(), btreeFields, compDecompFactory, false,
- typeTraitProvider.getTypeTrait(BuiltinType.ANULL), NullIntrospector.INSTANCE);
+ typeTraitProvider.getTypeTrait(BuiltinType.ANULL), NullIntrospector.INSTANCE, false);
default:
throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_DATASET_TYPE,
dataset.getDatasetType().toString());
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/BTreeResourceFactoryProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/BTreeResourceFactoryProvider.java
index 168b56e..d119067 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/BTreeResourceFactoryProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/BTreeResourceFactoryProvider.java
@@ -21,6 +21,7 @@
import java.util.List;
import java.util.Map;
+import org.apache.asterix.common.config.DatasetConfig;
import org.apache.asterix.common.config.DatasetConfig.DatasetType;
import org.apache.asterix.common.context.AsterixVirtualBufferCacheProvider;
import org.apache.asterix.common.context.IStorageComponentProvider;
@@ -92,13 +93,13 @@
filterTypeTraits, filterCmpFactories, filterFields, opTrackerFactory,
ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, true, bloomFilterFields,
- bloomFilterFalsePositiveRate, false, btreeFields, hasBloomFilter,
+ bloomFilterFalsePositiveRate, btreeFields, hasBloomFilter,
typeTraitProvider.getTypeTrait(BuiltinType.ANULL), NullIntrospector.INSTANCE)
: new ExternalBTreeWithBuddyLocalResourceFactory(storageManager, typeTraits, cmpFactories,
filterTypeTraits, filterCmpFactories, filterFields, opTrackerFactory,
ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, true, bloomFilterFields,
- bloomFilterFalsePositiveRate, false, btreeFields, hasBloomFilter,
+ bloomFilterFalsePositiveRate, btreeFields, hasBloomFilter,
typeTraitProvider.getTypeTrait(BuiltinType.ANULL), NullIntrospector.INSTANCE);
case INTERNAL:
AsterixVirtualBufferCacheProvider vbcProvider =
@@ -112,12 +113,15 @@
compDecompFactory = NoOpCompressorDecompressorFactory.INSTANCE;
}
+ boolean isSecondaryNoIncrementalMaintenance = index.getIndexType() == DatasetConfig.IndexType.SAMPLE;
+
return new LSMBTreeLocalResourceFactory(storageManager, typeTraits, cmpFactories, filterTypeTraits,
filterCmpFactories, filterFields, opTrackerFactory, ioOpCallbackFactory,
pageWriteCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider,
mergePolicyFactory, mergePolicyProperties, true, bloomFilterFields,
bloomFilterFalsePositiveRate, index.isPrimaryIndex(), btreeFields, compDecompFactory,
- hasBloomFilter, typeTraitProvider.getTypeTrait(BuiltinType.ANULL), NullIntrospector.INSTANCE);
+ hasBloomFilter, typeTraitProvider.getTypeTrait(BuiltinType.ANULL), NullIntrospector.INSTANCE,
+ isSecondaryNoIncrementalMaintenance);
default:
throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_DATASET_TYPE,
dataset.getDatasetType().toString());
@@ -127,7 +131,7 @@
private static ITypeTraits[] getTypeTraits(MetadataProvider metadataProvider, Dataset dataset, Index index,
ARecordType recordType, ARecordType metaType) throws AlgebricksException {
ITypeTraits[] primaryTypeTraits = dataset.getPrimaryTypeTraits(metadataProvider, recordType, metaType);
- if (index.isPrimaryIndex()) {
+ if (index.isPrimaryIndex() || index.getIndexType() == DatasetConfig.IndexType.SAMPLE) {
return primaryTypeTraits;
} else if (dataset.getDatasetType() == DatasetType.EXTERNAL
&& index.getIndexName().equals(IndexingConstants.getFilesIndexName(dataset.getDatasetName()))) {
@@ -162,7 +166,7 @@
Index index, ARecordType recordType, ARecordType metaType) throws AlgebricksException {
IBinaryComparatorFactory[] primaryCmpFactories =
dataset.getPrimaryComparatorFactories(metadataProvider, recordType, metaType);
- if (index.isPrimaryIndex()) {
+ if (index.isPrimaryIndex() || index.getIndexType() == DatasetConfig.IndexType.SAMPLE) {
return dataset.getPrimaryComparatorFactories(metadataProvider, recordType, metaType);
} else if (dataset.getDatasetType() == DatasetType.EXTERNAL
&& index.getIndexName().equals(IndexingConstants.getFilesIndexName(dataset.getDatasetName()))) {
@@ -200,7 +204,8 @@
if (index.isPrimaryIndex() || index.isPrimaryKeyIndex()) {
return dataset.getPrimaryBloomFilterFields();
}
- if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+ if (dataset.getDatasetType() == DatasetType.EXTERNAL
+ && index.getIndexType() != DatasetConfig.IndexType.SAMPLE) {
if (index.getIndexName().equals(IndexingConstants.getFilesIndexName(dataset.getDatasetName()))) {
return FilesIndexDescription.BLOOM_FILTER_FIELDS;
} else {
@@ -224,6 +229,8 @@
bloomFilterKeyFields[i] = i;
}
return bloomFilterKeyFields;
+ case SAMPLE:
+ return null;
default:
throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE,
String.valueOf(index.getIndexType()));
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DataSource.java
index 454d3da..95b0906 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DataSource.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DataSource.java
@@ -60,6 +60,7 @@
public static final byte FEED = 0x02;
public static final byte LOADABLE = 0x03;
public static final byte FUNCTION = 0x04;
+ public static final byte SAMPLE = 0x05;
// Hide implicit public constructor
private Type() {
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DataSourcePartitioningProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DataSourcePartitioningProvider.java
index 21b744f..324c660 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DataSourcePartitioningProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DataSourcePartitioningProvider.java
@@ -61,6 +61,7 @@
pp = getFeedPartitioningProperty(ds, domain, scanVariables);
break;
case DataSource.Type.INTERNAL_DATASET:
+ case DataSource.Type.SAMPLE:
Set<LogicalVariable> pvars = new ListSet<>();
pp = getInternalDatasetPartitioningProperty(ds, domain, scanVariables, pvars);
propsLocal.add(new LocalOrderProperty(getOrderColumns(pvars)));
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
index 1d82b19..30f5783 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;
@@ -53,6 +55,7 @@
import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
import org.apache.hyracks.api.job.JobSpecification;
import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
+import org.apache.hyracks.storage.am.common.impls.DefaultTupleProjectorFactory;
public class DatasetDataSource extends DataSource {
@@ -78,13 +81,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 +100,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 +123,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 +133,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());
@@ -135,7 +151,7 @@
return metadataProvider.buildBtreeRuntime(jobSpec, opSchema, typeEnv, context, true, false, null,
((DatasetDataSource) dataSource).getDataset(), primaryIndex.getIndexName(), null, null, true,
true, false, null, minFilterFieldIndexes, maxFilterFieldIndexes, tupleFilterFactory,
- outputLimit, false, false);
+ outputLimit, false, false, DefaultTupleProjectorFactory.INSTANCE);
default:
throw new AlgebricksException("Unknown datasource type");
}
@@ -157,6 +173,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();
@@ -174,4 +200,8 @@
return dataset.getDatasetType() == DatasetType.EXTERNAL;
}
+ @Override
+ public boolean compareProperties() {
+ 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..a7d560c 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
@@ -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 };
}
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..dc2d017 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;
@@ -169,6 +174,7 @@
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
import org.apache.hyracks.storage.am.rtree.dataflow.RTreeSearchOperatorDescriptor;
import org.apache.hyracks.storage.common.IStorageManager;
+import org.apache.hyracks.storage.common.projection.ITupleProjectorFactory;
public class MetadataProvider implements IMetadataProvider<DataSourceId, String> {
@@ -182,12 +188,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 +267,6 @@
this.isWriteTransaction = writeTransaction;
}
- public void setWriterFactory(IAWriterFactory writerFactory) {
- this.writerFactory = writerFactory;
- }
-
public void setMetadataTxnContext(MetadataTransactionContext mdTxnCtx) {
this.mdTxnCtx = mdTxnCtx;
}
@@ -274,10 +275,6 @@
return mdTxnCtx;
}
- public IAWriterFactory getWriterFactory() {
- return this.writerFactory;
- }
-
public FileSplit getOutputFile() {
return outputFile;
}
@@ -310,12 +307,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 +434,16 @@
return MetadataManagerUtil.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
}
+ public Index findSampleIndex(DataverseName dataverseName, String datasetName) throws AlgebricksException {
+ Pair<String, String> sampleIndexNames = IndexUtil.getSampleIndexNames(datasetName);
+ Index sampleIndex = getIndex(dataverseName, datasetName, sampleIndexNames.first);
+ if (sampleIndex != null && sampleIndex.getPendingOp() == MetadataUtil.PENDING_NO_OP) {
+ return sampleIndex;
+ }
+ sampleIndex = getIndex(dataverseName, datasetName, sampleIndexNames.second);
+ return sampleIndex != null && sampleIndex.getPendingOp() == MetadataUtil.PENDING_NO_OP ? sampleIndex : null;
+ }
+
public Triple<DataverseName, String, Boolean> resolveDatasetNameUsingSynonyms(DataverseName dataverseName,
String datasetName, boolean includingViews) throws AlgebricksException {
DataverseName dvName = getActiveDataverseName(dataverseName);
@@ -542,7 +549,8 @@
int[] lowKeyFields, int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
boolean propagateFilter, IMissingWriterFactory nonFilterWriterFactory, int[] minFilterFieldIndexes,
int[] maxFilterFieldIndexes, ITupleFilterFactory tupleFilterFactory, long outputLimit,
- boolean isIndexOnlyPlan, boolean isPrimaryIndexPointSearch) throws AlgebricksException {
+ boolean isIndexOnlyPlan, boolean isPrimaryIndexPointSearch, ITupleProjectorFactory tupleProjectorFactory)
+ throws AlgebricksException {
boolean isSecondary = true;
Index primaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
dataset.getDatasetName(), dataset.getDatasetName());
@@ -561,6 +569,12 @@
case BTREE:
numSecondaryKeys = ((Index.ValueIndexDetails) theIndex.getIndexDetails()).getKeyFieldNames().size();
break;
+ case SAMPLE:
+ if (isIndexOnlyPlan) {
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, "");
+ }
+ numSecondaryKeys = 0;
+ break;
default:
throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE,
theIndex.getIndexType().toString());
@@ -601,12 +615,13 @@
? new LSMBTreeBatchPointSearchOperatorDescriptor(jobSpec, outputRecDesc, lowKeyFields,
highKeyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory, retainInput,
retainMissing, nonMatchWriterFactory, searchCallbackFactory, minFilterFieldIndexes,
- maxFilterFieldIndexes, tupleFilterFactory, outputLimit)
+ maxFilterFieldIndexes, tupleFilterFactory, outputLimit, tupleProjectorFactory)
: new BTreeSearchOperatorDescriptor(jobSpec, outputRecDesc, lowKeyFields, highKeyFields,
lowKeyInclusive, highKeyInclusive, indexHelperFactory, retainInput, retainMissing,
nonMatchWriterFactory, searchCallbackFactory, minFilterFieldIndexes, maxFilterFieldIndexes,
propagateFilter, nonFilterWriterFactory, tupleFilterFactory, outputLimit,
- proceedIndexOnlyPlan, failValueForIndexOnlyPlan, successValueForIndexOnlyPlan);
+ proceedIndexOnlyPlan, failValueForIndexOnlyPlan, successValueForIndexOnlyPlan,
+ tupleProjectorFactory);
} else {
btreeSearchOp = new ExternalBTreeSearchOperatorDescriptor(jobSpec, outputRecDesc, lowKeyFields,
highKeyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory, retainInput, retainMissing,
@@ -683,7 +698,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 +707,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 +723,7 @@
ResultWriterOperatorDescriptor resultWriter = null;
try {
IResultSerializerFactory resultSerializedAppenderFactory = resultSerializerFactoryProvider
- .getAqlResultSerializerFactoryProvider(printColumns, printerFactories, getWriterFactory());
+ .getResultSerializerFactoryProvider(printColumns, printerFactories, writerFactory);
resultWriter = new ResultWriterOperatorDescriptor(spec, rsId, metadata, getResultAsyncMode(),
resultSerializedAppenderFactory, getMaxResultReads());
} catch (IOException e) {
@@ -859,7 +876,7 @@
*
* @param dataset
* @return Number of elements that will be used to create a bloom filter per
- * dataset per partition
+ * dataset per partition
* @throws AlgebricksException
*/
public long getCardinalityPerPartitionHint(Dataset dataset) throws AlgebricksException {
@@ -1818,6 +1835,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..8ed3e16
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/SampleDataSource.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.metadata.declared;
+
+import java.util.List;
+
+import org.apache.asterix.metadata.entities.Dataset;
+import org.apache.asterix.metadata.entities.InternalDatasetDetails;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSource;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IProjectionInfo;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
+import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
+import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
+import org.apache.hyracks.storage.am.common.impls.DefaultTupleProjectorFactory;
+
+public class SampleDataSource extends DataSource {
+
+ private final Dataset dataset;
+
+ private final String sampleIndexName;
+
+ public SampleDataSource(Dataset dataset, String sampleIndexName, IAType itemType, IAType metaItemType,
+ INodeDomain domain) throws AlgebricksException {
+ super(createSampleDataSourceId(dataset, sampleIndexName), itemType, metaItemType, Type.SAMPLE, domain);
+ this.dataset = dataset;
+ this.sampleIndexName = sampleIndexName;
+ this.schemaTypes = DatasetDataSource.createSchemaTypesForInternalDataset(itemType, metaItemType,
+ (InternalDatasetDetails) dataset.getDatasetDetails());
+ }
+
+ @Override
+ public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildDatasourceScanRuntime(
+ MetadataProvider metadataProvider, IDataSource<DataSourceId> dataSource,
+ List<LogicalVariable> scanVariables, List<LogicalVariable> projectVariables, boolean projectPushed,
+ List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars,
+ ITupleFilterFactory tupleFilterFactory, long outputLimit, IOperatorSchema opSchema,
+ IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig,
+ IProjectionInfo<?> projectionInfo) throws AlgebricksException {
+ return metadataProvider.buildBtreeRuntime(jobSpec, opSchema, typeEnv, context, true, false, null, dataset,
+ sampleIndexName, null, null, true, true, false, null, null, null, tupleFilterFactory, outputLimit,
+ false, false, DefaultTupleProjectorFactory.INSTANCE);
+ }
+
+ @Override
+ public boolean isScanAccessPathALeaf() {
+ return false;
+ }
+
+ private static DataSourceId createSampleDataSourceId(Dataset dataset, String sampleIndexName) {
+ return new DataSourceId(dataset.getDataverseName(), dataset.getDatasetName(), new String[] { sampleIndexName });
+ }
+}
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..bbccd65 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;
/**
@@ -275,6 +277,7 @@
switch (indexType) {
case ARRAY:
case BTREE:
+ case SAMPLE:
return ResourceType.LSM_BTREE;
case RTREE:
return ResourceType.LSM_RTREE;
@@ -296,7 +299,8 @@
public enum IndexCategory {
VALUE,
TEXT,
- ARRAY;
+ ARRAY,
+ SAMPLE;
public static IndexCategory of(IndexType indexType) {
switch (indexType) {
@@ -310,6 +314,8 @@
return TEXT;
case ARRAY:
return ARRAY;
+ case SAMPLE:
+ return SAMPLE;
default:
throw new IllegalArgumentException(String.valueOf(indexType));
}
@@ -533,6 +539,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/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/functions/ExternalFunctionCompilerUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/functions/ExternalFunctionCompilerUtil.java
index 5fd96a6..ce98008 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/functions/ExternalFunctionCompilerUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/functions/ExternalFunctionCompilerUtil.java
@@ -174,7 +174,7 @@
if (fnKind != FunctionKind.SCALAR) {
return false;
}
- if (!(fnInfo instanceof IExternalFunctionInfo)) {
+ if (!fnInfo.isExternal()) {
return false;
}
ExternalFunctionLanguage language = ((IExternalFunctionInfo) fnInfo).getLanguage();
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ExternalIndexingOperations.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ExternalIndexingOperations.java
index b899e16..c6eafe0 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ExternalIndexingOperations.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ExternalIndexingOperations.java
@@ -30,7 +30,6 @@
import org.apache.asterix.common.config.DatasetConfig.ExternalFilePendingOp;
import org.apache.asterix.common.config.DatasetConfig.TransactionState;
import org.apache.asterix.common.context.IStorageComponentProvider;
-import org.apache.asterix.common.exceptions.NoOpWarningCollector;
import org.apache.asterix.external.api.ITypedAdapterFactory;
import org.apache.asterix.external.indexing.ExternalFile;
import org.apache.asterix.external.indexing.IndexingConstants;
@@ -64,6 +63,7 @@
import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.NoOpWarningCollector;
import org.apache.hyracks.api.exceptions.SourceLocation;
import org.apache.hyracks.api.job.JobSpecification;
import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ISecondaryIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ISecondaryIndexOperationsHelper.java
new file mode 100644
index 0000000..13ff19d
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ISecondaryIndexOperationsHelper.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.metadata.utils;
+
+import java.util.Set;
+
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
+import org.apache.hyracks.storage.am.common.dataflow.IndexDropOperatorDescriptor;
+
+public interface ISecondaryIndexOperationsHelper {
+
+ void init() throws AlgebricksException;
+
+ JobSpecification buildCreationJobSpec() throws AlgebricksException;
+
+ JobSpecification buildLoadingJobSpec() throws AlgebricksException;
+
+ JobSpecification buildCompactJobSpec() throws AlgebricksException;
+
+ JobSpecification buildDropJobSpec(Set<IndexDropOperatorDescriptor.DropOption> options) throws AlgebricksException;
+
+ IFileSplitProvider getSecondaryFileSplitProvider();
+
+ RecordDescriptor getSecondaryRecDesc();
+
+ IBinaryComparatorFactory[] getSecondaryComparatorFactories();
+
+ AlgebricksPartitionConstraint getSecondaryPartitionConstraint();
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java
index ad92803..7e9cfc3 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java
@@ -79,6 +79,8 @@
} else if (index.getIndexType() == DatasetConfig.IndexType.ARRAY) {
numSecondaryKeys = ((Index.ArrayIndexDetails) index.getIndexDetails()).getElementList().stream()
.map(e -> e.getProjectList().size()).reduce(0, Integer::sum);
+ } else if (index.getIndexType() == DatasetConfig.IndexType.SAMPLE) {
+ return null;
} else {
throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE, index.getIndexType().toString());
}
@@ -111,6 +113,8 @@
case SINGLE_PARTITION_NGRAM_INVIX:
case SINGLE_PARTITION_WORD_INVIX:
break;
+ case SAMPLE:
+ break;
default:
throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE,
index.getIndexType().toString());
@@ -120,21 +124,21 @@
public static JobSpecification buildDropIndexJobSpec(Index index, MetadataProvider metadataProvider,
Dataset dataset, SourceLocation sourceLoc) throws AlgebricksException {
- SecondaryIndexOperationsHelper secondaryIndexHelper =
+ ISecondaryIndexOperationsHelper secondaryIndexHelper =
SecondaryIndexOperationsHelper.createIndexOperationsHelper(dataset, index, metadataProvider, sourceLoc);
return secondaryIndexHelper.buildDropJobSpec(EnumSet.noneOf(DropOption.class));
}
public static JobSpecification buildDropIndexJobSpec(Index index, MetadataProvider metadataProvider,
Dataset dataset, Set<DropOption> options, SourceLocation sourceLoc) throws AlgebricksException {
- SecondaryIndexOperationsHelper secondaryIndexHelper =
+ ISecondaryIndexOperationsHelper secondaryIndexHelper =
SecondaryIndexOperationsHelper.createIndexOperationsHelper(dataset, index, metadataProvider, sourceLoc);
return secondaryIndexHelper.buildDropJobSpec(options);
}
public static JobSpecification buildSecondaryIndexCreationJobSpec(Dataset dataset, Index index,
MetadataProvider metadataProvider, SourceLocation sourceLoc) throws AlgebricksException {
- SecondaryIndexOperationsHelper secondaryIndexHelper =
+ ISecondaryIndexOperationsHelper secondaryIndexHelper =
SecondaryIndexOperationsHelper.createIndexOperationsHelper(dataset, index, metadataProvider, sourceLoc);
return secondaryIndexHelper.buildCreationJobSpec();
}
@@ -147,8 +151,8 @@
public static JobSpecification buildSecondaryIndexLoadingJobSpec(Dataset dataset, Index index,
MetadataProvider metadataProvider, List<ExternalFile> files, SourceLocation sourceLoc)
throws AlgebricksException {
- SecondaryIndexOperationsHelper secondaryIndexHelper;
- if (dataset.isCorrelated()) {
+ ISecondaryIndexOperationsHelper secondaryIndexHelper;
+ if (dataset.isCorrelated() && supportsCorrelated(index.getIndexType())) { //TODO:REVISIT
secondaryIndexHelper = SecondaryCorrelatedTreeIndexOperationsHelper.createIndexOperationsHelper(dataset,
index, metadataProvider, sourceLoc);
} else {
@@ -156,14 +160,18 @@
metadataProvider, sourceLoc);
}
if (files != null) {
- secondaryIndexHelper.setExternalFiles(files);
+ ((SecondaryIndexOperationsHelper) secondaryIndexHelper).setExternalFiles(files);
}
return secondaryIndexHelper.buildLoadingJobSpec();
}
+ private static boolean supportsCorrelated(DatasetConfig.IndexType indexType) {
+ return indexType != DatasetConfig.IndexType.SAMPLE;
+ }
+
public static JobSpecification buildSecondaryIndexCompactJobSpec(Dataset dataset, Index index,
MetadataProvider metadataProvider, SourceLocation sourceLoc) throws AlgebricksException {
- SecondaryIndexOperationsHelper secondaryIndexHelper =
+ ISecondaryIndexOperationsHelper secondaryIndexHelper =
SecondaryIndexOperationsHelper.createIndexOperationsHelper(dataset, index, metadataProvider, sourceLoc);
return secondaryIndexHelper.buildCompactJobSpec();
}
@@ -240,4 +248,9 @@
public static boolean excludesUnknowns(Index index) {
return !includesUnknowns(index);
}
+
+ public static Pair<String, String> getSampleIndexNames(String datasetName) {
+ return new Pair<>(MetadataConstants.SAMPLE_INDEX_1_PREFIX + datasetName,
+ MetadataConstants.SAMPLE_INDEX_2_PREFIX + datasetName);
+ }
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/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/SecondaryIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
index 9e0d650..694b153 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
@@ -91,7 +91,7 @@
// TODO: We should eventually have a hierarchy of classes that can create all
// possible index job specs,
// not just for creation.
-public abstract class SecondaryIndexOperationsHelper {
+public abstract class SecondaryIndexOperationsHelper implements ISecondaryIndexOperationsHelper {
protected final MetadataProvider metadataProvider;
protected final Dataset dataset;
protected final Index index;
@@ -160,10 +160,10 @@
metadataProvider.getConfig(), sourceLoc);
}
- public static SecondaryIndexOperationsHelper createIndexOperationsHelper(Dataset dataset, Index index,
+ public static ISecondaryIndexOperationsHelper createIndexOperationsHelper(Dataset dataset, Index index,
MetadataProvider metadataProvider, SourceLocation sourceLoc) throws AlgebricksException {
- SecondaryIndexOperationsHelper indexOperationsHelper;
+ ISecondaryIndexOperationsHelper indexOperationsHelper;
switch (index.getIndexType()) {
case ARRAY:
indexOperationsHelper =
@@ -182,6 +182,9 @@
indexOperationsHelper =
new SecondaryInvertedIndexOperationsHelper(dataset, index, metadataProvider, sourceLoc);
break;
+ case SAMPLE:
+ indexOperationsHelper = new SampleOperationsHelper(dataset, index, metadataProvider, sourceLoc);
+ break;
default:
throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE, sourceLoc,
index.getIndexType());
@@ -202,7 +205,8 @@
protected abstract int getNumSecondaryKeys();
- protected void init() throws AlgebricksException {
+ @Override
+ public void init() throws AlgebricksException {
payloadSerde = SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(itemType);
metaSerde =
metaType == null ? null : SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(metaType);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryTreeIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryTreeIndexOperationsHelper.java
index 0373e32..e8825fc 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryTreeIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryTreeIndexOperationsHelper.java
@@ -72,13 +72,19 @@
@Override
public JobSpecification buildDropJobSpec(Set<DropOption> options) throws AlgebricksException {
+ return buildDropJobSpecImpl(dataset, index, options, metadataProvider, sourceLoc);
+ }
+
+ static JobSpecification buildDropJobSpecImpl(Dataset dataset, Index index, Set<DropOption> dropOptions,
+ MetadataProvider metadataProvider, SourceLocation sourceLoc) throws AlgebricksException {
JobSpecification spec = RuntimeUtils.createJobSpecification(metadataProvider.getApplicationContext());
Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
metadataProvider.getSplitProviderAndConstraints(dataset, index.getIndexName());
IIndexDataflowHelperFactory dataflowHelperFactory = new IndexDataflowHelperFactory(
metadataProvider.getStorageComponentProvider().getStorageManager(), splitsAndConstraint.first);
// The index drop operation should be persistent regardless of temp datasets or permanent dataset.
- IndexDropOperatorDescriptor btreeDrop = new IndexDropOperatorDescriptor(spec, dataflowHelperFactory, options);
+ IndexDropOperatorDescriptor btreeDrop =
+ new IndexDropOperatorDescriptor(spec, dataflowHelperFactory, dropOptions);
btreeDrop.setSourceLocation(sourceLoc);
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, btreeDrop,
splitsAndConstraint.second);
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/builders/RecordBuilder.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/builders/RecordBuilder.java
index 58e59d6..d08865b 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/builders/RecordBuilder.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/builders/RecordBuilder.java
@@ -186,9 +186,16 @@
if (data[offset] == ATypeTag.SERIALIZED_MISSING_TYPE_TAG) {
return;
}
- // ignore adding duplicate fields
byte[] nameBytes = name.getByteArray();
- int nameStart = name.getStartOffset() + 1;
+ int nameOffset = name.getStartOffset();
+ // ignore adding fields with NULL/MISSING names
+ if (nameBytes[nameOffset] == ATypeTag.SERIALIZED_MISSING_TYPE_TAG
+ || nameBytes[nameOffset] == ATypeTag.SERIALIZED_NULL_TYPE_TAG) {
+ // TODO(ali): issue a warning
+ return;
+ }
+ // ignore adding duplicate fields
+ int nameStart = nameOffset + 1;
int nameLength = name.getLength() - 1;
if (recType != null && recTypeInfo.getFieldIndex(nameBytes, nameStart, nameLength) >= 0) {
// TODO(ali): issue a warning
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AUUIDPartialBinaryComparatorFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AUUIDPartialBinaryComparatorFactory.java
index 01f2537..d842c89 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AUUIDPartialBinaryComparatorFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AUUIDPartialBinaryComparatorFactory.java
@@ -23,6 +23,7 @@
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.io.IJsonSerializable;
import org.apache.hyracks.api.io.IPersistedResourceRegistry;
+import org.apache.hyracks.data.std.api.IValueReference;
import org.apache.hyracks.data.std.primitive.LongPointable;
import com.fasterxml.jackson.databind.JsonNode;
@@ -37,6 +38,11 @@
return AUUIDPartialBinaryComparatorFactory::compare;
}
+ public static int compare(IValueReference valueA, IValueReference valueB) {
+ return compare(valueA.getByteArray(), valueA.getStartOffset(), valueA.getLength(), valueB.getByteArray(),
+ valueB.getStartOffset(), valueB.getLength());
+ }
+
@SuppressWarnings("squid:S1172") // unused parameter
public static int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
int msbCompare = Long.compare(LongPointable.getLong(b1, s1), LongPointable.getLong(b2, s2));
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ABoolean.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ABoolean.java
index 5589205..0664fec 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ABoolean.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ABoolean.java
@@ -50,7 +50,7 @@
@Override
public String toString() {
- return Boolean.toString(bVal).toUpperCase();
+ return Boolean.toString(bVal);
}
@Override
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ARecord.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ARecord.java
index d964079..cb6c5fe 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ARecord.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ARecord.java
@@ -20,6 +20,7 @@
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.utils.RecordUtil;
+import org.apache.hyracks.util.JSONUtil;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.node.ArrayNode;
@@ -89,7 +90,7 @@
if (i > 0) {
sb.append(", ");
}
- sb.append(type.getFieldNames()[i]);
+ JSONUtil.quoteAndEscape(sb, type.getFieldNames()[i]);
sb.append(": ");
sb.append(fields[i]);
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
index 0c9f0dd..cdc8111 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
@@ -119,8 +119,12 @@
import org.apache.asterix.om.typecomputer.impl.OrderedListOfAnyTypeComputer;
import org.apache.asterix.om.typecomputer.impl.PropagateTypeComputer;
import org.apache.asterix.om.typecomputer.impl.RecordAddFieldsTypeComputer;
+import org.apache.asterix.om.typecomputer.impl.RecordAddTypeComputer;
import org.apache.asterix.om.typecomputer.impl.RecordMergeTypeComputer;
+import org.apache.asterix.om.typecomputer.impl.RecordPutTypeComputer;
import org.apache.asterix.om.typecomputer.impl.RecordRemoveFieldsTypeComputer;
+import org.apache.asterix.om.typecomputer.impl.RecordRemoveTypeComputer;
+import org.apache.asterix.om.typecomputer.impl.RecordRenameTypeComputer;
import org.apache.asterix.om.typecomputer.impl.ScalarArrayAggTypeComputer;
import org.apache.asterix.om.typecomputer.impl.ScalarVersionOfAggregateResultType;
import org.apache.asterix.om.typecomputer.impl.SleepTypeComputer;
@@ -250,6 +254,12 @@
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "array-slice", 3);
public static final FunctionIdentifier ARRAY_EXCEPT =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "array-except", 2);
+ public static final FunctionIdentifier ARRAY_SWAP =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "array-swap", 3);
+ public static final FunctionIdentifier ARRAY_MOVE =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "array-move", 3);
+ public static final FunctionIdentifier ARRAY_BINARY_SEARCH =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "array-binary-search", 2);
// objects
public static final FunctionIdentifier RECORD_MERGE =
@@ -1699,6 +1709,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 +1812,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);
@@ -2407,6 +2418,9 @@
addFunction(ARRAY_SLICE_WITH_END_POSITION, AListTypeComputer.INSTANCE_SLICE, true);
addFunction(ARRAY_SLICE_WITHOUT_END_POSITION, AListTypeComputer.INSTANCE_SLICE, true);
addFunction(ARRAY_EXCEPT, ArrayExceptTypeComputer.INSTANCE, true);
+ addFunction(ARRAY_MOVE, AListTypeComputer.INSTANCE_MOVE, true);
+ addFunction(ARRAY_SWAP, AListTypeComputer.INSTANCE_SWAP, true);
+ addFunction(ARRAY_BINARY_SEARCH, AInt32TypeComputer.INSTANCE_NULLABLE, true);
// objects
addFunction(RECORD_MERGE, RecordMergeTypeComputer.INSTANCE, true);
@@ -2427,12 +2441,12 @@
addFunction(RECORD_PAIRS, OrderedListOfAnyTypeComputer.INSTANCE_NULLABLE, true);
addFunction(PAIRS, OrderedListOfAnyTypeComputer.INSTANCE_NULLABLE, true);
addFunction(GEOMETRY_CONSTRUCTOR, AGeometryTypeComputer.INSTANCE, true);
- addFunction(RECORD_REMOVE, OpenARecordTypeComputer.INSTANCE, true);
- addFunction(RECORD_RENAME, OpenARecordTypeComputer.INSTANCE, true);
+ addFunction(RECORD_REMOVE, RecordRemoveTypeComputer.INSTANCE, true);
+ addFunction(RECORD_RENAME, RecordRenameTypeComputer.INSTANCE, true);
addFunction(RECORD_UNWRAP, AnyTypeComputer.INSTANCE, true);
addFunction(RECORD_REPLACE, OpenARecordTypeComputer.INSTANCE, true);
- addFunction(RECORD_ADD, OpenARecordTypeComputer.INSTANCE, true);
- addFunction(RECORD_PUT, OpenARecordTypeComputer.INSTANCE, true);
+ addFunction(RECORD_ADD, RecordAddTypeComputer.INSTANCE, true);
+ addFunction(RECORD_PUT, RecordPutTypeComputer.INSTANCE, true);
addFunction(RECORD_VALUES, OrderedListOfAnyTypeComputer.INSTANCE, true);
// temporal type accessors
@@ -2532,6 +2546,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);
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/ExternalFunctionInfo.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/ExternalFunctionInfo.java
index a3ec9c6..8a2cefa 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/ExternalFunctionInfo.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/ExternalFunctionInfo.java
@@ -101,4 +101,9 @@
public boolean getNullCall() {
return nullCall;
}
+
+ @Override
+ public boolean isExternal() {
+ return true;
+ }
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/AbstractLazyNestedVisitablePointable.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/AbstractLazyNestedVisitablePointable.java
new file mode 100644
index 0000000..e9f8e8a
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/AbstractLazyNestedVisitablePointable.java
@@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.lazy;
+
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AbstractCollectionType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+
+/**
+ * A common implementation for nested values (i.e., {@link ATypeTag#OBJECT}, {@link ATypeTag#ARRAY}, and
+ * {@link ATypeTag#MULTISET}).
+ * <p>
+ * Contract:
+ * <p>
+ * 1- A child's value may or may not contain a type tag. Thus, it is the responsibility of the caller to check if the
+ * child's value contains a type tag by calling {@link #isTaggedChild()}.
+ * 2- The returned objects from {@link #getChildVisitablePointable()} and {@link #getChildValue()}, are reused
+ * when possible. Thus, when the caller does the following for example:
+ * <p>
+ * AbstractLazyVisitablePointable child1 = visitablePointable.getChildVisitablePointable();
+ * visitablePointable.nextChild();
+ * AbstractLazyVisitablePointable child2 = visitablePointable.getChildVisitablePointable();
+ * <p>
+ * both child1 and child2 may have the same value, which is the value of the second child.
+ */
+public abstract class AbstractLazyNestedVisitablePointable extends AbstractLazyVisitablePointable {
+ private final ATypeTag typeTag;
+ protected final VoidPointable currentValue;
+ protected byte currentChildTypeTag;
+
+ AbstractLazyNestedVisitablePointable(boolean tagged, ATypeTag typeTag) {
+ super(tagged);
+ this.typeTag = typeTag;
+ currentValue = new VoidPointable();
+ }
+
+ /**
+ * Prepare the value and the tag of the next child
+ */
+ public abstract void nextChild() throws HyracksDataException;
+
+ /**
+ * If the child contains a tag
+ *
+ * @return true if the child is tagged (open value), false otherwise
+ */
+ public abstract boolean isTaggedChild();
+
+ /**
+ * @return number of children
+ */
+ public abstract int getNumberOfChildren();
+
+ /**
+ * Gets a child visitable-pointable.
+ */
+ public abstract AbstractLazyVisitablePointable getChildVisitablePointable() throws HyracksDataException;
+
+ /**
+ * Returns a value reference of the child. Note that this is not a visitable-pointable reference.
+ */
+ public final IValueReference getChildValue() {
+ return currentValue;
+ }
+
+ /**
+ * The serialized type tag of a child
+ */
+ public final byte getChildSerializedTypeTag() {
+ return currentChildTypeTag;
+ }
+
+ /**
+ * The type tag of a child
+ */
+ public final ATypeTag getChildTypeTag() {
+ return ATypeTag.VALUE_TYPE_MAPPING[currentChildTypeTag];
+ }
+
+ /**
+ * @return The type tag that corresponds to {@code this} visitable-pointable
+ */
+ @Override
+ public final ATypeTag getTypeTag() {
+ return ATypeTag.VALUE_TYPE_MAPPING[getSerializedTypeTag()];
+ }
+
+ /**
+ * @return The serialized type tag that corresponds to {@code this} visitable-pointable
+ */
+ @Override
+ public final byte getSerializedTypeTag() {
+ return typeTag.serialize();
+ }
+
+ /**
+ * Helper method to create a typed (i.e., non-tagged) visitable-pointable
+ *
+ * @param type the required type
+ * @return a visitable pointable that corresponds to {@code type}
+ */
+ static AbstractLazyVisitablePointable createVisitable(IAType type) {
+ ATypeTag typeTag = type.getTypeTag();
+ switch (typeTag) {
+ case OBJECT:
+ return new TypedRecordLazyVisitablePointable(false, (ARecordType) type);
+ case ARRAY:
+ case MULTISET:
+ AbstractCollectionType listType = (AbstractCollectionType) type;
+ return NonTaggedFormatUtil.isFixedSizedCollection(listType.getItemType())
+ ? new FixedListLazyVisitablePointable(false, listType)
+ : new VariableListLazyVisitablePointable(false, listType);
+ default:
+ return new FlatLazyVisitablePointable(false, typeTag);
+
+ }
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/AbstractLazyVisitablePointable.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/AbstractLazyVisitablePointable.java
new file mode 100644
index 0000000..1d6425c
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/AbstractLazyVisitablePointable.java
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.lazy;
+
+import org.apache.asterix.om.pointables.base.IVisitablePointable;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.api.IValueReference;
+
+/**
+ * Similar to {@link IVisitablePointable}. The difference is, upon calling {@link #set(byte[], int, int)}, the nested
+ * values (children) will not call {@link #set(byte[], int, int)} recursively. Instead, it will wait until the
+ * child is accessed. Thus, when a processor (a class that implements {@link ILazyVisitablePointableVisitor}) wants
+ * to traverse the object in a <b>DFS mode</b>, the traversal will be done in a single pass - compared to the two passes
+ * when using the {@link IVisitablePointable}, where one pass is done when calling
+ * {@link IVisitablePointable#set(byte[], int, int)} and another pass is done by the processor (e.g., the result
+ * printer). Also, the lazy visitable-pointable requires less memory as we do not allocate any temporary buffers.
+ */
+public abstract class AbstractLazyVisitablePointable implements IPointable {
+ private final boolean tagged;
+ private byte[] data;
+ private int offset;
+ private int length;
+
+ AbstractLazyVisitablePointable(boolean tagged) {
+ this.tagged = tagged;
+ }
+
+ @Override
+ public final void set(byte[] data, int offset, int length) {
+ this.data = data;
+ this.offset = offset;
+ this.length = length;
+ init(data, offset, length);
+ }
+
+ @Override
+ public final void set(IValueReference pointer) {
+ set(pointer.getByteArray(), pointer.getStartOffset(), pointer.getLength());
+ }
+
+ @Override
+ public final byte[] getByteArray() {
+ return data;
+ }
+
+ @Override
+ public final int getStartOffset() {
+ return offset;
+ }
+
+ @Override
+ public final int getLength() {
+ return length;
+ }
+
+ /**
+ * @return The serialized type tag
+ */
+ public abstract byte getSerializedTypeTag();
+
+ /**
+ * @return The type tag
+ */
+ public abstract ATypeTag getTypeTag();
+
+ public abstract <R, T> R accept(ILazyVisitablePointableVisitor<R, T> visitor, T arg) throws HyracksDataException;
+
+ /**
+ * @return true if the value contains tag, false otherwise
+ */
+ public final boolean isTagged() {
+ return tagged;
+ }
+
+ /**
+ * Called by {@link #set(byte[], int, int)} to initialize the visitable-pointable
+ *
+ * @param data value's data
+ * @param offset value's start offset
+ * @param length value's length
+ */
+ abstract void init(byte[] data, int offset, int length);
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/AbstractListLazyVisitablePointable.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/AbstractListLazyVisitablePointable.java
new file mode 100644
index 0000000..68d558c
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/AbstractListLazyVisitablePointable.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.asterix.om.lazy;
+
+import java.util.Objects;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AbstractCollectionType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * Common implementation for both {@link ATypeTag#ARRAY} and {@link ATypeTag#MULTISET}
+ */
+public abstract class AbstractListLazyVisitablePointable extends AbstractLazyNestedVisitablePointable {
+ private final int headerSize;
+ private final AbstractLazyVisitablePointable itemVisitablePointable;
+ private int numberOfItems;
+ protected int currentIndex;
+ protected int itemsOffset;
+
+ AbstractListLazyVisitablePointable(boolean tagged, AbstractCollectionType listType) {
+ super(tagged, listType.getTypeTag());
+ Objects.requireNonNull(listType);
+ Objects.requireNonNull(listType.getItemType());
+ //1 for typeTag if tagged, 1 for itemTypeTag, 4 for length
+ headerSize = (isTagged() ? 1 : 0) + 1 + 4;
+ itemVisitablePointable = createVisitablePointable(listType.getItemType());
+ }
+
+ @Override
+ public final int getNumberOfChildren() {
+ return numberOfItems;
+ }
+
+ @Override
+ final void init(byte[] data, int offset, int length) {
+ int pointer = headerSize + offset;
+ numberOfItems = AInt32SerializerDeserializer.getInt(data, pointer);
+ itemsOffset = pointer + 4;
+ currentIndex = 0;
+ }
+
+ @Override
+ public AbstractLazyVisitablePointable getChildVisitablePointable() throws HyracksDataException {
+ itemVisitablePointable.set(getChildValue());
+ return itemVisitablePointable;
+ }
+
+ @Override
+ public <R, T> R accept(ILazyVisitablePointableVisitor<R, T> visitor, T arg) throws HyracksDataException {
+ return visitor.visit(this, arg);
+ }
+
+ abstract AbstractLazyVisitablePointable createVisitablePointable(IAType itemType);
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/FixedListLazyVisitablePointable.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/FixedListLazyVisitablePointable.java
new file mode 100644
index 0000000..f3153b2
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/FixedListLazyVisitablePointable.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.om.lazy;
+
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AbstractCollectionType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * This implementation is only for closed {@link ATypeTag#ARRAY} and {@link ATypeTag#MULTISET} with fixed-length
+ * items.
+ */
+public class FixedListLazyVisitablePointable extends AbstractListLazyVisitablePointable {
+ private final int itemSize;
+
+ public FixedListLazyVisitablePointable(boolean tagged, AbstractCollectionType listType) {
+ super(tagged, listType);
+ ATypeTag itemTag = listType.getItemType().getTypeTag();
+ currentChildTypeTag = itemTag.serialize();
+ try {
+ itemSize = NonTaggedFormatUtil.getFieldValueLength(null, -1, itemTag, false);
+ } catch (HyracksDataException e) {
+ throw new IllegalStateException(e);
+ }
+
+ }
+
+ @Override
+ public void nextChild() {
+ byte[] data = getByteArray();
+ int itemOffset = getStartOffset() + itemsOffset + currentIndex * itemSize;
+ currentValue.set(data, itemOffset, itemSize);
+ currentIndex++;
+ }
+
+ @Override
+ public boolean isTaggedChild() {
+ return false;
+ }
+
+ @Override
+ AbstractLazyVisitablePointable createVisitablePointable(IAType itemType) {
+ return createVisitable(itemType);
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/FlatLazyVisitablePointable.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/FlatLazyVisitablePointable.java
new file mode 100644
index 0000000..f441c7b
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/FlatLazyVisitablePointable.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.om.lazy;
+
+import java.util.Objects;
+
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class FlatLazyVisitablePointable extends AbstractLazyVisitablePointable {
+ private final ATypeTag typeTag;
+
+ public FlatLazyVisitablePointable(boolean tagged, ATypeTag typeTag) {
+ super(tagged);
+ Objects.requireNonNull(typeTag);
+ this.typeTag = typeTag;
+ }
+
+ @Override
+ public byte getSerializedTypeTag() {
+ if (isTagged()) {
+ return getByteArray()[getStartOffset()];
+ }
+ return typeTag.serialize();
+ }
+
+ @Override
+ public ATypeTag getTypeTag() {
+ if (isTagged()) {
+ return ATypeTag.VALUE_TYPE_MAPPING[getSerializedTypeTag()];
+ }
+ return typeTag;
+ }
+
+ @Override
+ public <R, T> R accept(ILazyVisitablePointableVisitor<R, T> visitor, T arg) throws HyracksDataException {
+ return visitor.visit(this, arg);
+ }
+
+ @Override
+ void init(byte[] data, int offset, int length) {
+ //noOp
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/GenericLazyVisitablePointable.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/GenericLazyVisitablePointable.java
new file mode 100644
index 0000000..f6d06ee
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/GenericLazyVisitablePointable.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.om.lazy;
+
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * This is a generic lazy visitable-pointable for tagged values (a.k.a. open values). Each nested visitable-pointable
+ * should only allocate a single instance of this class and reuse it for every open value.
+ */
+public class GenericLazyVisitablePointable extends AbstractLazyVisitablePointable {
+ private RecordLazyVisitablePointable object;
+ private VariableListLazyVisitablePointable array;
+ private VariableListLazyVisitablePointable multiset;
+ private FlatLazyVisitablePointable flat;
+
+ private AbstractLazyVisitablePointable current;
+
+ public GenericLazyVisitablePointable() {
+ super(true);
+ }
+
+ @Override
+ public final byte getSerializedTypeTag() {
+ return current.getSerializedTypeTag();
+ }
+
+ @Override
+ public final ATypeTag getTypeTag() {
+ return current.getTypeTag();
+ }
+
+ @Override
+ public <R, T> R accept(ILazyVisitablePointableVisitor<R, T> visitor, T arg) throws HyracksDataException {
+ switch (current.getTypeTag()) {
+ case OBJECT:
+ return visitor.visit(object, arg);
+ case ARRAY:
+ return visitor.visit(array, arg);
+ case MULTISET:
+ return visitor.visit(multiset, arg);
+ default:
+ return visitor.visit(flat, arg);
+ }
+ }
+
+ @Override
+ void init(byte[] data, int offset, int length) {
+ ATypeTag typeTag = ATypeTag.VALUE_TYPE_MAPPING[data[offset]];
+ AbstractLazyVisitablePointable visitable = getOrCreateVisitablePointable(typeTag);
+ visitable.set(data, offset, length);
+ current = visitable;
+ }
+
+ private AbstractLazyVisitablePointable getOrCreateVisitablePointable(ATypeTag typeTag) {
+ switch (typeTag) {
+ case OBJECT:
+ object = object == null ? new RecordLazyVisitablePointable(true) : object;
+ return object;
+ case ARRAY:
+ array = array == null ? new VariableListLazyVisitablePointable(true,
+ DefaultOpenFieldType.NESTED_OPEN_AORDERED_LIST_TYPE) : array;
+ return array;
+ case MULTISET:
+ multiset = multiset == null ? new VariableListLazyVisitablePointable(true,
+ DefaultOpenFieldType.NESTED_OPEN_AUNORDERED_LIST_TYPE) : multiset;
+ return multiset;
+ default:
+ flat = flat == null ? new FlatLazyVisitablePointable(true, ATypeTag.ANY) : flat;
+ return flat;
+ }
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/ILazyVisitablePointableVisitor.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/ILazyVisitablePointableVisitor.java
new file mode 100644
index 0000000..50116cd
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/ILazyVisitablePointableVisitor.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.om.lazy;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * A visitor for ADM values which utilizes the lazy visitable:
+ *
+ * @param <R> return type
+ * @param <T> argument type
+ * @see AbstractLazyVisitablePointable
+ * @see AbstractLazyNestedVisitablePointable
+ */
+public interface ILazyVisitablePointableVisitor<R, T> {
+ /**
+ * Visit record value
+ *
+ * @param pointable either typed {@link TypedRecordLazyVisitablePointable} or
+ * untyped {@link RecordLazyVisitablePointable}
+ * @param arg visitor argument
+ * @return return value
+ */
+ R visit(RecordLazyVisitablePointable pointable, T arg) throws HyracksDataException;
+
+ /**
+ * Visit list value
+ *
+ * @param pointable either a list with fixed-length items {@link FixedListLazyVisitablePointable} or
+ * a list with variable-length items {@link VariableListLazyVisitablePointable}
+ * @param arg visitor argument
+ * @return return value
+ */
+
+ R visit(AbstractListLazyVisitablePointable pointable, T arg) throws HyracksDataException;
+
+ /**
+ * Atomic values
+ *
+ * @param pointable any flat item (e.g., {@link org.apache.asterix.om.types.ATypeTag#BIGINT}
+ * @param arg visitor argument
+ * @return return value
+ * @throws HyracksDataException
+ */
+ R visit(FlatLazyVisitablePointable pointable, T arg) throws HyracksDataException;
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/RecordLazyVisitablePointable.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/RecordLazyVisitablePointable.java
new file mode 100644
index 0000000..154b9f2
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/RecordLazyVisitablePointable.java
@@ -0,0 +1,125 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.lazy;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+
+/**
+ * This implementation is to handle {@link ATypeTag#OBJECT} with open fields only
+ */
+public class RecordLazyVisitablePointable extends AbstractLazyNestedVisitablePointable {
+ protected final VoidPointable currentFieldName;
+ protected final AbstractLazyVisitablePointable openVisitable;
+ private int openValuesOffset;
+ private int numberOfOpenChildren;
+
+ public RecordLazyVisitablePointable(boolean tagged) {
+ super(tagged, ATypeTag.OBJECT);
+ currentFieldName = new VoidPointable();
+ openVisitable = new GenericLazyVisitablePointable();
+ }
+
+ @Override
+ public void nextChild() throws HyracksDataException {
+ byte[] data = getByteArray();
+
+ //set field name
+ int fieldNameLength = NonTaggedFormatUtil.getFieldValueLength(data, openValuesOffset, ATypeTag.STRING, false);
+ currentFieldName.set(data, openValuesOffset, fieldNameLength);
+ openValuesOffset += fieldNameLength;
+
+ //set Type tag
+ currentChildTypeTag = data[openValuesOffset];
+
+ //set value
+ int valueLength = NonTaggedFormatUtil.getFieldValueLength(data, openValuesOffset, getChildTypeTag(), true) + 1;
+ currentValue.set(data, openValuesOffset, valueLength);
+ openValuesOffset += valueLength;
+ }
+
+ @Override
+ public boolean isTaggedChild() {
+ return true;
+ }
+
+ @Override
+ public int getNumberOfChildren() {
+ return numberOfOpenChildren;
+ }
+
+ public IValueReference getFieldName() {
+ return currentFieldName;
+ }
+
+ @Override
+ public AbstractLazyVisitablePointable getChildVisitablePointable() throws HyracksDataException {
+ openVisitable.set(getChildValue());
+ return openVisitable;
+ }
+
+ @Override
+ public final <R, T> R accept(ILazyVisitablePointableVisitor<R, T> visitor, T arg) throws HyracksDataException {
+ return visitor.visit(this, arg);
+ }
+
+ @Override
+ void init(byte[] data, int offset, int length) {
+ initOpenPart(data, offset);
+ }
+
+ /* ******************************************************
+ * Init Open part
+ * ******************************************************
+ */
+ protected int initOpenPart(byte[] data, int pointer) {
+ //+1 for type tag and +4 for the length
+ int skipTypeTag = isTagged() ? 1 : 0;
+ int currentPointer = pointer + skipTypeTag + 4;
+
+ boolean isExpanded = data[currentPointer] == 1;
+ //Advance to numberOfClosedChildren or
+ currentPointer++;
+
+ if (isExpanded) {
+ /*
+ * -(isTagged() ? 0 : 1) because the open part is computed - in the record builder - with the
+ * assumption that the record type tag is always there. Hence, if the record contains a type tag at the
+ * beginning of the record, we subtract 0 (noop). However, if the record doesn't contain a tag, then we
+ * need to subtract by 1 to get the correct offset of the open part (i.e., as if the tag existed).
+ */
+ int openPartStart =
+ pointer + AInt32SerializerDeserializer.getInt(data, currentPointer) - (isTagged() ? 0 : 1);
+ //Skip open part offset to the beginning of closed part
+ currentPointer += 4;
+ //Number of children in the open part
+ numberOfOpenChildren = AInt32SerializerDeserializer.getInt(data, openPartStart);
+ //Skip the numberOfOpenChildren and the hashOffsetPair to the first open value
+ openValuesOffset = openPartStart + 4 + 8 * numberOfOpenChildren;
+ } else {
+ numberOfOpenChildren = 0;
+ }
+
+ return currentPointer;
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/TypedRecordLazyVisitablePointable.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/TypedRecordLazyVisitablePointable.java
new file mode 100644
index 0000000..19eb076
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/TypedRecordLazyVisitablePointable.java
@@ -0,0 +1,226 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.lazy;
+
+import java.util.Objects;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AStringSerializerDeserializer;
+import org.apache.asterix.om.base.AMutableString;
+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.IAType;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.RecordUtil;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.util.string.UTF8StringWriter;
+
+/**
+ * This implementation extends {@link RecordLazyVisitablePointable} to handle {@link ATypeTag#OBJECT} with open and
+ * closed fields
+ */
+public class TypedRecordLazyVisitablePointable extends RecordLazyVisitablePointable {
+ private static final IPointable MISSING_POINTABLE = createConstantPointable(ATypeTag.MISSING);
+ private static final IPointable NULL_POINTABLE = createConstantPointable(ATypeTag.NULL);
+ private final ARecordType recordType;
+
+ //Closed values
+ private final IValueReference[] closedFieldNames;
+ private final AbstractLazyVisitablePointable[] closedVisitables;
+ private final int numberOfClosedChildren;
+ private final ATypeTag[] closedChildTags;
+ //Record builder computes the fields' offset as if the type tag exists
+ private final int actualChildOffset;
+ private int currentIndex;
+ private int closedValuesOffset;
+
+ /**
+ * A constructor for the root record
+ *
+ * @param rootRecordType root record type
+ */
+ public TypedRecordLazyVisitablePointable(ARecordType rootRecordType) {
+ this(true, rootRecordType);
+ }
+
+ public TypedRecordLazyVisitablePointable(boolean tagged, ARecordType recordType) {
+ super(tagged);
+ Objects.requireNonNull(recordType);
+ this.recordType = recordType;
+ numberOfClosedChildren = this.recordType.getFieldTypes().length;
+ closedFieldNames = createSerializedClosedFieldNames(this.recordType);
+ closedVisitables = createClosedVisitables(this.recordType);
+ closedChildTags = createInitialClosedTypeTags(this.recordType);
+ //-1 if not tagged. The offsets were calculated as if the tag exists.
+ actualChildOffset = isTagged() ? 0 : -1;
+ }
+
+ @Override
+ public void nextChild() throws HyracksDataException {
+ currentIndex++;
+ if (isTaggedChild()) {
+ super.nextChild();
+ } else {
+ setClosedValueInfo();
+ }
+ }
+
+ @Override
+ public boolean isTaggedChild() {
+ return currentIndex >= numberOfClosedChildren;
+ }
+
+ @Override
+ public int getNumberOfChildren() {
+ return numberOfClosedChildren + super.getNumberOfChildren();
+ }
+
+ @Override
+ public AbstractLazyVisitablePointable getChildVisitablePointable() throws HyracksDataException {
+ AbstractLazyVisitablePointable visitablePointable;
+ if (isTaggedChild()) {
+ visitablePointable = openVisitable;
+ } else {
+ visitablePointable = closedVisitables[currentIndex];
+ }
+ visitablePointable.set(getChildValue());
+ return visitablePointable;
+ }
+
+ private void setClosedValueInfo() throws HyracksDataException {
+ ATypeTag typeTag = closedChildTags[currentIndex];
+ if (typeTag == ATypeTag.NULL) {
+ currentValue.set(NULL_POINTABLE);
+ } else if (typeTag == ATypeTag.MISSING) {
+ currentValue.set(MISSING_POINTABLE);
+ } else {
+ byte[] data = getByteArray();
+ int offset =
+ getStartOffset() + AInt32SerializerDeserializer.getInt(data, closedValuesOffset + 4 * currentIndex)
+ + actualChildOffset;
+ int length = NonTaggedFormatUtil.getFieldValueLength(data, offset, typeTag, false);
+ currentValue.set(data, offset, length);
+ }
+ currentFieldName.set(closedFieldNames[currentIndex]);
+ currentChildTypeTag = typeTag.serialize();
+ }
+
+ /* ******************************************************
+ * Init Open part
+ * ******************************************************
+ */
+ @Override
+ void init(byte[] data, int offset, int length) {
+ /*
+ * Skip length and the type tag if the current record contains a tag. Only the root can be tagged and typed
+ * at the same time. Nested typed records will not be tagged.
+ */
+ int skipTag = isTagged() ? 1 : 0;
+ currentIndex = -1;
+ //initOpenPart first. It will skip type tag and length.
+ int pointer = recordType.isOpen() ? initOpenPart(data, offset) : offset + skipTag + 4;
+ initClosedPart(pointer, data);
+ }
+
+ private void initClosedPart(int pointer, byte[] data) {
+ //+4 to skip the number of closed children
+ int currentPointer = pointer + 4;
+ if (NonTaggedFormatUtil.hasOptionalField(recordType)) {
+ initClosedChildrenTags(data, currentPointer);
+ currentPointer =
+ (numberOfClosedChildren % 4 == 0 ? numberOfClosedChildren / 4 : numberOfClosedChildren / 4 + 1);
+ }
+ closedValuesOffset = currentPointer;
+ }
+
+ private static IPointable createConstantPointable(ATypeTag tag) {
+ byte[] data = { tag.serialize() };
+ IPointable value = new VoidPointable();
+ value.set(data, 0, 1);
+ return value;
+ }
+
+ private void initClosedChildrenTags(byte[] data, int nullBitMapOffset) {
+ IAType[] types = recordType.getFieldTypes();
+ for (int i = 0; i < numberOfClosedChildren; i++) {
+ byte nullMissingOrValue = data[nullBitMapOffset + i / 4];
+ if (RecordUtil.isNull(nullMissingOrValue, i)) {
+ closedChildTags[i] = ATypeTag.NULL;
+ } else if (RecordUtil.isMissing(nullMissingOrValue, i)) {
+ closedChildTags[i] = ATypeTag.MISSING;
+ } else {
+ IAType type = types[i];
+ type = type.getTypeTag() == ATypeTag.UNION ? ((AUnionType) type).getActualType() : type;
+ closedChildTags[i] = type.getTypeTag();
+ }
+ }
+ }
+
+ private static ATypeTag[] createInitialClosedTypeTags(ARecordType recordType) {
+ IAType[] types = recordType.getFieldTypes();
+ ATypeTag[] typeTags = new ATypeTag[types.length];
+ for (int i = 0; i < types.length; i++) {
+ IAType type = types[i];
+ if (type.getTypeTag() == ATypeTag.UNION) {
+ type = ((AUnionType) type).getActualType();
+ }
+ typeTags[i] = type.getTypeTag();
+ }
+ return typeTags;
+ }
+
+ private static IValueReference[] createSerializedClosedFieldNames(ARecordType recordType) {
+ UTF8StringWriter writer = new UTF8StringWriter();
+ AMutableString mutableString = new AMutableString("");
+ AStringSerializerDeserializer serDer = new AStringSerializerDeserializer(writer, null);
+
+ String[] fieldNames = recordType.getFieldNames();
+ IValueReference[] fieldNameReferences = new IValueReference[fieldNames.length];
+ for (int i = 0; i < fieldNameReferences.length; i++) {
+ mutableString.setValue(fieldNames[i]);
+ fieldNameReferences[i] = createFieldName(mutableString, serDer);
+ }
+ return fieldNameReferences;
+ }
+
+ private static IValueReference createFieldName(AMutableString mutableString, AStringSerializerDeserializer serDer) {
+ ArrayBackedValueStorage storage = new ArrayBackedValueStorage();
+ try {
+ serDer.serialize(mutableString, storage.getDataOutput());
+ } catch (HyracksDataException e) {
+ throw new IllegalStateException(e);
+ }
+ return storage;
+ }
+
+ private static AbstractLazyVisitablePointable[] createClosedVisitables(ARecordType recordType) {
+ IAType[] types = recordType.getFieldTypes();
+ AbstractLazyVisitablePointable[] visitables = new AbstractLazyVisitablePointable[types.length];
+ for (int i = 0; i < types.length; i++) {
+ visitables[i] = createVisitable(types[i]);
+ }
+ return visitables;
+ }
+
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/VariableListLazyVisitablePointable.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/VariableListLazyVisitablePointable.java
new file mode 100644
index 0000000..9e4ab9f
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/VariableListLazyVisitablePointable.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.asterix.om.lazy;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AbstractCollectionType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * This implementation is for {@link ATypeTag#ARRAY} and {@link ATypeTag#MULTISET} with variable-length items.
+ */
+public class VariableListLazyVisitablePointable extends AbstractListLazyVisitablePointable {
+ private final ATypeTag itemTag;
+ //List builder computes the items' offset as if the type tag exists
+ private final int actualChildOffset;
+
+ public VariableListLazyVisitablePointable(boolean tagged, AbstractCollectionType listType) {
+ super(tagged, listType);
+ itemTag = listType.getItemType().getTypeTag();
+ //-1 if not tagged. The offsets were calculated as if the tag exists.
+ actualChildOffset = isTagged() ? 0 : -1;
+ }
+
+ @Override
+ public void nextChild() throws HyracksDataException {
+ byte[] data = getByteArray();
+ int itemOffset = getStartOffset() + AInt32SerializerDeserializer.getInt(data, itemsOffset + currentIndex * 4)
+ + actualChildOffset;
+ ATypeTag itemTypeTag = processTypeTag(data, itemOffset);
+ int itemSize = NonTaggedFormatUtil.getFieldValueLength(data, itemOffset, itemTypeTag, isTaggedChild());
+ currentValue.set(data, itemOffset, itemSize);
+ currentIndex++;
+ }
+
+ private ATypeTag processTypeTag(byte[] data, int itemOffset) {
+ if (itemTag == ATypeTag.ANY) {
+ currentChildTypeTag = data[itemOffset];
+ }
+ return itemTag;
+ }
+
+ @Override
+ public boolean isTaggedChild() {
+ return itemTag == ATypeTag.ANY;
+ }
+
+ @Override
+ AbstractLazyVisitablePointable createVisitablePointable(IAType itemType) {
+ if (itemType.getTypeTag() != ATypeTag.ANY) {
+ return createVisitable(itemType);
+ }
+ return new GenericLazyVisitablePointable();
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AListTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AListTypeComputer.java
index e6e491c..21e88e2 100755
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AListTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AListTypeComputer.java
@@ -41,6 +41,8 @@
public static final AListTypeComputer INSTANCE_INSERT = new AListTypeComputer(3, false, true, false);
public static final AListTypeComputer INSTANCE_REPLACE = new AListTypeComputer(3, false, true, false);
public static final AListTypeComputer INSTANCE_SLICE = new AListTypeComputer(-1, false, false, true);
+ public static final AListTypeComputer INSTANCE_MOVE = new AListTypeComputer(3, false, false, true);
+ public static final AListTypeComputer INSTANCE_SWAP = new AListTypeComputer(3, false, false, true);
private final int minNumArgs;
private final boolean listIsLast;
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AbstractRecordFunctionTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AbstractRecordFunctionTypeComputer.java
new file mode 100644
index 0000000..580cab4
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AbstractRecordFunctionTypeComputer.java
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.pointables.base.DefaultOpenFieldType;
+import org.apache.asterix.om.typecomputer.base.IResultTypeComputer;
+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.TypeHelper;
+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.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+
+/**
+ * Base type computer for the following record-functions:
+ * 1. {@link org.apache.asterix.om.functions.BuiltinFunctions#RECORD_ADD}
+ * 2. {@link org.apache.asterix.om.functions.BuiltinFunctions#RECORD_PUT}
+ * 3. {@link org.apache.asterix.om.functions.BuiltinFunctions#RECORD_REMOVE}
+ * 4. {@link org.apache.asterix.om.functions.BuiltinFunctions#RECORD_RENAME}
+ */
+public abstract class AbstractRecordFunctionTypeComputer implements IResultTypeComputer {
+ @Override
+ public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
+ IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
+ AbstractFunctionCallExpression functionCallExpression = (AbstractFunctionCallExpression) expression;
+
+ // Get our record type.
+ ILogicalExpression arg0 = functionCallExpression.getArguments().get(0).getValue();
+ IAType type0 = (IAType) env.getType(arg0);
+ IAType actualType0 = TypeComputeUtils.getActualType(type0);
+ ATypeTag tag0 = actualType0.getTypeTag();
+ if (tag0 == ATypeTag.ANY) {
+ return wrapTypeWithUnknown(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE, true, true);
+ } else if (tag0 == ATypeTag.MISSING) {
+ // Our output is always going to be MISSING.
+ return BuiltinType.AMISSING;
+ } else if (tag0 != ATypeTag.OBJECT) {
+ // Our output is always going to be NULL.
+ return BuiltinType.ANULL;
+ }
+ boolean isOutputMissable = TypeHelper.canBeMissing(type0);
+ boolean isOutputNullable = TypeHelper.canBeNull(type0);
+ ARecordType inputRecordType = TypeComputeUtils.extractRecordType(actualType0);
+
+ // Our second argument should be of type "string".
+ ILogicalExpression arg1 = functionCallExpression.getArguments().get(1).getValue();
+ IAType type1 = (IAType) env.getType(arg1);
+ IAType actualType1 = TypeComputeUtils.getActualType(type1);
+ ATypeTag tag1 = actualType1.getTypeTag();
+ if (tag1 == ATypeTag.ANY) {
+ // We cannot infer the type of our second argument-- our output may be MISSING or NULL.
+ return wrapTypeWithUnknown(type0, true, true);
+ } else if (tag1 == ATypeTag.MISSING) {
+ // Our output is always going to be MISSING.
+ return BuiltinType.AMISSING;
+ } else if (tag1 != ATypeTag.STRING) {
+ // Our output is always going to be NULL.
+ return BuiltinType.ANULL;
+ }
+ isOutputMissable |= TypeHelper.canBeMissing(type1);
+ isOutputNullable |= TypeHelper.canBeNull(type1);
+
+ // Compute our type.
+ return computeTypeImpl(functionCallExpression, env, inputRecordType, isOutputMissable, isOutputNullable);
+ }
+
+ protected abstract IAType computeTypeImpl(AbstractFunctionCallExpression functionCallExpression,
+ IVariableTypeEnvironment env, ARecordType inputRecordType, boolean isOutputMissable,
+ boolean isOutputNullable) throws AlgebricksException;
+
+ protected static IAType wrapTypeWithUnknown(IAType originalType, boolean isMissable, boolean isNullable) {
+ if (isNullable && isMissable) {
+ return AUnionType.createUnknownableType(originalType);
+ } else if (isNullable) { // && !isMissable
+ return AUnionType.createNullableType(originalType);
+ } else if (isMissable) { // && !isNullable
+ return AUnionType.createMissableType(originalType);
+ } else {
+ return originalType;
+ }
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordAddTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordAddTypeComputer.java
new file mode 100644
index 0000000..4815340
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordAddTypeComputer.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.om.typecomputer.impl;
+
+import java.util.Arrays;
+
+import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.constants.AsterixConstantValue;
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+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.base.LogicalExpressionTag;
+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;
+
+public class RecordAddTypeComputer extends AbstractRecordFunctionTypeComputer {
+ public static final RecordAddTypeComputer INSTANCE = new RecordAddTypeComputer();
+
+ private RecordAddTypeComputer() {
+ }
+
+ @Override
+ public IAType computeTypeImpl(AbstractFunctionCallExpression functionCallExpression, IVariableTypeEnvironment env,
+ ARecordType inputRecordType, boolean isOutputMissable, boolean isOutputNullable)
+ throws AlgebricksException {
+ // If our third argument is missing, then just return the type of our first argument.
+ ILogicalExpression arg2 = functionCallExpression.getArguments().get(2).getValue();
+ IAType type2 = (IAType) env.getType(arg2);
+ IAType actualType2 = TypeComputeUtils.getActualType(type2);
+ ATypeTag tag2 = actualType2.getTypeTag();
+ if (tag2 == ATypeTag.MISSING) {
+ IAType type0 = (IAType) env.getType(functionCallExpression.getArguments().get(0).getValue());
+ return wrapTypeWithUnknown(type0, isOutputMissable, isOutputNullable);
+ }
+
+ // We expect a constant for our second argument.
+ ILogicalExpression arg1 = functionCallExpression.getArguments().get(1).getValue();
+ if (arg1.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
+ return wrapTypeWithUnknown(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE, isOutputMissable,
+ isOutputNullable);
+ }
+ ConstantExpression constantExpression = (ConstantExpression) arg1;
+ AsterixConstantValue constantValue = (AsterixConstantValue) constantExpression.getValue();
+ String newFieldName = ((AString) constantValue.getObject()).getStringValue();
+
+ // If our field is found, return the original record type. Otherwise, add our new field.
+ ARecordType outputRecordType = inputRecordType;
+ if (!Arrays.asList(inputRecordType.getFieldNames()).contains(newFieldName)) {
+ String[] fieldNames = new String[inputRecordType.getFieldNames().length + 1];
+ IAType[] fieldTypes = new IAType[inputRecordType.getFieldTypes().length + 1];
+ int currentCursorPosition = 0;
+ for (; currentCursorPosition < inputRecordType.getFieldNames().length; currentCursorPosition++) {
+ fieldNames[currentCursorPosition] = inputRecordType.getFieldNames()[currentCursorPosition];
+ fieldTypes[currentCursorPosition] = inputRecordType.getFieldTypes()[currentCursorPosition];
+ }
+ fieldNames[currentCursorPosition] = newFieldName;
+ fieldTypes[currentCursorPosition] = type2;
+ String inputTypeName = inputRecordType.getTypeName();
+ String outputTypeName = inputTypeName != null ? inputTypeName + "_add_" + newFieldName : null;
+ outputRecordType = new ARecordType(outputTypeName, fieldNames, fieldTypes, inputRecordType.isOpen());
+ }
+ return wrapTypeWithUnknown(outputRecordType, isOutputMissable, isOutputNullable);
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordPutTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordPutTypeComputer.java
new file mode 100644
index 0000000..a92907d
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordPutTypeComputer.java
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 java.util.ArrayList;
+import java.util.List;
+
+import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.constants.AsterixConstantValue;
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+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.base.LogicalExpressionTag;
+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;
+
+public class RecordPutTypeComputer extends AbstractRecordFunctionTypeComputer {
+ public static final RecordPutTypeComputer INSTANCE = new RecordPutTypeComputer();
+
+ private RecordPutTypeComputer() {
+ }
+
+ @Override
+ public IAType computeTypeImpl(AbstractFunctionCallExpression functionCallExpression, IVariableTypeEnvironment env,
+ ARecordType inputRecordType, boolean isOutputMissable, boolean isOutputNullable)
+ throws AlgebricksException {
+ // Extract the type of our third argument. If it is MISSING, then we are performing a field removal.
+ ILogicalExpression arg2 = functionCallExpression.getArguments().get(2).getValue();
+ IAType type2 = (IAType) env.getType(arg2);
+ IAType actualType2 = TypeComputeUtils.getActualType(type2);
+ boolean isFieldRemoval = actualType2.getTypeTag() == ATypeTag.MISSING;
+
+ // We expect a constant for our second argument.
+ ILogicalExpression arg1 = functionCallExpression.getArguments().get(1).getValue();
+ if (arg1.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
+ return wrapTypeWithUnknown(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE, isOutputMissable,
+ isOutputNullable);
+ }
+ ConstantExpression constantExpression = (ConstantExpression) arg1;
+ AsterixConstantValue constantValue = (AsterixConstantValue) constantExpression.getValue();
+ String newFieldName = ((AString) constantValue.getObject()).getStringValue();
+
+ // Remove or replace our field name and type (dependent on the type of our third argument).
+ boolean fieldFound = false;
+ List<String> outputFieldNames = new ArrayList<>();
+ List<IAType> outputFieldTypes = new ArrayList<>();
+ for (int i = 0; i < inputRecordType.getFieldNames().length; i++) {
+ String inputFieldName = inputRecordType.getFieldNames()[i];
+ IAType inputFieldType = inputRecordType.getFieldTypes()[i];
+ if (!inputFieldName.equals(newFieldName)) {
+ outputFieldNames.add(inputFieldName);
+ outputFieldTypes.add(inputFieldType);
+
+ } else {
+ fieldFound = true;
+ if (!isFieldRemoval) {
+ // Replace our input field type.
+ outputFieldNames.add(inputFieldName);
+ outputFieldTypes.add(type2);
+ }
+ }
+ }
+
+ // Build our output record type.
+ ARecordType outputRecordType;
+ String inputTypeName = inputRecordType.getTypeName();
+ boolean doesRecordHaveTypeName = inputTypeName != null;
+ if (fieldFound && isFieldRemoval) {
+ // We have removed our argument field.
+ String outputTypeName = doesRecordHaveTypeName ? inputTypeName + "_remove_" + newFieldName : null;
+ outputRecordType = new ARecordType(outputTypeName, outputFieldNames.toArray(String[]::new),
+ outputFieldTypes.toArray(IAType[]::new), inputRecordType.isOpen());
+ } else if (fieldFound) { // && !isFieldRemoval
+ // We have replaced our argument field.
+ String outputTypeName = doesRecordHaveTypeName ? inputTypeName + "_replaced_" + newFieldName : null;
+ outputRecordType = new ARecordType(outputTypeName, outputFieldNames.toArray(String[]::new),
+ outputFieldTypes.toArray(IAType[]::new), inputRecordType.isOpen());
+ } else if (!isFieldRemoval) { // && !wasFieldFound
+ // We need to insert our argument field.
+ outputFieldNames.add(newFieldName);
+ outputFieldTypes.add(type2);
+ String outputTypeName = doesRecordHaveTypeName ? inputTypeName + "_add_" + newFieldName : null;
+ outputRecordType = new ARecordType(outputTypeName, outputFieldNames.toArray(String[]::new),
+ outputFieldTypes.toArray(IAType[]::new), inputRecordType.isOpen());
+ } else { // isFieldRemoval && !wasFieldFound
+ // We have not found the field to remove.
+ outputRecordType = inputRecordType;
+ }
+ return wrapTypeWithUnknown(outputRecordType, isOutputMissable, isOutputNullable);
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordRemoveFieldsTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordRemoveFieldsTypeComputer.java
index 83af00d..bdb7277 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordRemoveFieldsTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordRemoveFieldsTypeComputer.java
@@ -42,6 +42,7 @@
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.TypeHelper;
import org.apache.commons.lang3.mutable.Mutable;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
@@ -84,6 +85,7 @@
case STRING:
String fn = ((AString) fieldName).getStringValue();
fieldNameSet.add(fn);
+ pathList.add(List.of(fn));
break;
case ARRAY:
AOrderedList pathOrdereList = (AOrderedList) fieldName;
@@ -237,16 +239,18 @@
IAType[] fieldTypes = inputRecordType.getFieldTypes();
for (int i = 0; i < fieldNames.length; i++) {
+ IAType originalType = fieldTypes[i];
+ IAType actualType = TypeComputeUtils.getActualType(originalType);
if (!fieldNameSet.contains(fieldNames[i])) { // The main field is to be kept
addField(inputRecordType, fieldNames[i], resultFieldNames, resultFieldTypes);
- } else if (!pathList.isEmpty() && fieldTypes[i].getTypeTag() == ATypeTag.OBJECT) {
- ARecordType subRecord = (ARecordType) fieldTypes[i];
+ } else if (!pathList.isEmpty() && actualType.getTypeTag() == ATypeTag.OBJECT) {
+ ARecordType subRecord = (ARecordType) actualType;
fieldPathStack.push(fieldNames[i]);
- subRecord = deepCheckAndCopy(fieldPathStack, subRecord, pathList, inputRecordType.isOpen());
+ subRecord = deepCheckAndCopy(fieldPathStack, subRecord, pathList, subRecord.isOpen());
fieldPathStack.pop();
if (subRecord != null) {
resultFieldNames.add(fieldNames[i]);
- resultFieldTypes.add(subRecord);
+ resultFieldTypes.add(wrapWithOriginalType(subRecord, originalType));
}
}
}
@@ -259,6 +263,18 @@
}
+ private static IAType wrapWithOriginalType(IAType typeToModify, IAType originalType) {
+ if (TypeHelper.canBeMissing(originalType) && !TypeHelper.canBeNull(originalType)) {
+ return AUnionType.createMissableType(typeToModify);
+ } else if (!TypeHelper.canBeMissing(originalType) && TypeHelper.canBeNull(originalType)) {
+ return AUnionType.createNullableType(typeToModify);
+ } else if (TypeHelper.canBeUnknown(originalType)) {
+ return AUnionType.createUnknownableType(typeToModify);
+ } else {
+ return typeToModify;
+ }
+ }
+
/**
* Comparison elements of two paths
* Note: l2 uses a LIFO insert and removal.
@@ -314,12 +330,14 @@
for (int i = 0; i < srcFieldNames.length; i++) {
fieldPath.push(srcFieldNames[i]);
if (!isRemovePath(fieldPath, pathList)) {
- if (srcFieldTypes[i].getTypeTag() == ATypeTag.OBJECT) {
- ARecordType subRecord = (ARecordType) srcFieldTypes[i];
+ IAType originalType = srcFieldTypes[i];
+ IAType actualType = TypeComputeUtils.getActualType(originalType);
+ if (actualType.getTypeTag() == ATypeTag.OBJECT) {
+ ARecordType subRecord = (ARecordType) actualType;
subRecord = deepCheckAndCopy(fieldPath, subRecord, pathList, isOpen);
if (subRecord != null) {
destFieldNames.add(srcFieldNames[i]);
- destFieldTypes.add(subRecord);
+ destFieldTypes.add(wrapWithOriginalType(subRecord, originalType));
}
} else {
destFieldNames.add(srcFieldNames[i]);
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordRemoveTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordRemoveTypeComputer.java
new file mode 100644
index 0000000..c447e65
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordRemoveTypeComputer.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 java.util.Arrays;
+
+import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.constants.AsterixConstantValue;
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.IAType;
+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.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+
+public class RecordRemoveTypeComputer extends AbstractRecordFunctionTypeComputer {
+ public static final RecordRemoveTypeComputer INSTANCE = new RecordRemoveTypeComputer();
+
+ private RecordRemoveTypeComputer() {
+ }
+
+ @Override
+ public IAType computeTypeImpl(AbstractFunctionCallExpression functionCallExpression, IVariableTypeEnvironment env,
+ ARecordType inputRecordType, boolean isOutputMissable, boolean isOutputNullable) {
+ // We expect a CONSTANT expression. Otherwise, defer the removal to runtime.
+ ILogicalExpression arg1 = functionCallExpression.getArguments().get(1).getValue();
+ if (arg1.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
+ return wrapTypeWithUnknown(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE, isOutputMissable,
+ isOutputNullable);
+ }
+ ConstantExpression constantExpression = (ConstantExpression) arg1;
+ AsterixConstantValue constantValue = (AsterixConstantValue) constantExpression.getValue();
+ String fieldName = ((AString) constantValue.getObject()).getStringValue();
+
+ // If our field is found, remove it. Otherwise, return the original record type.
+ ARecordType outputRecordType = inputRecordType;
+ if (Arrays.asList(inputRecordType.getFieldNames()).contains(fieldName)) {
+ String[] fieldNames = new String[inputRecordType.getFieldNames().length - 1];
+ IAType[] fieldTypes = new IAType[inputRecordType.getFieldTypes().length - 1];
+ int currentOutputCursor = 0;
+ for (int i = 0; i < inputRecordType.getFieldNames().length; i++) {
+ String inputName = inputRecordType.getFieldNames()[i];
+ IAType inputType = inputRecordType.getFieldTypes()[i];
+ if (!inputName.equals(fieldName)) {
+ fieldNames[currentOutputCursor] = inputName;
+ fieldTypes[currentOutputCursor] = inputType;
+ currentOutputCursor++;
+ }
+ }
+ String inputTypeName = inputRecordType.getTypeName();
+ String outputTypeName = inputTypeName != null ? inputTypeName + "_remove_" + fieldName : null;
+ outputRecordType = new ARecordType(outputTypeName, fieldNames, fieldTypes, inputRecordType.isOpen());
+ }
+ return wrapTypeWithUnknown(outputRecordType, isOutputMissable, isOutputNullable);
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordRenameTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordRenameTypeComputer.java
new file mode 100644
index 0000000..4302082
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordRenameTypeComputer.java
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 java.util.Arrays;
+
+import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.constants.AsterixConstantValue;
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+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.TypeHelper;
+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.LogicalExpressionTag;
+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;
+
+public class RecordRenameTypeComputer extends AbstractRecordFunctionTypeComputer {
+ public static final RecordRenameTypeComputer INSTANCE = new RecordRenameTypeComputer();
+
+ private RecordRenameTypeComputer() {
+ }
+
+ @Override
+ public IAType computeTypeImpl(AbstractFunctionCallExpression functionCallExpression, IVariableTypeEnvironment env,
+ ARecordType inputRecordType, boolean isOutputMissable, boolean isOutputNullable)
+ throws AlgebricksException {
+ // Our third argument should be of type "string".
+ ILogicalExpression arg2 = functionCallExpression.getArguments().get(2).getValue();
+ IAType type2 = (IAType) env.getType(arg2);
+ IAType actualType2 = TypeComputeUtils.getActualType(type2);
+ ATypeTag tag2 = actualType2.getTypeTag();
+ if (tag2 == ATypeTag.ANY) {
+ // We cannot infer the type of our third argument-- our output may be MISSING or NULL.
+ return AUnionType.createUnknownableType(inputRecordType, inputRecordType.getTypeName() + "?");
+ } else if (tag2 == ATypeTag.MISSING) {
+ // Our output is always going to be MISSING.
+ return BuiltinType.AMISSING;
+ } else if (tag2 != ATypeTag.STRING) {
+ // Our output is always going to be NULL.
+ return BuiltinType.ANULL;
+ }
+ isOutputMissable |= TypeHelper.canBeMissing(type2);
+ isOutputNullable |= TypeHelper.canBeNull(type2);
+
+ // We expect a CONSTANT expression for both arguments. Otherwise, defer the replacement to runtime.
+ ILogicalExpression arg1 = functionCallExpression.getArguments().get(1).getValue();
+ if (arg1.getExpressionTag() != LogicalExpressionTag.CONSTANT
+ || arg2.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
+ return wrapTypeWithUnknown(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE, isOutputMissable,
+ isOutputNullable);
+ }
+ ConstantExpression arg1ConstantExpression = (ConstantExpression) arg1;
+ ConstantExpression arg2ConstantExpression = (ConstantExpression) arg2;
+ AsterixConstantValue arg1ConstantValue = (AsterixConstantValue) arg1ConstantExpression.getValue();
+ AsterixConstantValue arg2ConstantValue = (AsterixConstantValue) arg2ConstantExpression.getValue();
+ String oldFieldName = ((AString) arg1ConstantValue.getObject()).getStringValue();
+ String newFieldName = ((AString) arg2ConstantValue.getObject()).getStringValue();
+
+ // If our field is found, replace it. Otherwise, return the original record type.
+ Mutable<Boolean> fieldFound = new MutableObject<>(false);
+ String[] newFieldNames = Arrays.stream(inputRecordType.getFieldNames()).map(f -> {
+ if (f.equals(oldFieldName)) {
+ fieldFound.setValue(true);
+ return newFieldName;
+ } else {
+ return f;
+ }
+ }).toArray(String[]::new);
+ ARecordType outputRecordType;
+ if (!fieldFound.getValue()) {
+ outputRecordType = inputRecordType;
+ } else {
+ String inputTypeName = inputRecordType.getTypeName();
+ String outputTypeName = inputTypeName != null ? inputTypeName + "_replaced_" + oldFieldName : null;
+ outputRecordType = new ARecordType(outputTypeName, newFieldNames, inputRecordType.getFieldTypes(),
+ inputRecordType.isOpen());
+ }
+ return wrapTypeWithUnknown(outputRecordType, isOutputMissable, isOutputNullable);
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java
index e256e1b..fd51433 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java
@@ -25,7 +25,6 @@
import java.util.HashSet;
import java.util.List;
import java.util.Map;
-import java.util.Objects;
import java.util.Set;
import org.apache.asterix.common.annotations.IRecordTypeAnnotation;
@@ -73,30 +72,21 @@
private final Set<String> allPossibleAdditionalFieldNames;
/**
- * @param typeName
- * the name of the type
- * @param fieldNames
- * the names of the closed fields
- * @param fieldTypes
- * the types of the closed fields
- * @param isOpen
- * whether the record is open
+ * @param typeName the name of the type
+ * @param fieldNames the names of the closed fields
+ * @param fieldTypes the types of the closed fields
+ * @param isOpen whether the record is open
*/
public ARecordType(String typeName, String[] fieldNames, IAType[] fieldTypes, boolean isOpen) {
this(typeName, fieldNames, fieldTypes, isOpen, null);
}
/**
- * @param typeName
- * the name of the type
- * @param fieldNames
- * the names of the closed fields
- * @param fieldTypes
- * the types of the closed fields
- * @param isOpen
- * whether the record is open
- * @param allPossibleAdditionalFieldNames,
- * all possible additional field names.
+ * @param typeName the name of the type
+ * @param fieldNames the names of the closed fields
+ * @param fieldTypes the types of the closed fields
+ * @param isOpen whether the record is open
+ * @param allPossibleAdditionalFieldNames, all possible additional field names.
*/
public ARecordType(String typeName, String[] fieldNames, IAType[] fieldTypes, boolean isOpen,
Set<String> allPossibleAdditionalFieldNames) {
@@ -189,8 +179,7 @@
/**
* Returns the position of the field in the closed schema or -1 if the field does not exist.
*
- * @param fieldName
- * the name of the field whose position is sought
+ * @param fieldName the name of the field whose position is sought
* @return the position of the field in the closed schema or -1 if the field does not exist.
*/
public int getFieldIndex(String fieldName) {
@@ -206,10 +195,8 @@
}
/**
- * @param subFieldName
- * The full pathname of the child
- * @param parent
- * The type of the parent
+ * @param subFieldName The full pathname of the child
+ * @param parent The type of the parent
* @return the type of the child
*/
@@ -222,8 +209,7 @@
}
/**
- * @param subFieldName
- * The full pathname of the child
+ * @param subFieldName The full pathname of the child
* @return the type of the child
* @throws AsterixException
*/
@@ -248,9 +234,7 @@
}
/**
- *
- * @param subFieldName
- * The full pathname of the field
+ * @param subFieldName The full pathname of the field
* @return The nullability of the field
* @throws AlgebricksException
*/
@@ -283,10 +267,9 @@
/**
* Returns the field type of the field name if it exists, otherwise null.
*
- * @param fieldName
- * the fieldName whose type is sought
+ * @param fieldName the fieldName whose type is sought
* @return the field type of the field name if it exists, otherwise null
- * NOTE: this method doesn't work for nested fields
+ * NOTE: this method doesn't work for nested fields
*/
public IAType getFieldType(String fieldName) {
int fieldPos = getFieldIndex(fieldName);
@@ -299,8 +282,7 @@
/**
* Returns true or false indicating whether or not a field is closed.
*
- * @param fieldName
- * the name of the field to check
+ * @param fieldName the name of the field to check
* @return true if fieldName is a closed field, otherwise false
*/
public boolean isClosedField(String fieldName) {
@@ -360,8 +342,7 @@
}
ARecordType rt = (ARecordType) obj;
return (isOpen == rt.isOpen) && Arrays.deepEquals(fieldNames, rt.fieldNames)
- && Arrays.deepEquals(fieldTypes, rt.fieldTypes)
- && Objects.equals(allPossibleAdditionalFieldNames, rt.allPossibleAdditionalFieldNames);
+ && Arrays.deepEquals(fieldTypes, rt.fieldTypes);
}
@Override
@@ -382,11 +363,7 @@
ObjectNode type = om.createObjectNode();
type.put("type", ARecordType.class.getName());
type.put("name", typeName);
- if (isOpen) {
- type.put("open", true);
- } else {
- type.put("open", false);
- }
+ type.put("open", isOpen);
ArrayNode fields = om.createArrayNode();
for (int i = 0; i < fieldNames.length; i++) {
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/visitor/PathStringBuilderForIATypeVisitor.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/visitor/PathStringBuilderForIATypeVisitor.java
new file mode 100644
index 0000000..ddc2437
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/visitor/PathStringBuilderForIATypeVisitor.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.om.types.visitor;
+
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.AbstractCollectionType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.types.IATypeVisitor;
+import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
+
+/**
+ * Produces a path from a type
+ */
+public class PathStringBuilderForIATypeVisitor implements IATypeVisitor<Void, StringBuilder> {
+ private boolean root = true;
+
+ @Override
+ public Void visit(ARecordType recordType, StringBuilder arg) {
+ // here we assume the record type has only one child
+ if (root) {
+ arg.append("$$root");
+ root = false;
+ }
+ arg.append('.');
+ arg.append(recordType.getFieldNames()[0]);
+ recordType.getFieldTypes()[0].accept(this, arg);
+ return null;
+ }
+
+ @Override
+ public Void visit(AbstractCollectionType collectionType, StringBuilder arg) {
+ arg.append("[*]");
+ collectionType.getItemType().accept(this, arg);
+ return null;
+ }
+
+ @Override
+ public Void visit(AUnionType unionType, StringBuilder arg) {
+ throw new NotImplementedException("Check how to represent this");
+ }
+
+ @Override
+ public Void visitFlat(IAType flatType, StringBuilder arg) {
+ return null;
+ }
+}
diff --git a/asterixdb/asterix-docker/docker/.gitattributes b/asterixdb/asterix-podman/docker/.gitattributes
similarity index 100%
rename from asterixdb/asterix-docker/docker/.gitattributes
rename to asterixdb/asterix-podman/docker/.gitattributes
diff --git a/asterixdb/asterix-docker/docker/Dockerfile b/asterixdb/asterix-podman/docker/Dockerfile
similarity index 100%
rename from asterixdb/asterix-docker/docker/Dockerfile
rename to asterixdb/asterix-podman/docker/Dockerfile
diff --git a/asterixdb/asterix-docker/docker/asterix-configuration.xml b/asterixdb/asterix-podman/docker/asterix-configuration.xml
similarity index 100%
rename from asterixdb/asterix-docker/docker/asterix-configuration.xml
rename to asterixdb/asterix-podman/docker/asterix-configuration.xml
diff --git a/asterixdb/asterix-docker/docker/fbm.adm b/asterixdb/asterix-podman/docker/fbm.adm
similarity index 100%
rename from asterixdb/asterix-docker/docker/fbm.adm
rename to asterixdb/asterix-podman/docker/fbm.adm
diff --git a/asterixdb/asterix-docker/docker/fbu.adm b/asterixdb/asterix-podman/docker/fbu.adm
similarity index 100%
rename from asterixdb/asterix-docker/docker/fbu.adm
rename to asterixdb/asterix-podman/docker/fbu.adm
diff --git a/asterixdb/asterix-docker/docker/supervisord.conf b/asterixdb/asterix-podman/docker/supervisord.conf
similarity index 100%
rename from asterixdb/asterix-docker/docker/supervisord.conf
rename to asterixdb/asterix-podman/docker/supervisord.conf
diff --git a/asterixdb/asterix-docker/docker/twm.adm b/asterixdb/asterix-podman/docker/twm.adm
similarity index 100%
rename from asterixdb/asterix-docker/docker/twm.adm
rename to asterixdb/asterix-podman/docker/twm.adm
diff --git a/asterixdb/asterix-docker/docker/twu.adm b/asterixdb/asterix-podman/docker/twu.adm
similarity index 100%
rename from asterixdb/asterix-docker/docker/twu.adm
rename to asterixdb/asterix-podman/docker/twu.adm
diff --git a/asterixdb/asterix-podman/pom.xml b/asterixdb/asterix-podman/pom.xml
new file mode 100644
index 0000000..23928aa
--- /dev/null
+++ b/asterixdb/asterix-podman/pom.xml
@@ -0,0 +1,156 @@
+<!--
+ ! Licensed to the Apache Software Foundation (ASF) under one
+ ! or more contributor license agreements. See the NOTICE file
+ ! distributed with this work for additional information
+ ! regarding copyright ownership. The ASF licenses this file
+ ! to you under the Apache License, Version 2.0 (the
+ ! "License"); you may not use this file except in compliance
+ ! with the License. You may obtain a copy of the License at
+ !
+ ! http://www.apache.org/licenses/LICENSE-2.0
+ !
+ ! Unless required by applicable law or agreed to in writing,
+ ! software distributed under the License is distributed on an
+ ! "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ ! KIND, either express or implied. See the License for the
+ ! specific language governing permissions and limitations
+ ! under the License.
+ !-->
+
+<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/xsd/maven-4.0.0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+ <parent>
+ <artifactId>apache-asterixdb</artifactId>
+ <groupId>org.apache.asterix</groupId>
+ <version>0.9.9-SNAPSHOT</version>
+ </parent>
+ <artifactId>asterix-podman</artifactId>
+ <dependencies>
+ <dependency>
+ <groupId>${project.groupId}</groupId>
+ <artifactId>asterix-server</artifactId>
+ <version>${project.version}</version>
+ <type>deb</type>
+ <scope>provided</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.asterix</groupId>
+ <artifactId>asterix-app</artifactId>
+ <version>${project.version}</version>
+ <classifier>tests</classifier>
+ <type>test-jar</type>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>junit</groupId>
+ <artifactId>junit</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.asterix</groupId>
+ <artifactId>asterix-test-framework</artifactId>
+ <version>${project.version}</version>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.testcontainers</groupId>
+ <artifactId>testcontainers</artifactId>
+ <version>1.17.1</version>
+ <scope>test</scope>
+ </dependency>
+ </dependencies>
+
+ <properties>
+ <root.dir>${basedir}/..</root.dir>
+ </properties>
+
+ <licenses>
+ <license>
+ <name>Apache License, Version 2.0</name>
+ <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
+ <distribution>repo</distribution>
+ <comments>A business-friendly OSS license</comments>
+ </license>
+ </licenses>
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.rat</groupId>
+ <artifactId>apache-rat-plugin</artifactId>
+ <configuration>
+ <excludes combine.children="append">
+ <exclude>src/test/resources/setup.sh</exclude>
+ <exclude>src/test/resources/passwd</exclude>
+ <exclude>src/test/resources/socktest/Containerfile</exclude>
+ <exclude>src/test/resources/testenv.conf</exclude>
+ </excludes>
+ </configuration>
+ </plugin>
+ </plugins>
+ </build>
+ <profiles>
+ <profile>
+ <id>podman.tests</id>
+ <properties>
+ <test.excludes>**/*.java</test.excludes>
+ <itest.includes>**/PodmanPythonFunctionIT.java</itest.includes>
+ <failIfNoTests>false</failIfNoTests>
+ </properties>
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>nl.lexemmens</groupId>
+ <artifactId>podman-maven-plugin</artifactId>
+ <version>1.8.0</version>
+ <executions>
+ <execution>
+ <goals>
+ <goal>build</goal>
+ </goals>
+ <phase>generate-test-resources</phase>
+ </execution>
+ </executions>
+ <configuration>
+ <skipAuth>true</skipAuth>
+ <images>
+ <image>
+ <name>asterixdb/socktest</name>
+ <build>
+ <pull>false</pull>
+ <createLatestTag>true</createLatestTag>
+ <containerFileDir>src/test/resources/socktest</containerFileDir>
+ </build>
+ </image>
+ </images>
+ </configuration>
+ </plugin>
+ <plugin>
+ <artifactId>maven-resources-plugin</artifactId>
+ <executions>
+ <execution>
+ <id>copy-external-data-resources</id>
+ <phase>generate-resources</phase>
+ <goals>
+ <goal>copy-resources</goal>
+ </goals>
+ <configuration>
+ <outputDirectory>target/</outputDirectory>
+ <overwrite>true</overwrite>
+ <resources>
+ <resource>
+ <directory>../asterix-server/target</directory>
+ <includes>
+ <include>asterix-server*.deb</include>
+ </includes>
+ </resource>
+ </resources>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ </plugins>
+ </build>
+ </profile>
+ </profiles>
+
+</project>
diff --git a/asterixdb/asterix-podman/src/test/java/org/apache/asterix/test/podman/PodmanPythonFunctionIT.java b/asterixdb/asterix-podman/src/test/java/org/apache/asterix/test/podman/PodmanPythonFunctionIT.java
new file mode 100644
index 0000000..f0f89cd
--- /dev/null
+++ b/asterixdb/asterix-podman/src/test/java/org/apache/asterix/test/podman/PodmanPythonFunctionIT.java
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.podman;
+
+import java.net.InetSocketAddress;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.asterix.test.common.TestExecutor;
+import org.apache.asterix.test.runtime.ExecutionTestUtil;
+import org.apache.asterix.test.runtime.LangExecutionUtil;
+import org.apache.asterix.testframework.context.TestCaseContext;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+import org.testcontainers.DockerClientFactory;
+import org.testcontainers.containers.BindMode;
+import org.testcontainers.containers.GenericContainer;
+import org.testcontainers.utility.DockerImageName;
+
+import com.github.dockerjava.api.DockerClient;
+
+/**
+ * Runs the Python UDF tests within a container using domain sockets.
+ */
+@RunWith(Parameterized.class)
+public class PodmanPythonFunctionIT {
+ public static final DockerImageName ASTERIX_IMAGE = DockerImageName.parse("asterixdb/socktest");
+ @ClassRule
+ public static GenericContainer<?> asterix = new GenericContainer(ASTERIX_IMAGE).withExposedPorts(19004, 5006, 19002)
+ .withFileSystemBind("../asterix-app/", "/var/tmp/asterix-app/", BindMode.READ_WRITE);
+ protected static final String TEST_CONFIG_FILE_NAME = "../asterix-app/src/test/resources/cc.conf";
+ private static final boolean cleanupOnStop = true;
+
+ @BeforeClass
+ public static void setUp() throws Exception {
+ final TestExecutor testExecutor = new TestExecutor(
+ List.of(InetSocketAddress.createUnresolved(asterix.getHost(), asterix.getMappedPort(19002))));
+ asterix.execInContainer("/opt/setup.sh");
+ LangExecutionUtil.setUp(TEST_CONFIG_FILE_NAME, testExecutor, false, true, new PodmanUDFLibrarian(asterix));
+ setEndpoints(testExecutor);
+ testExecutor.waitForClusterActive(60, TimeUnit.SECONDS);
+ }
+
+ @AfterClass
+ public static void tearDown() throws Exception {
+ try {
+ } finally {
+ ExecutionTestUtil.tearDown(cleanupOnStop);
+ DockerClient dc = DockerClientFactory.instance().client();
+ dc.removeImageCmd(ASTERIX_IMAGE.asCanonicalNameString()).withForce(true).exec();
+ }
+ }
+
+ @Parameters(name = "PodmanPythonFunctionIT {index}: {0}")
+ public static Collection<Object[]> tests() throws Exception {
+ return LangExecutionUtil.tests("only_sqlpp.xml", "testsuite_it_python.xml",
+ "../asterix-app/src/test/resources/runtimets");
+ }
+
+ protected TestCaseContext tcCtx;
+
+ public PodmanPythonFunctionIT(TestCaseContext tcCtx) {
+ this.tcCtx = tcCtx;
+ }
+
+ @Test
+ public void test() throws Exception {
+ LangExecutionUtil.test(tcCtx);
+ }
+
+ private static void setEndpoints(TestExecutor testExecutor) {
+ final Map<String, InetSocketAddress> ncEndPoints = new HashMap<>();
+ final String ip = asterix.getHost();
+ final String nodeId = "asterix_nc";
+ int apiPort = asterix.getMappedPort(19004);
+ ncEndPoints.put(nodeId, InetSocketAddress.createUnresolved(ip, apiPort));
+ testExecutor.setNcEndPoints(ncEndPoints);
+ }
+}
diff --git a/asterixdb/asterix-podman/src/test/java/org/apache/asterix/test/podman/PodmanUDFLibrarian.java b/asterixdb/asterix-podman/src/test/java/org/apache/asterix/test/podman/PodmanUDFLibrarian.java
new file mode 100644
index 0000000..025f607
--- /dev/null
+++ b/asterixdb/asterix-podman/src/test/java/org/apache/asterix/test/podman/PodmanUDFLibrarian.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.test.podman;
+
+import java.io.IOException;
+import java.net.URI;
+
+import org.apache.asterix.app.external.IExternalUDFLibrarian;
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.testcontainers.containers.Container;
+import org.testcontainers.containers.GenericContainer;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.core.json.JsonReadFeature;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+public class PodmanUDFLibrarian implements IExternalUDFLibrarian {
+ final GenericContainer<?> asterix;
+ private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+
+ public PodmanUDFLibrarian(GenericContainer asterix) {
+ OBJECT_MAPPER.configure(JsonReadFeature.ALLOW_UNESCAPED_CONTROL_CHARS.mappedFeature(), true);
+ this.asterix = asterix;
+ }
+
+ @Override
+ public void install(URI path, String type, String libPath, Pair<String, String> credentials) throws Exception {
+ Container.ExecResult curlResult = null;
+ int retryCt = 0;
+ while (retryCt < 10) {
+ try {
+ curlResult = asterix.execInContainer("curl", "--no-progress-meter", "-X", "POST", "-u",
+ credentials.first + ":" + credentials.second, "-F",
+ "data=@" + "/var/tmp/asterix-app/" + libPath, "-F", "type=" + type,
+ "http://localhost:19004" + path.getRawPath());
+ handleResponse(curlResult);
+ return;
+ } catch (RuntimeException e) {
+ retryCt++;
+ if (retryCt > 9)
+ throw e;
+ }
+ }
+ }
+
+ @Override
+ public void uninstall(URI path, Pair<String, String> credentials) throws IOException, AsterixException {
+ try {
+ Container.ExecResult curlResult = asterix.execInContainer("curl", "-X", "DELETE", "-u",
+ credentials.first + ":" + credentials.second, "http://localhost:19004" + path.getPath());
+ handleResponse(curlResult);
+ } catch (InterruptedException e) {
+ throw new IOException(e);
+ }
+ }
+
+ private void handleResponse(Container.ExecResult result) throws AsterixException, JsonProcessingException {
+ if (result.getExitCode() != 0) {
+ throw new AsterixException(result.getStderr());
+ }
+ JsonNode resp = OBJECT_MAPPER.readTree(result.getStdout().replace('\0', ' '));
+ if (resp.has("error")) {
+ throw new AsterixException(resp.get("error").toString());
+ }
+ return;
+ }
+}
diff --git a/asterixdb/asterix-podman/src/test/resources/cc.conf b/asterixdb/asterix-podman/src/test/resources/cc.conf
new file mode 100644
index 0000000..e4cbd73
--- /dev/null
+++ b/asterixdb/asterix-podman/src/test/resources/cc.conf
@@ -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.
+
+[nc/asterix_nc1]
+txn.log.dir=/opt/apache-asterixdb/data/txnlog
+core.dump.dir=/opt/apache-asterixdb/logs/coredump
+iodevices=/opt/apache-asterixdb/data/
+nc.api.port=19004
+
+[nc]
+address=127.0.0.1
+command=asterixnc
+credential.file=/opt/apache-asterixdb/etc/passwd
+jvm.args=-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=*:5006
+python.ds.path = /tmp/pyudf.socket
+
+[cc]
+address = 127.0.0.1
+
+[common]
+log.level = INFO
+log.dir = /opt/apache-asterixdb/logs/
diff --git a/asterixdb/asterix-podman/src/test/resources/passwd b/asterixdb/asterix-podman/src/test/resources/passwd
new file mode 100644
index 0000000..a1ea5b0
--- /dev/null
+++ b/asterixdb/asterix-podman/src/test/resources/passwd
@@ -0,0 +1 @@
+admin:$2a$12$JxgDzf/uOn1NS2Y3exhrDOf7JY/eUHQH7HeH90s5Ye2gALoO0FsQy
diff --git a/asterixdb/asterix-podman/src/test/resources/setup.sh b/asterixdb/asterix-podman/src/test/resources/setup.sh
new file mode 100644
index 0000000..e3523aa
--- /dev/null
+++ b/asterixdb/asterix-podman/src/test/resources/setup.sh
@@ -0,0 +1,8 @@
+#!/bin/bash
+cd /var/tmp/asterix-app/
+shiv -o target/TweetSent.pyz --site-packages src/test/resources/TweetSent scikit-learn
+cp -a /var/tmp/asterix-app/data/classifications /opt/apache-asterixdb/data/
+cp -a /var/tmp/asterix-app/data/twitter /opt/apache-asterixdb/data/
+cp -a /var/tmp/asterix-app/data/big-object /opt/apache-asterixdb/data/
+mkdir -p /opt/apache-asterixdb/target/data/
+cp -a /var/tmp/asterix-app/target/data/big-object /opt/apache-asterixdb/target/data/
\ No newline at end of file
diff --git a/asterixdb/asterix-podman/src/test/resources/socktest/Containerfile b/asterixdb/asterix-podman/src/test/resources/socktest/Containerfile
new file mode 100644
index 0000000..a7546d5
--- /dev/null
+++ b/asterixdb/asterix-podman/src/test/resources/socktest/Containerfile
@@ -0,0 +1,17 @@
+FROM ubuntu:22.04
+RUN apt -y update
+RUN DEBIAN_FRONTEND=noninteractive TZ=Etc/UTC apt -y install systemd openjdk-17-jre-headless unzip wget curl python3-pip python3-venv python3-systemd
+RUN pip3 install shiv msgpack
+COPY target/asterix-server_*all.deb .
+RUN dpkg -i asterix-server*.deb
+COPY src/test/resources/cc.conf /opt/apache-asterixdb/cc.conf
+COPY src/test/resources/passwd /opt/apache-asterixdb/etc/passwd
+RUN mkdir -p /etc/systemd/system/pyudf@.service.d/
+COPY src/test/resources/testenv.conf /etc/systemd/system/pyudf@.service.d/
+COPY src/test/resources/setup.sh /opt
+RUN chmod +x /opt/setup.sh
+RUN systemctl enable asterix-nc asterix-cc pyudf.socket
+
+EXPOSE 19001 19002 19004
+
+CMD [ "/lib/systemd/systemd" ]
diff --git a/asterixdb/asterix-podman/src/test/resources/testenv.conf b/asterixdb/asterix-podman/src/test/resources/testenv.conf
new file mode 100644
index 0000000..0c2f182
--- /dev/null
+++ b/asterixdb/asterix-podman/src/test/resources/testenv.conf
@@ -0,0 +1,3 @@
+[Service]
+Environment="FOO=BAR=BAZ"
+Environment="BAR=BAZ"
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/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/AbstractArrayMoveSwapEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractArrayMoveSwapEval.java
new file mode 100644
index 0000000..218e5ac
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractArrayMoveSwapEval.java
@@ -0,0 +1,189 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 static org.apache.asterix.om.types.EnumDeserializer.ATYPETAGDESERIALIZER;
+
+import java.io.IOException;
+
+import org.apache.asterix.builders.IAsterixListBuilder;
+import org.apache.asterix.builders.OrderedListBuilder;
+import org.apache.asterix.om.types.AOrderedListType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AbstractCollectionType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.types.TypeTagUtil;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
+import org.apache.asterix.runtime.evaluators.common.ListAccessor;
+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.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;
+
+/**
+ * Abstract class for the ARRAY_MOVE and ARRAY_SWAP functions as they share a large portion of their code
+ * with each other. Any classes that use this abstract one should override the `buildList` method, as it just
+ * otherwise builds the same input list.
+ */
+public abstract class AbstractArrayMoveSwapEval implements IScalarEvaluator {
+
+ private final ArrayBackedValueStorage storage;
+ private final IScalarEvaluator listArgEval;
+ private final IScalarEvaluator oldIndexEval;
+ private final IScalarEvaluator newIndexEval;
+ private final IPointable listArg;
+ private final IPointable oldIndex;
+ private final IPointable newIndex;
+ private final ListAccessor listAccessor;
+ private IAsterixListBuilder listBuilder;
+ private String funcIdentifier;
+ private IAType inputListType;
+
+ AbstractArrayMoveSwapEval(IScalarEvaluatorFactory[] args, IEvaluatorContext ctx, String funcIdentifier,
+ IAType inputListType) throws HyracksDataException {
+
+ this.funcIdentifier = funcIdentifier;
+ this.inputListType = inputListType;
+ storage = new ArrayBackedValueStorage();
+ listArgEval = args[0].createScalarEvaluator(ctx);
+ oldIndexEval = args[1].createScalarEvaluator(ctx);
+ newIndexEval = args[2].createScalarEvaluator(ctx);
+ listArg = new VoidPointable();
+ oldIndex = new VoidPointable();
+ newIndex = new VoidPointable();
+ listAccessor = new ListAccessor();
+ listBuilder = new OrderedListBuilder();
+ }
+
+ @Override
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+
+ storage.reset();
+
+ // Check that our args aren't missing/null
+ listArgEval.evaluate(tuple, listArg);
+ oldIndexEval.evaluate(tuple, oldIndex);
+ newIndexEval.evaluate(tuple, newIndex);
+ if (PointableHelper.checkAndSetMissingOrNull(result, listArg, oldIndex, newIndex)) {
+ return;
+ }
+
+ byte[] listBytes = listArg.getByteArray();
+ int offset = listArg.getStartOffset();
+ ATypeTag listType = ATYPETAGDESERIALIZER.deserialize(listBytes[offset]);
+
+ byte[] oldIndexBytes = oldIndex.getByteArray();
+ int oldIndexOffset = oldIndex.getStartOffset();
+ ATypeTag oldIndexType = ATYPETAGDESERIALIZER.deserialize((oldIndexBytes[oldIndexOffset]));
+
+ byte[] newIndexBytes = newIndex.getByteArray();
+ int newIndexOffset = newIndex.getStartOffset();
+ ATypeTag newIndexType = ATYPETAGDESERIALIZER.deserialize(newIndexBytes[newIndexOffset]);
+
+ // Checks that the list is of ordered list type, and that the two indices are valid numeric values.
+ // e.g) 1.0, 2, 4.0 works, but 4.5, 3.2 would not.
+ if (!(listType == ATypeTag.ARRAY) || !ATypeHierarchy.isCompatible(oldIndexType, ATypeTag.DOUBLE)
+ || !ATypeHierarchy.isCompatible(newIndexType, ATypeTag.DOUBLE)) {
+ PointableHelper.setNull(result);
+ return;
+ }
+
+ listAccessor.reset(listBytes, offset);
+
+ AbstractCollectionType outputListType;
+
+ ATypeTag listItemTypeTag = listAccessor.getItemType();
+ IAType listItemType = TypeTagUtil.getBuiltinTypeByTag(listItemTypeTag);
+ if (listAccessor.getListType() == ATypeTag.ARRAY) {
+ outputListType = new AOrderedListType(listItemType, listItemType.getTypeName());
+ }
+ // Known list type, use it directly
+ else {
+ outputListType = (AbstractCollectionType) inputListType;
+ }
+
+ listBuilder.reset(outputListType);
+
+ try {
+
+ int listLen = listAccessor.size();
+
+ double oldIndexVal = ATypeHierarchy.getDoubleValue(funcIdentifier, 1, oldIndexBytes, oldIndexOffset);
+ double newIndexVal = ATypeHierarchy.getDoubleValue(funcIdentifier, 2, newIndexBytes, newIndexOffset);
+
+ //Checks that old/new indices are within the range of the list and whether they are valid values
+ if (Double.isNaN(oldIndexVal) || Double.isInfinite(oldIndexVal) || Math.floor(oldIndexVal) < oldIndexVal
+ || newIndexVal > (listLen - 1) || newIndexVal < -(listLen) || oldIndexVal < -(listLen)
+ || oldIndexVal > (listLen - 1)) {
+ PointableHelper.setNull(result);
+ return;
+ }
+
+ // Converting the indices values into integers to be used in iteration. Also accounting for the negative indices case by using modulo
+ int oldIndexInt = (int) oldIndexVal;
+ int newIndexInt = (int) newIndexVal;
+
+ // use modulus to account for negative indices case
+ oldIndexInt = (oldIndexInt + listLen) % listLen;
+ newIndexInt = (newIndexInt + listLen) % listLen;
+
+ // if no changes are to be made, then return original list
+ if (oldIndexInt == newIndexInt || listLen <= 1) {
+ result.set(listArg);
+ return;
+ }
+
+ buildList(oldIndexInt, newIndexInt, listLen, listAccessor, listBuilder);
+
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ storage.reset();
+ listBuilder.write(storage.getDataOutput(), true);
+ result.set(storage);
+ }
+
+ /**
+ *
+ * Default: Builds a list with the exact same values as the input list. Depending on the function that extends
+ * this abstract class, this method should be overridden to suit the extending function.
+ *
+ * @param oldIndexInt - Position of the item at the old index.
+ * @param newIndexInt - Position where the item at the old index wants to be.
+ * @param listLen - to iterate through the list
+ * @param listAccessor
+ * @param listBuilder
+ * @throws IOException
+ */
+ protected void buildList(int oldIndexInt, int newIndexInt, int listLen, ListAccessor listAccessor,
+ IAsterixListBuilder listBuilder) throws IOException {
+ for (int i = 0; i < listLen; i++) {
+ storage.reset();
+ listAccessor.writeItem(oldIndexInt, storage.getDataOutput());
+ listBuilder.addItem(storage);
+ }
+
+ }
+
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayBinarySearchDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayBinarySearchDescriptor.java
new file mode 100644
index 0000000..23e13d4
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayBinarySearchDescriptor.java
@@ -0,0 +1,232 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 static org.apache.asterix.om.types.EnumDeserializer.ATYPETAGDESERIALIZER;
+
+import java.io.IOException;
+
+import org.apache.asterix.formats.nontagged.BinaryComparatorFactoryProvider;
+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.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AbstractCollectionType;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.common.ListAccessor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
+import org.apache.asterix.runtime.utils.DescriptorFactoryUtil;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+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.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;
+
+/**
+ * array_binary_search(orderedList, searchValue) returns the index of the search value if it exists within the
+ * ordered list.
+ *
+ * It returns in order:
+ * Missing, if any of the arguments are missing.
+ * Null, if the arguments are null, if the list argument is not a list, or if the searchValue argument is not numerical.
+ * Otherwise, it returns the index of the first occurrence of the search value in the input list.
+ */
+
+public class ArrayBinarySearchDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ private static final long serialVersionUID = 1L;
+
+ private IAType[] argTypes;
+
+ public static final IFunctionDescriptorFactory FACTORY = DescriptorFactoryUtil
+ .createFactory(ArrayBinarySearchDescriptor::new, FunctionTypeInferers.SET_ARGUMENTS_TYPE);
+
+ @Override
+ public void setImmutableStates(Object... states) {
+ argTypes = (IAType[]) states;
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ARRAY_BINARY_SEARCH;
+ }
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
+ throws AlgebricksException {
+ return new IScalarEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(final IEvaluatorContext ctx) throws HyracksDataException {
+ return new ArrayBinarySearchDescriptor.ArrayBinarySearchEval(args, ctx, argTypes);
+ }
+ };
+ }
+
+ public class ArrayBinarySearchEval implements IScalarEvaluator {
+
+ private final ArrayBackedValueStorage storage;
+ private final ArrayBackedValueStorage tempStorage;
+ private final IScalarEvaluator listArgEval;
+ private final IScalarEvaluator searchArgEval;
+ private final IPointable listArg;
+ private final IPointable searchArg;
+ private final IPointable tempVal;
+ private final IPointable tempVal2;
+ private final ListAccessor listAccessor;
+ private final IBinaryComparator comp;
+ private final ISerializerDeserializer<AInt32> serde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32);
+ private final AMutableInt32 resIndex = new AMutableInt32(0);
+
+ public ArrayBinarySearchEval(IScalarEvaluatorFactory[] args, IEvaluatorContext ctx, IAType[] argTypes)
+ throws HyracksDataException {
+ storage = new ArrayBackedValueStorage();
+ tempStorage = new ArrayBackedValueStorage();
+ listArg = new VoidPointable();
+ searchArg = new VoidPointable();
+ tempVal = new VoidPointable();
+ tempVal2 = new VoidPointable();
+ listArgEval = args[0].createScalarEvaluator(ctx);
+ searchArgEval = args[1].createScalarEvaluator(ctx);
+ listAccessor = new ListAccessor();
+ comp = createComparator(argTypes[0], argTypes[1]);
+ }
+
+ private IBinaryComparator createComparator(IAType listType, IAType searchValueType) {
+ IAType itemType = listType.getTypeTag().isListType() ? ((AbstractCollectionType) listType).getItemType()
+ : BuiltinType.ANY;
+ return BinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(itemType, searchValueType, true)
+ .createBinaryComparator();
+ }
+
+ @Override
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+
+ // argument missing/null checks
+ listArgEval.evaluate(tuple, listArg);
+ searchArgEval.evaluate(tuple, searchArg);
+ if (PointableHelper.checkAndSetMissingOrNull(result, listArg, searchArg)) {
+ return;
+ }
+
+ // Checking that our list arg is in fact a list
+ byte[] listBytes = listArg.getByteArray();
+ int offset = listArg.getStartOffset();
+ ATypeTag listType = ATYPETAGDESERIALIZER.deserialize(listBytes[offset]);
+
+ if (listType != ATypeTag.ARRAY) {
+ PointableHelper.setNull(result);
+ return;
+ }
+
+ byte[] searchBytes = searchArg.getByteArray();
+ int searchOffset = searchArg.getStartOffset();
+
+ listAccessor.reset(listBytes, offset);
+
+ int listLen = listAccessor.size();
+ int low = 0;
+ int high = listLen - 1;
+
+ try {
+ while (low <= high) {
+ int mid = low + ((high - low) / 2);
+ storage.reset();
+ listAccessor.getOrWriteItem(mid, tempVal, storage);
+ int comparison = comp.compare(tempVal.getByteArray(), tempVal.getStartOffset(), tempVal.getLength(),
+ searchBytes, searchOffset, searchArg.getLength());
+ if (comparison == 0) {
+ // if found, then find the first occurrence of the searchValue (from left to right)
+ int firstFoundIndex =
+ fetchFirstValue(mid, storage, tempStorage, listAccessor, comp, tempVal, tempVal2);
+ storage.reset();
+ resIndex.setValue(firstFoundIndex);
+ serde.serialize(resIndex, storage.getDataOutput());
+ result.set(storage);
+ return;
+ } else if (comparison < 0) {
+ low = mid + 1;
+ } else {
+ high = mid - 1;
+ }
+ }
+ storage.reset();
+ resIndex.setValue(-1);
+ serde.serialize(resIndex, storage.getDataOutput());
+ result.set(storage);
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ }
+ }
+
+ private int fetchFirstValue(int midIndexArg, ArrayBackedValueStorage storage, ArrayBackedValueStorage storage2,
+ ListAccessor listAccessor, IBinaryComparator comp, IPointable tempVal1, IPointable tempVal2)
+ throws IOException {
+
+ int midIndex = midIndexArg;
+
+ if (midIndex == 0) {
+ return midIndex;
+ }
+ storage.reset();
+ listAccessor.getOrWriteItem(midIndex, tempVal1, storage);
+ storage.reset();
+ listAccessor.getOrWriteItem(midIndex - 1, tempVal2, storage2);
+ int prevComparison = comp.compare(tempVal1.getByteArray(), tempVal1.getStartOffset(), tempVal1.getLength(),
+ tempVal2.getByteArray(), tempVal2.getStartOffset(), tempVal2.getLength());
+ // If values before current value are not equal, then return current position.
+ if (prevComparison != 0) {
+ return midIndex;
+ } else {
+ // midIndex-1 position was already checked, so we now start checking the previous positions
+ midIndex--;
+ // to count the number of positions before the "midIndex" value to find first occurrence of search value.
+ int counter = 0;
+ while (prevComparison == 0) {
+ counter++;
+ if (midIndex - counter == 0) {
+ return 0;
+ }
+ storage2.reset();
+ listAccessor.getOrWriteItem(midIndex - counter, tempVal2, storage2);
+ prevComparison = comp.compare(tempVal1.getByteArray(), tempVal1.getStartOffset(), tempVal1.getLength(),
+ tempVal2.getByteArray(), tempVal2.getStartOffset(), tempVal2.getLength());
+ if (prevComparison != 0) {
+ return (midIndex - counter + 1);
+ }
+ }
+ }
+ return -1;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayMoveDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayMoveDescriptor.java
new file mode 100644
index 0000000..82b5465
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayMoveDescriptor.java
@@ -0,0 +1,135 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.IOException;
+
+import org.apache.asterix.builders.IAsterixListBuilder;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.common.ListAccessor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
+import org.apache.asterix.runtime.utils.DescriptorFactoryUtil;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+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.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+
+/**
+ * array_move(list, position1, position2) returns a new list, moving the item in position1 in such a way
+ * that now, it will be in position2. It will also move all other items accordingly.
+ *
+ * It returns in order:
+ * Missing, if any of the input arguments are missing.
+ * Null, if the arguments are null, if the list argument is not a list, or if the positional arguments are not numerical.
+ * Otherwise, it returns a new list, where the item at the old position is now in the new position, and all other
+ * items are moved accordingly.
+ */
+
+public class ArrayMoveDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ private IAType inputListType;
+ private String funcIDString = String.valueOf(BuiltinFunctions.ARRAY_MOVE);
+
+ public static final IFunctionDescriptorFactory FACTORY =
+ DescriptorFactoryUtil.createFactory(ArrayMoveDescriptor::new, FunctionTypeInferers.SET_ARGUMENTS_TYPE);
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ARRAY_MOVE;
+ }
+
+ @Override
+ public void setImmutableStates(Object... states) {
+ inputListType = (IAType) states[0];
+ }
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
+ throws AlgebricksException {
+ return new IScalarEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(final IEvaluatorContext ctx) throws HyracksDataException {
+ return new ArrayMoveDescriptor.ArrayMoveEval(args, ctx);
+ }
+ };
+ }
+
+ public class ArrayMoveEval extends AbstractArrayMoveSwapEval {
+
+ private final ArrayBackedValueStorage storage;
+
+ ArrayMoveEval(IScalarEvaluatorFactory[] args, IEvaluatorContext ctx) throws HyracksDataException {
+ super(args, ctx, funcIDString, inputListType);
+ storage = new ArrayBackedValueStorage();
+ }
+
+ @Override
+ protected void buildList(int oldIndexInt, int newIndexInt, int listLen, ListAccessor listAccessor,
+ IAsterixListBuilder listBuilder) throws IOException {
+
+ for (int i = 0; i < listLen; i++) {
+
+ if (oldIndexInt < newIndexInt) {
+ // if i outside of input indices range, just add items normally
+ if (i < oldIndexInt || i > newIndexInt) {
+ storage.reset();
+ listAccessor.writeItem(i, storage.getDataOutput());
+ listBuilder.addItem(storage);
+ }
+ // if within range, but not equal to the new index, then shift the items down by 1
+ else if (i >= oldIndexInt && i < newIndexInt) {
+ storage.reset();
+ listAccessor.writeItem(i + 1, storage.getDataOutput());
+ listBuilder.addItem(storage);
+ }
+ // if at new index position, then add the item that was at old index.
+ else {
+ storage.reset();
+ listAccessor.writeItem(oldIndexInt, storage.getDataOutput());
+ listBuilder.addItem(storage);
+ }
+ } else {
+ if (i < newIndexInt || i > oldIndexInt) {
+ storage.reset();
+ listAccessor.writeItem(i, storage.getDataOutput());
+ listBuilder.addItem(storage);
+ } else if (i > newIndexInt && i <= oldIndexInt) {
+ storage.reset();
+ listAccessor.writeItem(i - 1, storage.getDataOutput());
+ listBuilder.addItem(storage);
+ } else {
+ storage.reset();
+ listAccessor.writeItem(oldIndexInt, storage.getDataOutput());
+ listBuilder.addItem(storage);
+ }
+ }
+ }
+ }
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArraySwapDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArraySwapDescriptor.java
new file mode 100644
index 0000000..279982c
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArraySwapDescriptor.java
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.IOException;
+
+import org.apache.asterix.builders.IAsterixListBuilder;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.common.ListAccessor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
+import org.apache.asterix.runtime.utils.DescriptorFactoryUtil;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+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.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+
+/**
+ * array_swap(list, position1, position2) returns a new list, switching the positions of both the items in position1
+ * and position2.
+ *
+ * It returns in order:
+ * Missing, if any of the input arguments are missing.
+ * Null, if the arguments are null, if the list argument is not a list, or if the positional arguments is not numerical.
+ * Otherwise, it returns a new list, where the two items at the
+ */
+
+public class ArraySwapDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ private IAType inputListType;
+ private String funcIDString = String.valueOf(BuiltinFunctions.ARRAY_SWAP);
+
+ public static final IFunctionDescriptorFactory FACTORY =
+ DescriptorFactoryUtil.createFactory(ArraySwapDescriptor::new, FunctionTypeInferers.SET_ARGUMENTS_TYPE);
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ARRAY_SWAP;
+ }
+
+ @Override
+ public void setImmutableStates(Object... states) {
+ inputListType = (IAType) states[0];
+ }
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
+ throws AlgebricksException {
+ return new IScalarEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(final IEvaluatorContext ctx) throws HyracksDataException {
+ return new ArraySwapDescriptor.ArraySwapEval(args, ctx);
+ }
+ };
+ }
+
+ public class ArraySwapEval extends AbstractArrayMoveSwapEval {
+ private final ArrayBackedValueStorage storage;
+
+ ArraySwapEval(IScalarEvaluatorFactory[] args, IEvaluatorContext ctx) throws HyracksDataException {
+ super(args, ctx, funcIDString, inputListType);
+ storage = new ArrayBackedValueStorage();
+ }
+
+ @Override
+ protected void buildList(int oldIndexInt, int newIndexInt, int listLen, ListAccessor listAccessor,
+ IAsterixListBuilder listBuilder) throws IOException {
+ for (int i = 0; i < listLen; i++) {
+ if (oldIndexInt < newIndexInt) {
+ if (i == oldIndexInt) {
+ storage.reset();
+ listAccessor.writeItem(newIndexInt, storage.getDataOutput());
+ listBuilder.addItem(storage);
+ }
+ // the other case is when it is one of the two input indices, in which case we want to swap the two
+ else if (i == newIndexInt) {
+ storage.reset();
+ listAccessor.writeItem(oldIndexInt, storage.getDataOutput());
+ listBuilder.addItem(storage);
+ } else {
+ storage.reset();
+ listAccessor.writeItem(i, storage.getDataOutput());
+ listBuilder.addItem(storage);
+ }
+ } else if (oldIndexInt > newIndexInt) {
+ if (i == newIndexInt) {
+ storage.reset();
+ listAccessor.writeItem(oldIndexInt, storage.getDataOutput());
+ listBuilder.addItem(storage);
+ } else if (i == oldIndexInt) {
+ storage.reset();
+ listAccessor.writeItem(newIndexInt, storage.getDataOutput());
+ listBuilder.addItem(storage);
+ } else {
+ storage.reset();
+ listAccessor.writeItem(i, storage.getDataOutput());
+ listBuilder.addItem(storage);
+ }
+ }
+ }
+ }
+
+ }
+
+}
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/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/records/AbstractRecordAddPutEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/AbstractRecordAddPutEvaluator.java
index 47e84eb..ed7da3f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/AbstractRecordAddPutEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/AbstractRecordAddPutEvaluator.java
@@ -16,101 +16,64 @@
* specific language governing permissions and limitations
* under the License.
*/
-
package org.apache.asterix.runtime.evaluators.functions.records;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.asterix.builders.RecordBuilder;
-import org.apache.asterix.om.pointables.ARecordVisitablePointable;
-import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.runtime.evaluators.functions.CastTypeEvaluator;
import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
-import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.accessors.UTF8StringBinaryComparatorFactory;
import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
-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;
-abstract class AbstractRecordAddPutEvaluator implements IScalarEvaluator {
-
- private final CastTypeEvaluator inputRecordCaster;
- private final CastTypeEvaluator argRecordCaster;
+abstract class AbstractRecordAddPutEvaluator extends AbstractRecordFunctionEvaluator {
private final IScalarEvaluator eval0;
private final IScalarEvaluator eval1;
private final IScalarEvaluator eval2;
- final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
- final DataOutput resultOutput = resultStorage.getDataOutput();
- final IPointable inputRecordPointable = new VoidPointable();
- final UTF8StringPointable newFieldNamePointable = new UTF8StringPointable();
- final IPointable newFieldValuePointable = new VoidPointable();
- final IBinaryComparator stringBinaryComparator =
- UTF8StringBinaryComparatorFactory.INSTANCE.createBinaryComparator();
- final RecordBuilder outRecordBuilder = new RecordBuilder();
- final ARecordVisitablePointable inputOpenRecordPointable;
- boolean newFieldValueIsMissing = false;
+ protected boolean newFieldValueIsMissing = false;
AbstractRecordAddPutEvaluator(IScalarEvaluator eval0, IScalarEvaluator eval1, IScalarEvaluator eval2,
- IAType[] argTypes) {
+ ARecordType outRecType, ARecordType inRecType) {
+ super(outRecType, inRecType);
this.eval0 = eval0;
this.eval1 = eval1;
this.eval2 = eval2;
- inputOpenRecordPointable = new ARecordVisitablePointable(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
- inputRecordCaster = new CastTypeEvaluator(BuiltinType.ANY, argTypes[0], eval0);
- argRecordCaster = new CastTypeEvaluator(BuiltinType.ANY, argTypes[2], eval2);
}
@Override
public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
- eval0.evaluate(tuple, inputRecordPointable);
+ eval0.evaluate(tuple, inputPointable);
eval1.evaluate(tuple, newFieldNamePointable);
eval2.evaluate(tuple, newFieldValuePointable);
- if (containsMissing(inputRecordPointable, newFieldNamePointable)) {
- writeTypeTag(ATypeTag.SERIALIZED_MISSING_TYPE_TAG);
- result.set(resultStorage);
+ ATypeTag inputTypeTag = PointableHelper.getTypeTag(inputPointable);
+ ATypeTag newFieldNameTypeTag = PointableHelper.getTypeTag(newFieldNamePointable);
+ if (inputTypeTag == ATypeTag.MISSING || newFieldNameTypeTag == ATypeTag.MISSING) {
+ PointableHelper.setMissing(result);
return;
}
- final ATypeTag inputObjectType = PointableHelper.getTypeTag(inputRecordPointable);
- final ATypeTag newFieldNameValueType = PointableHelper.getTypeTag(newFieldNamePointable);
- if (inputObjectType != ATypeTag.OBJECT || newFieldNameValueType != ATypeTag.STRING) {
+ if (inputTypeTag != ATypeTag.OBJECT || newFieldNameTypeTag != ATypeTag.STRING) {
PointableHelper.setNull(result);
return;
}
- inputRecordCaster.evaluate(tuple, inputRecordPointable);
- final ATypeTag newFieldValueTag = PointableHelper.getTypeTag(newFieldValuePointable);
- if (newFieldValueTag.isDerivedType()) {
- argRecordCaster.evaluate(tuple, newFieldValuePointable);
+ newFieldValueIsMissing = PointableHelper.getTypeTag(newFieldValuePointable) == ATypeTag.MISSING;
+ outputRecordTypeInfo.reset(outRecType);
+ if (inputRecordPointable == null) {
+ inputRecordPointable = pointableAllocator.allocateRecordValue(inRecType);
}
- newFieldValueIsMissing = newFieldValueTag == ATypeTag.MISSING;
- buildOutputRecord();
+ buildOutputRecord(result);
result.set(resultStorage);
}
- protected abstract void buildOutputRecord() throws HyracksDataException;
+ protected abstract void buildOutputRecord(IPointable result) throws HyracksDataException;
- private boolean containsMissing(IPointable... pointables) {
- for (int i = 0; i < pointables.length; i++) {
- if (PointableHelper.getTypeTag(pointables[i]) == ATypeTag.MISSING) {
- return true;
- }
- }
- return false;
- }
-
- private void writeTypeTag(byte typeTag) throws HyracksDataException {
- try {
- resultOutput.writeByte(typeTag);
- } catch (IOException e) {
- throw HyracksDataException.create(e);
+ protected void addField(IPointable fieldName, IPointable fieldValue) throws HyracksDataException {
+ int pos = outputRecordTypeInfo.getFieldIndex(fieldName.getByteArray(), fieldName.getStartOffset() + 1,
+ fieldName.getLength() - 1);
+ if (pos >= 0) {
+ outRecordBuilder.addField(pos, fieldValue);
+ } else {
+ outRecordBuilder.addField(fieldName, fieldValue);
}
}
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/AbstractRecordFunctionEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/AbstractRecordFunctionEvaluator.java
new file mode 100644
index 0000000..2bb192d
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/AbstractRecordFunctionEvaluator.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.evaluators.functions.records;
+
+import java.io.DataOutput;
+
+import org.apache.asterix.builders.RecordBuilder;
+import org.apache.asterix.om.pointables.ARecordVisitablePointable;
+import org.apache.asterix.om.pointables.PointableAllocator;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.runtime.RuntimeRecordTypeInfo;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.data.std.accessors.UTF8StringBinaryComparatorFactory;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+
+/**
+ * Base evaluator class for the following functions:
+ * 1. {@link org.apache.asterix.om.functions.BuiltinFunctions#RECORD_ADD}
+ * 2. {@link org.apache.asterix.om.functions.BuiltinFunctions#RECORD_PUT}
+ * 3. {@link org.apache.asterix.om.functions.BuiltinFunctions#RECORD_REMOVE}
+ * 4. {@link org.apache.asterix.om.functions.BuiltinFunctions#RECORD_RENAME}
+ */
+abstract class AbstractRecordFunctionEvaluator implements IScalarEvaluator {
+ protected final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+ protected final DataOutput resultOutput = resultStorage.getDataOutput();
+ protected final RecordBuilder outRecordBuilder = new RecordBuilder();
+
+ protected final IPointable newFieldNamePointable = new VoidPointable();
+ protected final IPointable newFieldValuePointable = new VoidPointable();
+ protected final IBinaryComparator stringBinaryComparator =
+ UTF8StringBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+
+ protected final PointableAllocator pointableAllocator = new PointableAllocator();
+ protected final IPointable inputPointable = new VoidPointable();
+ protected final ARecordType inRecType;
+ protected ARecordVisitablePointable inputRecordPointable;
+
+ protected final ARecordType outRecType;
+ protected final RuntimeRecordTypeInfo outputRecordTypeInfo = new RuntimeRecordTypeInfo();
+
+ AbstractRecordFunctionEvaluator(ARecordType outRecType, ARecordType inRecType) {
+ this.outRecType = outRecType;
+ this.inRecType = inRecType;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddDescriptor.java
index 67f2c8d..075b803 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddDescriptor.java
@@ -22,6 +22,8 @@
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.om.typecomputer.impl.TypeComputeUtils;
+import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.IAType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.functions.FunctionTypeInferers;
@@ -32,7 +34,6 @@
import org.apache.hyracks.api.exceptions.HyracksDataException;
public class RecordAddDescriptor extends AbstractScalarFunctionDynamicDescriptor {
-
public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
@Override
public IFunctionDescriptor createFunctionDescriptor() {
@@ -41,19 +42,18 @@
@Override
public IFunctionTypeInferer createFunctionTypeInferer() {
- return FunctionTypeInferers.SET_ARGUMENTS_TYPE;
+ return FunctionTypeInferers.RECORD_MODIFY_INFERER;
}
};
private static final long serialVersionUID = 1L;
- private IAType[] argTypes;
+ private ARecordType outRecType;
+ private ARecordType inRecType;
@Override
public void setImmutableStates(Object... states) {
- argTypes = new IAType[states.length];
- for (int i = 0; i < states.length; i++) {
- argTypes[i] = (IAType) states[i];
- }
+ outRecType = TypeComputeUtils.extractRecordType((IAType) states[0]);
+ inRecType = TypeComputeUtils.extractRecordType((IAType) states[1]);
}
@Override
@@ -67,7 +67,7 @@
for (int i = 0; i < args.length; i++) {
argEvals[i] = args[i].createScalarEvaluator(ctx);
}
- return new RecordAddEvaluator(argEvals[0], argEvals[1], argEvals[2], argTypes);
+ return new RecordAddEvaluator(argEvals[0], argEvals[1], argEvals[2], outRecType, inRecType);
}
};
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddEvaluator.java
index 52a69bf..3df16d0 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddEvaluator.java
@@ -16,42 +16,41 @@
* specific language governing permissions and limitations
* under the License.
*/
-
package org.apache.asterix.runtime.evaluators.functions.records;
import java.util.List;
-import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
import org.apache.asterix.om.pointables.base.IVisitablePointable;
-import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
class RecordAddEvaluator extends AbstractRecordAddPutEvaluator {
-
- RecordAddEvaluator(IScalarEvaluator eval0, IScalarEvaluator eval1, IScalarEvaluator eval2, IAType[] argTypes) {
- super(eval0, eval1, eval2, argTypes);
+ RecordAddEvaluator(IScalarEvaluator eval0, IScalarEvaluator eval1, IScalarEvaluator eval2, ARecordType outRecType,
+ ARecordType inRecType) {
+ super(eval0, eval1, eval2, outRecType, inRecType);
}
@Override
- protected void buildOutputRecord() throws HyracksDataException {
+ protected void buildOutputRecord(IPointable result) throws HyracksDataException {
resultStorage.reset();
- outRecordBuilder.reset(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
- outRecordBuilder.init();
- inputOpenRecordPointable.set(inputRecordPointable);
- final List<IVisitablePointable> fieldNames = inputOpenRecordPointable.getFieldNames();
- final List<IVisitablePointable> fieldValues = inputOpenRecordPointable.getFieldValues();
+ outRecordBuilder.reset(outRecType);
+ inputRecordPointable.set(inputPointable);
+ final List<IVisitablePointable> fieldNames = inputRecordPointable.getFieldNames();
+ final List<IVisitablePointable> fieldValues = inputRecordPointable.getFieldValues();
boolean newFieldFound = false;
for (int i = 0, fieldCount = fieldNames.size(); i < fieldCount; i++) {
final IVisitablePointable fieldName = fieldNames.get(i);
- if (PointableHelper.isEqual(fieldName, newFieldNamePointable, stringBinaryComparator)) {
+ final IVisitablePointable fieldValue = fieldValues.get(i);
+ if (!newFieldFound && PointableHelper.isEqual(fieldName, newFieldNamePointable, stringBinaryComparator)) {
newFieldFound = true;
}
- outRecordBuilder.addField(fieldName, fieldValues.get(i));
+ addField(fieldName, fieldValue);
}
if (!newFieldValueIsMissing && !newFieldFound) {
- outRecordBuilder.addField(newFieldNamePointable, newFieldValuePointable);
+ addField(newFieldNamePointable, newFieldValuePointable);
}
outRecordBuilder.write(resultOutput, true);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordConcatEvalFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordConcatEvalFactory.java
index bc1cacc..890bb13 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordConcatEvalFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordConcatEvalFactory.java
@@ -25,6 +25,7 @@
import java.util.List;
import org.apache.asterix.builders.RecordBuilder;
+import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.om.functions.BuiltinFunctions;
import org.apache.asterix.om.pointables.ARecordVisitablePointable;
import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
@@ -41,12 +42,16 @@
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.api.exceptions.Warning;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
import org.apache.hyracks.data.std.util.BinaryEntry;
import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+import org.apache.hyracks.util.LogRedactionUtil;
+import org.apache.hyracks.util.string.UTF8StringUtil;
class RecordConcatEvalFactory implements IScalarEvaluatorFactory {
@@ -77,7 +82,7 @@
for (int i = 0; i < args.length; i++) {
argEvals[i] = args[i].createScalarEvaluator(ctx);
}
- return new RecordConcatEvaluator(argEvals);
+ return new RecordConcatEvaluator(argEvals, ctx.getWarningCollector());
}
private final class RecordConcatEvaluator implements IScalarEvaluator {
@@ -109,10 +114,13 @@
private final BinaryEntry keyEntry;
private final BinaryEntry valEntry;
+ private final IWarningCollector warningCollector;
+
private int numRecords;
- private RecordConcatEvaluator(IScalarEvaluator[] argEvals) {
+ private RecordConcatEvaluator(IScalarEvaluator[] argEvals, IWarningCollector warningCollector) {
this.argEvals = argEvals;
+ this.warningCollector = warningCollector;
firstArg = new VoidPointable();
openRecordPointable = new ARecordVisitablePointable(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
@@ -309,6 +317,12 @@
if (canAppendField(fieldName.getByteArray(), fieldName.getStartOffset() + 1,
fieldName.getLength() - 1)) {
outRecordBuilder.addField(fieldName, fieldValues.get(i));
+ } else {
+ if (warningCollector.shouldWarn()) {
+ warningCollector.warn(Warning.of(sourceLoc, ErrorCode.DUPLICATE_FIELD_NAME,
+ LogRedactionUtil.userData(UTF8StringUtil.toString(fieldName.getByteArray(),
+ fieldName.getStartOffset() + 1))));
+ }
}
}
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPutDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPutDescriptor.java
index e4d72e5..870a2e8 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPutDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPutDescriptor.java
@@ -22,6 +22,8 @@
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.om.typecomputer.impl.TypeComputeUtils;
+import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.IAType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.functions.FunctionTypeInferers;
@@ -32,7 +34,6 @@
import org.apache.hyracks.api.exceptions.HyracksDataException;
public class RecordPutDescriptor extends AbstractScalarFunctionDynamicDescriptor {
-
public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
@Override
public IFunctionDescriptor createFunctionDescriptor() {
@@ -41,19 +42,18 @@
@Override
public IFunctionTypeInferer createFunctionTypeInferer() {
- return FunctionTypeInferers.SET_ARGUMENTS_TYPE;
+ return FunctionTypeInferers.RECORD_MODIFY_INFERER;
}
};
private static final long serialVersionUID = 1L;
- private IAType[] argTypes;
+ private ARecordType outRecType;
+ private ARecordType inRecType;
@Override
public void setImmutableStates(Object... states) {
- argTypes = new IAType[states.length];
- for (int i = 0; i < states.length; i++) {
- argTypes[i] = (IAType) states[i];
- }
+ outRecType = TypeComputeUtils.extractRecordType((IAType) states[0]);
+ inRecType = TypeComputeUtils.extractRecordType((IAType) states[1]);
}
@Override
@@ -67,7 +67,7 @@
for (int i = 0; i < args.length; i++) {
argEvals[i] = args[i].createScalarEvaluator(ctx);
}
- return new RecordPutEvaluator(argEvals[0], argEvals[1], argEvals[2], argTypes);
+ return new RecordPutEvaluator(argEvals[0], argEvals[1], argEvals[2], outRecType, inRecType);
}
};
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPutEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPutEvaluator.java
index 857bd2e..3a9f3f8 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPutEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPutEvaluator.java
@@ -16,51 +16,46 @@
* specific language governing permissions and limitations
* under the License.
*/
-
package org.apache.asterix.runtime.evaluators.functions.records;
import java.util.List;
-import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
import org.apache.asterix.om.pointables.base.IVisitablePointable;
-import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
class RecordPutEvaluator extends AbstractRecordAddPutEvaluator {
-
- RecordPutEvaluator(IScalarEvaluator eval0, IScalarEvaluator eval1, IScalarEvaluator eval2, IAType[] argTypes) {
- super(eval0, eval1, eval2, argTypes);
+ RecordPutEvaluator(IScalarEvaluator eval0, IScalarEvaluator eval1, IScalarEvaluator eval2, ARecordType outRecType,
+ ARecordType inRecType) {
+ super(eval0, eval1, eval2, outRecType, inRecType);
}
@Override
- protected void buildOutputRecord() throws HyracksDataException {
+ protected void buildOutputRecord(IPointable result) throws HyracksDataException {
resultStorage.reset();
- outRecordBuilder.reset(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
- outRecordBuilder.init();
- inputOpenRecordPointable.set(inputRecordPointable);
- final List<IVisitablePointable> fieldNames = inputOpenRecordPointable.getFieldNames();
- final List<IVisitablePointable> fieldValues = inputOpenRecordPointable.getFieldValues();
+ outRecordBuilder.reset(outRecType);
+ inputRecordPointable.set(inputPointable);
+ final List<IVisitablePointable> fieldNames = inputRecordPointable.getFieldNames();
+ final List<IVisitablePointable> fieldValues = inputRecordPointable.getFieldValues();
boolean newFieldFound = false;
for (int i = 0, fieldCount = fieldNames.size(); i < fieldCount; i++) {
final IVisitablePointable fieldName = fieldNames.get(i);
- if (!PointableHelper.isEqual(fieldName, newFieldNamePointable, stringBinaryComparator)) {
- outRecordBuilder.addField(fieldName, fieldValues.get(i));
+ final IVisitablePointable fieldValue = fieldValues.get(i);
+ if (!newFieldFound && !PointableHelper.isEqual(fieldName, newFieldNamePointable, stringBinaryComparator)) {
+ addField(fieldName, fieldValue);
} else {
newFieldFound = true;
if (!newFieldValueIsMissing) {
- putNewField();
+ addField(newFieldNamePointable, newFieldValuePointable);
}
}
}
if (!newFieldFound) {
- putNewField();
+ addField(newFieldNamePointable, newFieldValuePointable);
}
outRecordBuilder.write(resultOutput, true);
}
-
- private void putNewField() throws HyracksDataException {
- outRecordBuilder.addField(newFieldNamePointable, newFieldValuePointable);
- }
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveDescriptor.java
index 8e44ada..818b650 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveDescriptor.java
@@ -23,7 +23,9 @@
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.om.typecomputer.impl.TypeComputeUtils;
import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.IAType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.functions.FunctionTypeInferers;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,7 +36,6 @@
@MissingNullInOutFunction
public class RecordRemoveDescriptor extends AbstractScalarFunctionDynamicDescriptor {
-
public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
@Override
public IFunctionDescriptor createFunctionDescriptor() {
@@ -43,16 +44,18 @@
@Override
public IFunctionTypeInferer createFunctionTypeInferer() {
- return FunctionTypeInferers.RecordAccessorTypeInferer.INSTANCE_LAX;
+ return FunctionTypeInferers.RECORD_MODIFY_INFERER;
}
};
private static final long serialVersionUID = 1L;
- private ARecordType recordType;
+ private ARecordType outRecType;
+ private ARecordType inRecType;
@Override
public void setImmutableStates(Object... states) {
- recordType = (ARecordType) states[0];
+ outRecType = TypeComputeUtils.extractRecordType((IAType) states[0]);
+ inRecType = TypeComputeUtils.extractRecordType((IAType) states[1]);
}
@Override
@@ -66,7 +69,7 @@
for (int i = 0; i < args.length; i++) {
argEvals[i] = args[i].createScalarEvaluator(ctx);
}
- return new RecordRemoveEvaluator(argEvals[0], argEvals[1], recordType);
+ return new RecordRemoveEvaluator(argEvals[0], argEvals[1], outRecType, inRecType);
}
};
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveEvaluator.java
index 51f66b0..e560c10 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveEvaluator.java
@@ -16,140 +16,87 @@
* specific language governing permissions and limitations
* under the License.
*/
-
package org.apache.asterix.runtime.evaluators.functions.records;
-import java.io.DataOutput;
import java.io.IOException;
import java.util.List;
-import org.apache.asterix.builders.RecordBuilder;
-import org.apache.asterix.om.pointables.ARecordVisitablePointable;
-import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
import org.apache.asterix.om.pointables.base.IVisitablePointable;
-import org.apache.asterix.om.pointables.cast.ACastVisitor;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.IAType;
import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
-import org.apache.hyracks.algebricks.common.utils.Triple;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
-import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.accessors.UTF8StringBinaryComparatorFactory;
import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
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;
-class RecordRemoveEvaluator implements IScalarEvaluator {
-
- private final IPointable inputRecordPointable = new VoidPointable();
- private final UTF8StringPointable fieldToRemovePointable = new UTF8StringPointable();
- private final IBinaryComparator stringBinaryComparator =
- UTF8StringBinaryComparatorFactory.INSTANCE.createBinaryComparator();
- private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
- private final DataOutput resultOutput = resultStorage.getDataOutput();
- private final RecordBuilder outRecordBuilder = new RecordBuilder();
+class RecordRemoveEvaluator extends AbstractRecordFunctionEvaluator {
private final IScalarEvaluator eval0;
private final IScalarEvaluator eval1;
- private final ARecordVisitablePointable openRecordPointable;
- private ARecordVisitablePointable inputRecordVisitable;
- private boolean requiresCast = false;
- private ACastVisitor castVisitor;
- private Triple<IVisitablePointable, IAType, Boolean> castVisitorArg;
+ private final IPointable fieldToRemovePointable = new VoidPointable();
- RecordRemoveEvaluator(IScalarEvaluator eval0, IScalarEvaluator eval1, ARecordType recordType) {
+ RecordRemoveEvaluator(IScalarEvaluator eval0, IScalarEvaluator eval1, ARecordType outRecType,
+ ARecordType inRecType) {
+ super(outRecType, inRecType);
this.eval0 = eval0;
this.eval1 = eval1;
- openRecordPointable = new ARecordVisitablePointable(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
- if (recordType != null) {
- inputRecordVisitable = new ARecordVisitablePointable(recordType);
- if (hasDerivedType(recordType.getFieldTypes())) {
- requiresCast = true;
- castVisitor = new ACastVisitor();
- castVisitorArg =
- new Triple<>(openRecordPointable, openRecordPointable.getInputRecordType(), Boolean.FALSE);
- }
- }
}
@Override
public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
- boolean returnNull = false;
- eval0.evaluate(tuple, inputRecordPointable);
+ eval0.evaluate(tuple, inputPointable);
eval1.evaluate(tuple, fieldToRemovePointable);
-
- if (PointableHelper.checkAndSetMissingOrNull(result, inputRecordPointable, fieldToRemovePointable)) {
+ if (PointableHelper.checkAndSetMissingOrNull(result, inputPointable, fieldToRemovePointable)) {
return;
}
- byte[] data = inputRecordPointable.getByteArray();
- int offset = inputRecordPointable.getStartOffset();
+ // Check the type of our first argument.
+ byte[] data = inputPointable.getByteArray();
+ int offset = inputPointable.getStartOffset();
byte typeTag = data[offset];
if (typeTag != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
- returnNull = true;
+ PointableHelper.setNull(result);
+ return;
}
+ // Check the type of our second argument.
data = fieldToRemovePointable.getByteArray();
offset = fieldToRemovePointable.getStartOffset();
typeTag = data[offset];
if (typeTag != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
- returnNull = true;
- }
- if (returnNull) {
PointableHelper.setNull(result);
return;
}
- evaluate();
- result.set(resultStorage);
- }
- private void evaluate() throws HyracksDataException {
- resultStorage.reset();
try {
- final ARecordVisitablePointable inputRecord = getInputRecordVisitablePointable();
- buildOutputRecord(inputRecord);
+ outRecordBuilder.reset(outRecType);
+ outputRecordTypeInfo.reset(outRecType);
+ if (inputRecordPointable == null) {
+ inputRecordPointable = pointableAllocator.allocateRecordValue(inRecType);
+ }
+ inputRecordPointable.set(inputPointable);
+ final List<IVisitablePointable> fieldNames = inputRecordPointable.getFieldNames();
+ final List<IVisitablePointable> fieldValues = inputRecordPointable.getFieldValues();
+ for (int i = 0, fieldCount = fieldNames.size(); i < fieldCount; i++) {
+ final IVisitablePointable fieldName = fieldNames.get(i);
+ final IVisitablePointable fieldValue = fieldValues.get(i);
+ if (!PointableHelper.isEqual(fieldName, fieldToRemovePointable, stringBinaryComparator)) {
+ int pos = outputRecordTypeInfo.getFieldIndex(fieldName.getByteArray(),
+ fieldName.getStartOffset() + 1, fieldName.getLength() - 1);
+ if (pos >= 0) {
+ outRecordBuilder.addField(pos, fieldValue);
+ } else {
+ outRecordBuilder.addField(fieldName, fieldValue);
+ }
+ }
+ }
+ outRecordBuilder.write(resultOutput, true);
+
} catch (IOException e) {
throw HyracksDataException.create(e);
}
- }
-
- private void buildOutputRecord(ARecordVisitablePointable inputRecord) throws HyracksDataException {
- outRecordBuilder.reset(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
- outRecordBuilder.init();
- final List<IVisitablePointable> fieldNames = inputRecord.getFieldNames();
- final List<IVisitablePointable> fieldValues = inputRecord.getFieldValues();
- for (int i = 0, fieldCount = fieldNames.size(); i < fieldCount; i++) {
- final IVisitablePointable fieldName = fieldNames.get(i);
- if (!PointableHelper.isEqual(fieldName, fieldToRemovePointable, stringBinaryComparator)) {
- outRecordBuilder.addField(fieldName, fieldValues.get(i));
- }
- }
- outRecordBuilder.write(resultOutput, true);
- }
-
- private ARecordVisitablePointable getInputRecordVisitablePointable() throws HyracksDataException {
- inputRecordVisitable.set(inputRecordPointable);
- if (requiresCast) {
- return castToOpenRecord();
- }
- return inputRecordVisitable;
- }
-
- private boolean hasDerivedType(IAType[] types) {
- for (IAType type : types) {
- if (type.getTypeTag().isDerivedType()) {
- return true;
- }
- }
- return false;
- }
-
- private ARecordVisitablePointable castToOpenRecord() throws HyracksDataException {
- inputRecordVisitable.accept(castVisitor, castVisitorArg);
- return openRecordPointable;
+ result.set(resultStorage);
}
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveFieldsEvalFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveFieldsEvalFactory.java
index c8d4824..1a453a7 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveFieldsEvalFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveFieldsEvalFactory.java
@@ -33,6 +33,7 @@
import org.apache.asterix.om.pointables.PointableAllocator;
import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
import org.apache.asterix.om.pointables.base.IVisitablePointable;
+import org.apache.asterix.om.typecomputer.impl.TypeComputeUtils;
import org.apache.asterix.om.types.AOrderedListType;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.ATypeTag;
@@ -170,8 +171,9 @@
int pos = runtimeRecordTypeInfo.getFieldIndex(fieldNamePointable.getByteArray(),
fieldNamePointable.getStartOffset() + 1, fieldNamePointable.getLength() - 1);
if (pos >= 0) { // Closed field
- if (PointableHelper.sameType(ATypeTag.OBJECT, fieldTypePointable)) {
- processRecord((ARecordType) requiredType.getFieldTypes()[pos],
+ if (PointableHelper.sameType(ATypeTag.OBJECT, fieldTypePointable)
+ && PointableHelper.sameType(ATypeTag.OBJECT, fieldValuePointable)) {
+ processRecord((ARecordType) TypeComputeUtils.getActualType(requiredType.getFieldTypes()[pos]),
(ARecordVisitablePointable) fieldValuePointable, inputList, nestedLevel + 1);
tabvs.reset();
rbStack.get(nestedLevel + 1).write(tabvs.getDataOutput(), true);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRenameDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRenameDescriptor.java
index 8b345aa..ea4f6db 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRenameDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRenameDescriptor.java
@@ -23,7 +23,9 @@
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.om.typecomputer.impl.TypeComputeUtils;
import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.IAType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.functions.FunctionTypeInferers;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,7 +36,6 @@
@MissingNullInOutFunction
public class RecordRenameDescriptor extends AbstractScalarFunctionDynamicDescriptor {
-
public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
@Override
public IFunctionDescriptor createFunctionDescriptor() {
@@ -43,16 +44,18 @@
@Override
public IFunctionTypeInferer createFunctionTypeInferer() {
- return FunctionTypeInferers.RecordAccessorTypeInferer.INSTANCE_LAX;
+ return FunctionTypeInferers.RECORD_MODIFY_INFERER;
}
};
private static final long serialVersionUID = 1L;
- private ARecordType recordType;
+ private ARecordType outRecType;
+ private ARecordType inRecType;
@Override
public void setImmutableStates(Object... states) {
- recordType = (ARecordType) states[0];
+ outRecType = TypeComputeUtils.extractRecordType((IAType) states[0]);
+ inRecType = TypeComputeUtils.extractRecordType((IAType) states[1]);
}
@Override
@@ -66,7 +69,7 @@
for (int i = 0; i < args.length; i++) {
argEvals[i] = args[i].createScalarEvaluator(ctx);
}
- return new RecordRenameEvaluator(argEvals[0], argEvals[1], argEvals[2], recordType);
+ return new RecordRenameEvaluator(argEvals[0], argEvals[1], argEvals[2], outRecType, inRecType);
}
};
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRenameEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRenameEvaluator.java
index ac09ffa..1461536 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRenameEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRenameEvaluator.java
@@ -16,156 +16,105 @@
* specific language governing permissions and limitations
* under the License.
*/
-
package org.apache.asterix.runtime.evaluators.functions.records;
-import java.io.DataOutput;
import java.io.IOException;
import java.util.List;
-import org.apache.asterix.builders.RecordBuilder;
-import org.apache.asterix.om.pointables.ARecordVisitablePointable;
-import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
import org.apache.asterix.om.pointables.base.IVisitablePointable;
-import org.apache.asterix.om.pointables.cast.ACastVisitor;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.IAType;
import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
-import org.apache.hyracks.algebricks.common.utils.Triple;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
-import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.accessors.UTF8StringBinaryComparatorFactory;
import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
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;
-class RecordRenameEvaluator implements IScalarEvaluator {
-
- private final IPointable inputRecordPointable = new VoidPointable();
- private final UTF8StringPointable oldFieldNamePointable = new UTF8StringPointable();
- private final UTF8StringPointable newFieldNamePointable = new UTF8StringPointable();
- private final IBinaryComparator stringBinaryComparator =
- UTF8StringBinaryComparatorFactory.INSTANCE.createBinaryComparator();
- private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
- private final DataOutput resultOutput = resultStorage.getDataOutput();
- private final RecordBuilder outRecordBuilder = new RecordBuilder();
+class RecordRenameEvaluator extends AbstractRecordFunctionEvaluator {
private final IScalarEvaluator eval0;
private final IScalarEvaluator eval1;
private final IScalarEvaluator eval2;
- private final ARecordVisitablePointable openRecordPointable;
- private ARecordVisitablePointable inputRecordVisitable;
- private boolean requiresCast = false;
- private ACastVisitor castVisitor;
- private Triple<IVisitablePointable, IAType, Boolean> castVisitorArg;
+ private final IPointable oldFieldNamePointable = new VoidPointable();
RecordRenameEvaluator(IScalarEvaluator eval0, IScalarEvaluator eval1, IScalarEvaluator eval2,
- ARecordType recordType) {
+ ARecordType outRecType, ARecordType inRecType) {
+ super(outRecType, inRecType);
this.eval0 = eval0;
this.eval1 = eval1;
this.eval2 = eval2;
- openRecordPointable = new ARecordVisitablePointable(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
- if (recordType != null) {
- inputRecordVisitable = new ARecordVisitablePointable(recordType);
- if (hasDerivedType(recordType.getFieldTypes())) {
- requiresCast = true;
- castVisitor = new ACastVisitor();
- castVisitorArg =
- new Triple<>(openRecordPointable, openRecordPointable.getInputRecordType(), Boolean.FALSE);
- }
- }
}
@Override
public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
- boolean returnNull = false;
- eval0.evaluate(tuple, inputRecordPointable);
+ eval0.evaluate(tuple, inputPointable);
eval1.evaluate(tuple, oldFieldNamePointable);
eval2.evaluate(tuple, newFieldNamePointable);
-
- if (PointableHelper.checkAndSetMissingOrNull(result, inputRecordPointable, oldFieldNamePointable,
+ if (PointableHelper.checkAndSetMissingOrNull(result, inputPointable, oldFieldNamePointable,
newFieldNamePointable)) {
return;
}
- byte[] data = inputRecordPointable.getByteArray();
- int offset = inputRecordPointable.getStartOffset();
+ // Check the type of our first argument.
+ byte[] data = inputPointable.getByteArray();
+ int offset = inputPointable.getStartOffset();
byte typeTag = data[offset];
if (typeTag != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
- returnNull = true;
+ PointableHelper.setNull(result);
+ return;
}
+ // Check the type of our second argument.
data = oldFieldNamePointable.getByteArray();
offset = oldFieldNamePointable.getStartOffset();
typeTag = data[offset];
if (typeTag != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
- returnNull = true;
+ PointableHelper.setNull(result);
+ return;
}
+ // Check the type of our third argument.
data = newFieldNamePointable.getByteArray();
offset = newFieldNamePointable.getStartOffset();
typeTag = data[offset];
if (typeTag != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
- returnNull = true;
- }
- if (returnNull) {
PointableHelper.setNull(result);
return;
}
- evaluate();
- result.set(resultStorage);
- }
- private void evaluate() throws HyracksDataException {
- resultStorage.reset();
try {
- final ARecordVisitablePointable inputRecord = getInputRecordVisitablePointable();
- buildOutputRecord(inputRecord);
+ outRecordBuilder.reset(outRecType);
+ outputRecordTypeInfo.reset(outRecType);
+ if (inputRecordPointable == null) {
+ inputRecordPointable = pointableAllocator.allocateRecordValue(inRecType);
+ }
+ inputRecordPointable.set(inputPointable);
+ final List<IVisitablePointable> fieldNames = inputRecordPointable.getFieldNames();
+ final List<IVisitablePointable> fieldValues = inputRecordPointable.getFieldValues();
+ for (int i = 0, fieldCount = fieldNames.size(); i < fieldCount; i++) {
+ final IVisitablePointable fieldName = fieldNames.get(i);
+ final IVisitablePointable fieldValue = fieldValues.get(i);
+ if (!PointableHelper.isEqual(fieldName, oldFieldNamePointable, stringBinaryComparator)) {
+ addField(fieldName, fieldValue);
+ } else {
+ addField(newFieldNamePointable, fieldValue);
+ }
+ }
+ outRecordBuilder.write(resultOutput, true);
} catch (IOException e) {
throw HyracksDataException.create(e);
}
+ result.set(resultStorage);
}
- private void buildOutputRecord(ARecordVisitablePointable inputRecord) throws HyracksDataException {
- outRecordBuilder.reset(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
- outRecordBuilder.init();
- final List<IVisitablePointable> fieldNames = inputRecord.getFieldNames();
- final List<IVisitablePointable> fieldValues = inputRecord.getFieldValues();
- for (int i = 0, fieldCount = fieldNames.size(); i < fieldCount; i++) {
- final IVisitablePointable fieldName = fieldNames.get(i);
- if (!PointableHelper.isEqual(fieldName, oldFieldNamePointable, stringBinaryComparator)) {
- outRecordBuilder.addField(fieldName, fieldValues.get(i));
- } else {
- outRecordBuilder.addField(newFieldNamePointable, fieldValues.get(i));
- }
+ private void addField(IPointable fieldName, IPointable fieldValue) throws HyracksDataException {
+ int pos = outputRecordTypeInfo.getFieldIndex(fieldName.getByteArray(), fieldName.getStartOffset() + 1,
+ fieldName.getLength() - 1);
+ if (pos >= 0) {
+ outRecordBuilder.addField(pos, fieldValue);
+ } else {
+ outRecordBuilder.addField(fieldName, fieldValue);
}
- outRecordBuilder.write(resultOutput, true);
}
-
- private ARecordVisitablePointable getInputRecordVisitablePointable() throws HyracksDataException {
- inputRecordVisitable.set(inputRecordPointable);
- if (requiresCast) {
- return castToOpenRecord();
- }
- return inputRecordVisitable;
- }
-
- private boolean hasDerivedType(IAType[] types) {
- for (IAType type : types) {
- if (type.getTypeTag().isDerivedType()) {
- return true;
- }
- }
- return false;
- }
-
- private ARecordVisitablePointable castToOpenRecord() throws HyracksDataException {
- inputRecordVisitable.accept(castVisitor, castVisitorArg);
- return openRecordPointable;
- }
-
}
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..63dae91 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
@@ -327,6 +327,7 @@
import org.apache.asterix.runtime.evaluators.functions.AndDescriptor;
import org.apache.asterix.runtime.evaluators.functions.AnyCollectionMemberDescriptor;
import org.apache.asterix.runtime.evaluators.functions.ArrayAppendDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.ArrayBinarySearchDescriptor;
import org.apache.asterix.runtime.evaluators.functions.ArrayConcatDescriptor;
import org.apache.asterix.runtime.evaluators.functions.ArrayContainsDescriptor;
import org.apache.asterix.runtime.evaluators.functions.ArrayDistinctDescriptor;
@@ -335,6 +336,7 @@
import org.apache.asterix.runtime.evaluators.functions.ArrayIfNullDescriptor;
import org.apache.asterix.runtime.evaluators.functions.ArrayInsertDescriptor;
import org.apache.asterix.runtime.evaluators.functions.ArrayIntersectDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.ArrayMoveDescriptor;
import org.apache.asterix.runtime.evaluators.functions.ArrayPositionDescriptor;
import org.apache.asterix.runtime.evaluators.functions.ArrayPrependDescriptor;
import org.apache.asterix.runtime.evaluators.functions.ArrayPutDescriptor;
@@ -349,6 +351,7 @@
import org.apache.asterix.runtime.evaluators.functions.ArraySliceWithoutEndPositionDescriptor;
import org.apache.asterix.runtime.evaluators.functions.ArraySortDescriptor;
import org.apache.asterix.runtime.evaluators.functions.ArrayStarDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.ArraySwapDescriptor;
import org.apache.asterix.runtime.evaluators.functions.ArraySymDiffDescriptor;
import org.apache.asterix.runtime.evaluators.functions.ArraySymDiffnDescriptor;
import org.apache.asterix.runtime.evaluators.functions.ArrayUnionDescriptor;
@@ -446,10 +449,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;
@@ -669,6 +674,9 @@
fc.add(ArraySymDiffnDescriptor.FACTORY);
fc.add(ArrayStarDescriptor.FACTORY);
fc.add(ArrayExceptDescriptor.FACTORY);
+ fc.add(ArrayMoveDescriptor.FACTORY);
+ fc.add(ArraySwapDescriptor.FACTORY);
+ fc.add(ArrayBinarySearchDescriptor.FACTORY);
// unnesting functions
fc.add(TidRunningAggregateDescriptor.FACTORY);
@@ -1169,6 +1177,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 +1324,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..5abebd2 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,16 @@
}
};
+ public static final IFunctionTypeInferer RECORD_MODIFY_INFERER = (expr, fd, context, compilerProps) -> {
+ AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) expr;
+ IAType outType = (IAType) context.getType(expr);
+ IAType inType = (IAType) context.getType(f.getArguments().get(0).getValue());
+ if (inType.getTypeTag().equals(ATypeTag.ANY)) {
+ inType = DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE;
+ }
+ fd.setImmutableStates(outType, inType);
+ };
+
public static final class CastTypeInferer implements IFunctionTypeInferer {
@Override
public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context,
@@ -300,6 +310,8 @@
IAType type1 = (IAType) context.getType(le);
if (type0.getTypeTag().equals(ATypeTag.ANY)) {
type0 = DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE;
+ } else if (type0.getTypeTag().equals(ATypeTag.UNION)) {
+ type0 = ((AUnionType) type0).getActualType();
}
if (type1.getTypeTag().equals(ATypeTag.ANY)) {
type1 = DefaultOpenFieldType.NESTED_OPEN_AORDERED_LIST_TYPE;
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..16135ee
--- /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.getPageReads().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.getPageReads().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..31a40ce
--- /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;
+ private Map<String, IndexStats> indexStats;
+
+ 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;
+
+ @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.getOperatorStats(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/projection/FunctionCallInformation.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/projection/FunctionCallInformation.java
index 5cb26fd..be96358 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/projection/FunctionCallInformation.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/projection/FunctionCallInformation.java
@@ -27,7 +27,6 @@
import java.util.Objects;
import java.util.Set;
-import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.om.exceptions.ExceptionUtil;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -41,16 +40,20 @@
private static final long serialVersionUID = -7884346933746232736L;
private final String functionName;
private final SourceLocation sourceLocation;
+ private final IProjectionFiltrationWarningFactory warningFactory;
private Set<ATypeTag> typeMismatches;
- public FunctionCallInformation(String functionName, SourceLocation sourceLocation) {
- this(functionName, sourceLocation, Collections.emptySet());
+ public FunctionCallInformation(String functionName, SourceLocation sourceLocation,
+ IProjectionFiltrationWarningFactory warningFactory) {
+ this(functionName, sourceLocation, Collections.emptySet(), warningFactory);
}
- private FunctionCallInformation(String functionName, SourceLocation sourceLocation, Set<ATypeTag> typeMismatches) {
+ private FunctionCallInformation(String functionName, SourceLocation sourceLocation, Set<ATypeTag> typeMismatches,
+ IProjectionFiltrationWarningFactory warningFactory) {
this.functionName = functionName;
this.sourceLocation = sourceLocation;
this.typeMismatches = typeMismatches;
+ this.warningFactory = warningFactory;
}
public String getFunctionName() {
@@ -61,22 +64,23 @@
return sourceLocation;
}
- public Warning createTypeMismatchWarning(ATypeTag expectedType, ATypeTag actualType) {
- if (typeMismatches == null) {
+ public Warning createWarning(ATypeTag expectedType, ATypeTag actualType) {
+ if (typeMismatches.isEmpty()) {
typeMismatches = EnumSet.noneOf(ATypeTag.class);
} else if (typeMismatches.contains(actualType)) {
//We already issued a warning containing the same actual type. So, we ignore it
return null;
}
typeMismatches.add(actualType);
- return Warning.of(getSourceLocation(), ErrorCode.TYPE_MISMATCH_FUNCTION, getFunctionName(),
- ExceptionUtil.indexToPosition(0), expectedType, actualType);
+ return warningFactory.createWarning(getSourceLocation(), getFunctionName(), ExceptionUtil.indexToPosition(0),
+ expectedType, actualType);
}
public void writeFields(DataOutput output) throws IOException {
output.writeUTF(functionName);
SourceLocation.writeFields(sourceLocation, output);
output.writeInt(typeMismatches.size());
+ output.writeInt(warningFactory.getErrorCode().intValue());
for (ATypeTag typeTag : typeMismatches) {
output.write(typeTag.serialize());
}
@@ -87,10 +91,12 @@
SourceLocation sourceLocation = SourceLocation.create(in);
int typeMismatchesLength = in.readInt();
Set<ATypeTag> typeMismatches = EnumSet.noneOf(ATypeTag.class);
+ IProjectionFiltrationWarningFactory warningFactory =
+ ProjectionFiltrationWarningFactoryProvider.getWarningFactory(in.readInt());
for (int i = 0; i < typeMismatchesLength; i++) {
typeMismatches.add(ATypeTag.VALUE_TYPE_MAPPING[in.readByte()]);
}
- return new FunctionCallInformation(functionName, sourceLocation, typeMismatches);
+ return new FunctionCallInformation(functionName, sourceLocation, typeMismatches, warningFactory);
}
@Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/projection/IProjectionFiltrationWarningFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/projection/IProjectionFiltrationWarningFactory.java
new file mode 100644
index 0000000..4612f05
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/projection/IProjectionFiltrationWarningFactory.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.asterix.runtime.projection;
+
+import java.io.Serializable;
+
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.api.exceptions.Warning;
+
+public interface IProjectionFiltrationWarningFactory extends Serializable {
+ Warning createWarning(SourceLocation sourceLocation, String functionName, String position, ATypeTag expectedType,
+ ATypeTag actualType);
+
+ ErrorCode getErrorCode();
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/projection/ProjectionFiltrationWarningFactoryProvider.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/projection/ProjectionFiltrationWarningFactoryProvider.java
new file mode 100644
index 0000000..7d548ce
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/projection/ProjectionFiltrationWarningFactoryProvider.java
@@ -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.
+ */
+package org.apache.asterix.runtime.projection;
+
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.om.exceptions.ExceptionUtil;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.api.exceptions.Warning;
+
+public class ProjectionFiltrationWarningFactoryProvider {
+ private ProjectionFiltrationWarningFactoryProvider() {
+ }
+
+ public static IProjectionFiltrationWarningFactory getWarningFactory(int errorCode) {
+ if (errorCode == ErrorCode.TYPE_MISMATCH_FUNCTION.intValue()) {
+ return TYPE_MISMATCH_FACTORY;
+ } else if (errorCode == ErrorCode.INCOMPARABLE_TYPES.intValue()) {
+ return INCOMPARABLE_TYPES_FACTORY;
+ }
+ throw new UnsupportedOperationException("Unsupported error code " + errorCode);
+ }
+
+ public static final IProjectionFiltrationWarningFactory TYPE_MISMATCH_FACTORY =
+ new IProjectionFiltrationWarningFactory() {
+ private static final long serialVersionUID = 4263556611813387010L;
+
+ @Override
+ public Warning createWarning(SourceLocation sourceLocation, String functionName, String position,
+ ATypeTag expectedType, ATypeTag actualType) {
+ return Warning.of(sourceLocation, ErrorCode.TYPE_MISMATCH_FUNCTION, functionName,
+ ExceptionUtil.indexToPosition(0), expectedType, actualType);
+ }
+
+ @Override
+ public ErrorCode getErrorCode() {
+ return ErrorCode.TYPE_MISMATCH_FUNCTION;
+ }
+ };
+
+ public static final IProjectionFiltrationWarningFactory INCOMPARABLE_TYPES_FACTORY =
+ new IProjectionFiltrationWarningFactory() {
+ private static final long serialVersionUID = -7447187099851545763L;
+
+ @Override
+ public Warning createWarning(SourceLocation sourceLocation, String functionName, String position,
+ ATypeTag expectedType, ATypeTag actualType) {
+ return Warning.of(sourceLocation, ErrorCode.INCOMPARABLE_TYPES, expectedType, actualType);
+ }
+
+ @Override
+ public ErrorCode getErrorCode() {
+ return ErrorCode.INCOMPARABLE_TYPES;
+ }
+ };
+}
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/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 052f568..d3c87ff 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
@@ -420,6 +420,7 @@
clusterPartitions.remove(nodePartition.getPartitionId());
}
participantNodes.remove(nodeId);
+ failedNodes.remove(nodeId);
}
}
diff --git a/asterixdb/asterix-server/pom.xml b/asterixdb/asterix-server/pom.xml
index 361ed76..54526df 100644
--- a/asterixdb/asterix-server/pom.xml
+++ b/asterixdb/asterix-server/pom.xml
@@ -220,13 +220,13 @@
<!-- Hadoop AWS SDK -->
<override>
<gavs>
- <gav>com.amazonaws:aws-java-sdk-core:1.12.109</gav>
- <gav>com.amazonaws:jmespath-java:1.12.109</gav>
- <gav>com.amazonaws:aws-java-sdk-s3:1.12.109</gav>
- <gav>com.amazonaws:aws-java-sdk-kms:1.12.109</gav>
- <gav>com.amazonaws:aws-java-sdk-dynamodb:1.12.109</gav>
+ <gav>com.amazonaws:aws-java-sdk-core:1.12.402</gav>
+ <gav>com.amazonaws:jmespath-java:1.12.402</gav>
+ <gav>com.amazonaws:aws-java-sdk-s3:1.12.402</gav>
+ <gav>com.amazonaws:aws-java-sdk-kms:1.12.402</gav>
+ <gav>com.amazonaws:aws-java-sdk-dynamodb:1.12.402</gav>
</gavs>
- <noticeUrl>https://raw.githubusercontent.com/aws/aws-sdk-java/1.12.109/NOTICE.txt</noticeUrl>
+ <noticeUrl>https://raw.githubusercontent.com/aws/aws-sdk-java/1.12.402/NOTICE.txt</noticeUrl>
</override>
<override>
<gav>software.amazon.eventstream:eventstream:1.0.1</gav>
@@ -314,84 +314,122 @@
</override>
<override>
<gavs>
- <gav>com.google.http-client:google-http-client:1.41.0</gav>
- <gav>com.google.http-client:google-http-client-jackson2:1.41.0</gav>
- <gav>com.google.http-client:google-http-client-appengine:1.41.0</gav>
- <gav>com.google.http-client:google-http-client-gson:1.41.0</gav>
- <gav>com.google.http-client:google-http-client-apache-v2:1.41.0</gav>
+ <gav>com.google.http-client:google-http-client:1.42.3</gav>
+ <gav>com.google.http-client:google-http-client-jackson2:1.42.3</gav>
+ <gav>com.google.http-client:google-http-client-appengine:1.42.3</gav>
+ <gav>com.google.http-client:google-http-client-gson:1.42.3</gav>
+ <gav>com.google.http-client:google-http-client-apache-v2:1.42.3</gav>
</gavs>
- <url>https://raw.githubusercontent.com/googleapis/google-http-java-client/v1.41.0/LICENSE</url>
+ <url>https://raw.githubusercontent.com/googleapis/google-http-java-client/v1.42.3/LICENSE</url>
</override>
<override>
- <gav>com.google.oauth-client:google-oauth-client:1.32.1</gav>
- <url>https://raw.githubusercontent.com/googleapis/google-oauth-java-client/v1.32.1/LICENSE</url>
+ <gav>com.google.oauth-client:google-oauth-client:1.34.1</gav>
+ <url>https://raw.githubusercontent.com/googleapis/google-oauth-java-client/v1.34.1/LICENSE</url>
</override>
<override>
<gavs>
- <gav>com.google.protobuf:protobuf-java:3.19.2</gav>
- <gav>com.google.protobuf:protobuf-java-util:3.19.2</gav>
+ <gav>com.google.protobuf:protobuf-java:3.21.12</gav>
+ <gav>com.google.protobuf:protobuf-java-util:3.21.12</gav>
</gavs>
- <url>https://raw.githubusercontent.com/protocolbuffers/protobuf/v3.19.2/LICENSE</url>
+ <url>https://raw.githubusercontent.com/protocolbuffers/protobuf/v3.21.12/LICENSE</url>
</override>
<override>
<gavs>
- <gav>com.google.auth:google-auth-library-oauth2-http:1.3.0</gav>
- <gav>com.google.auth:google-auth-library-credentials:1.3.0</gav>
+ <gav>com.google.auth:google-auth-library-oauth2-http:1.14.0</gav>
+ <gav>com.google.auth:google-auth-library-credentials:1.14.0</gav>
</gavs>
- <url>https://raw.githubusercontent.com/googleapis/google-auth-library-java/v1.3.0/LICENSE</url>
+ <url>https://raw.githubusercontent.com/googleapis/google-auth-library-java/v1.14.0/LICENSE</url>
</override>
<override>
<gavs>
- <gav>com.google.cloud:google-cloud-core:2.3.5</gav>
- <gav>com.google.cloud:google-cloud-core-http:2.3.5</gav>
+ <gav>com.google.cloud:google-cloud-core:2.9.4</gav>
+ <gav>com.google.cloud:google-cloud-core-http:2.9.4</gav>
+ <gav>com.google.cloud:google-cloud-core-grpc:2.9.4</gav>
</gavs>
- <url>https://raw.githubusercontent.com/googleapis/java-core/v2.3.5/LICENSE</url>
+ <url>https://raw.githubusercontent.com/googleapis/java-core/v2.9.4/LICENSE</url>
</override>
<override>
- <gav>com.google.cloud:google-cloud-storage:2.3.0</gav>
- <url>https://raw.githubusercontent.com/googleapis/java-storage/v2.3.0/LICENSE</url>
+ <gav>com.google.cloud:google-cloud-storage:2.17.2</gav>
+ <gav>com.google.api.grpc:gapic-google-cloud-storage-v2:2.17.2-alpha</gav>
+ <gav>com.google.api.grpc:proto-google-cloud-storage-v2:2.17.2-alpha</gav>
+ <gav>com.google.api.grpc:grpc-google-cloud-storage-v2:2.17.2-alpha</gav>
+ <url>https://raw.githubusercontent.com/googleapis/java-storage/v2.17.2/LICENSE</url>
+ </override>
+ <override>
+ <gav>com.google.api.grpc:proto-google-cloud-monitoring-v3:1.64.0</gav>
+ <url>https://raw.githubusercontent.com/googleapis/googleapis/master/LICENSE</url>
</override>
<override>
<gavs>
- <gav>io.opencensus:opencensus-api:0.28.0</gav>
- <gav>io.opencensus:opencensus-contrib-http-util:0.28.0</gav>
+ <gav>io.opencensus:opencensus-proto:0.2.0</gav>
</gavs>
- <url>https://raw.githubusercontent.com/census-instrumentation/opencensus-java/v0.28.0/LICENSE</url>
- </override>
- <override>
- <gav>com.google.api-client:google-api-client:1.33.0</gav>
- <url>https://raw.githubusercontent.com/googleapis/google-api-java-client/v1.33.0/LICENSE</url>
- </override>
- <override>
- <gav>com.google.api.grpc:proto-google-iam-v1:1.2.0</gav>
- <url>https://raw.githubusercontent.com/googleapis/java-iam/v1.2.0/proto-google-iam-v1/LICENSE</url>
- </override>
- <override>
- <gav>com.google.api.grpc:proto-google-common-protos:2.7.1</gav>
- <url>https://raw.githubusercontent.com/googleapis/java-common-protos/v2.7.1/proto-google-common-protos/LICENSE</url>
- </override>
- <override>
- <gav>com.google.api:api-common:2.1.2</gav>
- <url>https://raw.githubusercontent.com/googleapis/api-common-java/v2.1.2/LICENSE</url>
+ <url>https://raw.githubusercontent.com/census-instrumentation/opencensus-proto/v0.2.0/LICENSE</url>
</override>
<override>
<gavs>
- <gav>com.google.api:gax-httpjson:0.93.1</gav>
- <gav>com.google.api:gax:2.8.1</gav>
+ <gav>io.opencensus:opencensus-proto:0.2.0</gav>
</gavs>
- <url>https://raw.githubusercontent.com/googleapis/gax-java/v2.8.1/LICENSE</url>
+ <url>https://raw.githubusercontent.com/census-instrumentation/opencensus-proto/v0.2.0/LICENSE</url>
</override>
<override>
- <gav>com.google.auto.value:auto-value-annotations:1.9</gav>
- <url>https://raw.githubusercontent.com/google/auto/auto-value-1.9/LICENSE</url>
+ <gavs>
+ <gav>io.opencensus:opencensus-api:0.31.1</gav>
+ <gav>io.opencensus:opencensus-contrib-http-util:0.31.1</gav>
+ </gavs>
+ <url>https://raw.githubusercontent.com/census-instrumentation/opencensus-java/v0.31.1/LICENSE</url>
+ </override>
+ <override>
+ <gavs>
+ <gav>io.opencensus:opencensus-contrib-exemplar-util:0.31.0</gav>
+ <gav>io.opencensus:opencensus-contrib-grpc-metrics:0.31.0</gav>
+ <gav>io.opencensus:opencensus-contrib-resource-util:0.31.0</gav>
+ <gav>io.opencensus:opencensus-exporter-metrics-util:0.31.0</gav>
+ <gav>io.opencensus:opencensus-exporter-stats-stackdriver:0.31.0</gav>
+ <gav>io.opencensus:opencensus-impl-core:0.31.0</gav>
+ <gav>io.opencensus:opencensus-impl:0.31.0</gav>
+ </gavs>
+ <url>https://raw.githubusercontent.com/census-instrumentation/opencensus-java/v0.31.0/LICENSE</url>
+ </override>
+ <override>
+ <gav>com.google.api-client:google-api-client:2.1.2</gav>
+ <url>https://raw.githubusercontent.com/googleapis/google-api-java-client/v2.1.2/LICENSE</url>
+ </override>
+ <override>
+ <gav>com.google.api.grpc:proto-google-iam-v1:1.8.0</gav>
+ <gav>com.google.api.grpc:proto-google-common-protos:2.13.0</gav>
+ <url>https://raw.githubusercontent.com/googleapis/gapic-generator-java/v2.13.0/java-common-protos/LICENSE</url>
+ </override>
+ <!-- api-common-java has been moved to a new repo, using the same license, for more info, see:
+ https://github.com/googleapis/gapic-generator-java/tree/v2.13.0/api-common-java
+ -->
+ <override>
+ <gav>com.google.api:api-common:2.5.0</gav>
+ <url>https://raw.githubusercontent.com/googleapis/gapic-generator-java/v2.13.0/api-common-java/LICENSE</url>
+ </override>
+ <!-- gax has been moved to a new repo, using the same license, for more info, see:
+ https://github.com/googleapis/gax-java
+ https://github.com/googleapis/gapic-generator-java/tree/v2.13.0/gax-java
+ -->
+ <override>
+ <gavs>
+ <gav>com.google.api:gax-httpjson:0.107.0</gav>
+ <gav>com.google.api:gax:2.22.0</gav>
+ <gav>com.google.api:gax-grpc:2.22.0</gav>
+ </gavs>
+ <url>https://raw.githubusercontent.com/googleapis/gapic-generator-java/v2.13.0/gax-java/LICENSE</url>
+ </override>
+ <override>
+ <gav>com.google.auto.value:auto-value:1.10.1</gav>
+ <gav>com.google.auto.value:auto-value-annotations:1.10.1</gav>
+ <url>https://raw.githubusercontent.com/google/auto/auto-value-1.10.1/LICENSE</url>
</override>
<override>
<gav>com.google.code.findbugs:jsr305:3.0.2</gav>
<url>https://raw.githubusercontent.com/findbugsproject/findbugs/3.0.2_preview2/findbugs/licenses/LICENSE-jsr305.txt</url>
</override>
<override>
- <gav>com.google.code.gson:gson:2.8.9</gav>
- <url>https://raw.githubusercontent.com/google/gson/gson-parent-2.8.9/LICENSE</url>
+ <gav>com.google.code.gson:gson:2.10.1</gav>
+ <url>https://raw.githubusercontent.com/google/gson/gson-parent-2.10.1/LICENSE</url>
</override>
<override>
<gav>com.google.guava:listenablefuture:9999.0-empty-to-avoid-conflict-with-guava</gav>
@@ -402,11 +440,41 @@
<url>https://raw.githubusercontent.com/google/j2objc/1.3/LICENSE</url>
</override>
<override>
- <gav>io.grpc:grpc-context:1.43.2</gav>
+ <gavs>
+ <gav>io.grpc:grpc-alts:1.43.2</gav>
+ <gav>io.grpc:grpc-api:1.43.2</gav>
+ <gav>io.grpc:grpc-auth:1.43.2</gav>
+ <gav>io.grpc:grpc-census:1.43.2</gav>
+ <gav>io.grpc:grpc-core:1.43.2</gav>
+ <gav>io.grpc:grpc-grpclb:1.43.2</gav>
+ <gav>io.grpc:grpc-protobuf-lite:1.43.2</gav>
+ <gav>io.grpc:grpc-protobuf:1.43.2</gav>
+ <gav>io.grpc:grpc-services:1.43.2</gav>
+ <gav>io.grpc:grpc-stub:1.43.2</gav>
+ <gav>io.grpc:grpc-xds:1.43.2</gav>
+ </gavs>
<url>https://raw.githubusercontent.com/grpc/grpc-java/v1.43.2/LICENSE</url>
<noticeUrl>https://raw.githubusercontent.com/grpc/grpc-java/v1.43.2/NOTICE.txt</noticeUrl>
</override>
<override>
+ <gavs>
+ <gav>io.grpc:grpc-alts:1.52.1</gav>
+ <gav>io.grpc:grpc-api:1.52.1</gav>
+ <gav>io.grpc:grpc-auth:1.52.1</gav>
+ <gav>io.grpc:grpc-context:1.52.1</gav>
+ <gav>io.grpc:grpc-core:1.52.1</gav>
+ <gav>io.grpc:grpc-grpclb:1.52.1</gav>
+ <gav>io.grpc:grpc-googleapis:1.52.1</gav>
+ <gav>io.grpc:grpc-protobuf-lite:1.52.1</gav>
+ <gav>io.grpc:grpc-protobuf:1.52.1</gav>
+ <gav>io.grpc:grpc-services:1.52.1</gav>
+ <gav>io.grpc:grpc-stub:1.52.1</gav>
+ <gav>io.grpc:grpc-xds:1.52.1</gav>
+ </gavs>
+ <url>https://raw.githubusercontent.com/grpc/grpc-java/v1.52.1/LICENSE</url>
+ <noticeUrl>https://raw.githubusercontent.com/grpc/grpc-java/v1.52.1/NOTICE.txt</noticeUrl>
+ </override>
+ <override>
<gav>org.mindrot:jbcrypt:0.4</gav>
<url>http://www.mindrot.org/files/jBCrypt/LICENSE</url>
</override>
@@ -415,8 +483,84 @@
<noticeUrl>https://raw.githubusercontent.com/msgpack/msgpack-java/0.8.20/NOTICE</noticeUrl>
</override>
<override>
- <gav>com.github.luben:zstd-jni:1.4.9-1</gav>
- <url>https://raw.githubusercontent.com/luben/zstd-jni/v1.4.9-1/LICENSE</url>
+ <gav>com.github.luben:zstd-jni:1.5.0-1</gav>
+ <url>https://raw.githubusercontent.com/luben/zstd-jni/v1.5.0-1/LICENSE</url>
+ </override>
+ <override>
+ <gav>org.slf4j:slf4j-reload4j:1.7.36</gav>
+ <url>https://raw.githubusercontent.com/qos-ch/slf4j/v_1.7.36/LICENSE.txt</url>
+ </override>
+ <override>
+ <gav>org.conscrypt:conscrypt-openjdk-uber:2.5.1</gav>
+ <url>https://raw.githubusercontent.com/google/conscrypt/2.5.1/LICENSE</url>
+ <noticeUrl>https://raw.githubusercontent.com/google/conscrypt/2.5.1/NOTICE</noticeUrl>
+ </override>
+ <override>
+ <gav>io.perfmark:perfmark-api:0.26.0</gav>
+ <url>https://raw.githubusercontent.com/perfmark/perfmark/v0.26.0/LICENSE</url>
+ <noticeUrl>https://raw.githubusercontent.com/perfmark/perfmark/v0.26.0/NOTICE</noticeUrl>
+ </override>
+ <override>
+ <gavs>
+ <gav>com.google.cloud.bigdataoss:gcs-connector:hadoop3-2.2.6</gav>
+ <gav>com.google.cloud.bigdataoss:gcsio:2.2.6</gav>
+ <gav>com.google.cloud.bigdataoss:util-hadoop:hadoop3-2.2.6</gav>
+ <gav>com.google.cloud.bigdataoss:util:2.2.6</gav>
+ </gavs>
+ <url>https://raw.githubusercontent.com/GoogleCloudDataproc/hadoop-connectors/v2.2.6/LICENSE</url>
+ </override>
+ <override>
+ <gavs>
+ <gav>com.google.flogger:flogger:0.7.1</gav>
+ <gav>com.google.flogger:google-extensions:0.7.1</gav>
+ <gav>com.google.flogger:flogger-system-backend:0.7.1</gav>
+ </gavs>
+ <url>https://raw.githubusercontent.com/google/flogger/flogger-0.7.1/LICENSE</url>
+ </override>
+ <override>
+ <gavs>
+ <gav>com.google.api.grpc:proto-google-cloud-monitoring-v3:1.64.0</gav>
+ <gav>com.google.api.grpc:proto-google-cloud-storage-v2:2.2.2-alpha</gav>
+ <gav>com.google.api.grpc:grpc-google-cloud-storage-v2:2.2.2-alpha</gav>
+ </gavs>
+ <url>https://raw.githubusercontent.com/googleapis/googleapis/master/LICENSE</url>
+ </override>
+ <override>
+ <gav>com.lmax:disruptor:3.4.2</gav>
+ <url>https://raw.githubusercontent.com/LMAX-Exchange/disruptor/3.4.2/LICENCE.txt</url>
+ </override>
+ <override>
+ <gav>com.google.cloud:google-cloud-core-grpc:1.82.0</gav>
+ <url>https://raw.githubusercontent.com/googleapis/java-core/v1.82.0/LICENSE</url>
+ </override>
+ <override>
+ <gav>com.google.cloud:google-cloud-monitoring:1.82.0</gav>
+ <url>https://raw.githubusercontent.com/googleapis/java-monitoring/1.82.0/LICENSE</url>
+ </override>
+ <override>
+ <gav>com.google.api-client:google-api-client-jackson2:1.32.2</gav>
+ <url>https://raw.githubusercontent.com/googleapis/google-api-java-client/v1.32.2/LICENSE</url>
+ </override>
+ <override>
+ <gav>org.conscrypt:conscrypt-openjdk-uber:2.5.1</gav>
+ <url>https://raw.githubusercontent.com/google/conscrypt/2.5.1/LICENSE</url>
+ <noticeUrl>https://raw.githubusercontent.com/google/conscrypt/2.5.1/NOTICE</noticeUrl>
+ </override>
+ <override>
+ <gav>io.perfmark:perfmark-api:0.23.0</gav>
+ <url>https://raw.githubusercontent.com/perfmark/perfmark/v0.23.0/LICENSE</url>
+ <noticeUrl>https://raw.githubusercontent.com/perfmark/perfmark/v0.23.0/NOTICE</noticeUrl>
+ </override>
+ <override>
+ <gav>com.google.api:gax-grpc:2.7.1</gav>
+ <url>https://raw.githubusercontent.com/googleapis/gax-java/v2.7.1/LICENSE</url>
+ </override>
+ <override>
+ <gavs>
+ <gav>org.bouncycastle:bcprov-jdk15on:1.60</gav>
+ <gav>org.bouncycastle:bcpkix-jdk15on:1.60</gav>
+ </gavs>
+ <url>https://raw.githubusercontent.com/bcgit/bc-java/r1rv60/LICENSE.html</url>
</override>
</overrides>
<licenses>
@@ -460,12 +604,37 @@
<aliasUrl>https://raw.githubusercontent.com/reactor/reactor-core/v3.4.10/LICENSE</aliasUrl>
<aliasUrl>https://raw.githubusercontent.com/codehaus/stax/master/dev/ASF2.0.txt</aliasUrl>
<aliasUrl>https://bitbucket.org/connect2id/oauth-2.0-sdk-with-openid-connect-extensions/raw/5d13925b57ace092ea5e1131c338f464d85545f4/LICENSE.txt</aliasUrl>
- <aliasUrl>https://raw.githubusercontent.com/google/auto/auto-value-1.9/LICENSE</aliasUrl>
+ <aliasUrl>https://raw.githubusercontent.com/google/auto/auto-value-1.10.1/LICENSE</aliasUrl>
<aliasUrl>https://raw.githubusercontent.com/google/j2objc/1.3/LICENSE</aliasUrl>
- <aliasUrl>https://raw.githubusercontent.com/googleapis/java-common-protos/v2.7.1/proto-google-common-protos/LICENSE</aliasUrl>
- <aliasUrl>https://raw.githubusercontent.com/googleapis/java-iam/v1.2.0/proto-google-iam-v1/LICENSE</aliasUrl>
- <aliasUrl>https://raw.githubusercontent.com/googleapis/java-storage/v2.3.0/LICENSE</aliasUrl>
+ <aliasUrl>https://raw.githubusercontent.com/googleapis/java-storage/v2.17.2/LICENSE</aliasUrl>
<aliasUrl>http://repository.jboss.org/licenses/apache-2.0.txt</aliasUrl>
+ <aliasUrl>https://raw.githubusercontent.com/census-instrumentation/opencensus-java/v0.31.0/LICENSE</aliasUrl>
+ <aliasUrl>https://raw.githubusercontent.com/census-instrumentation/opencensus-java/v0.31.1/LICENSE</aliasUrl>
+ <aliasUrl>https://raw.githubusercontent.com/census-instrumentation/opencensus-proto/v0.2.0/LICENSE</aliasUrl>
+ <aliasUrl>https://raw.githubusercontent.com/google/conscrypt/2.5.2/LICENSE</aliasUrl>
+ <aliasUrl>https://raw.githubusercontent.com/googleapis/googleapis/master/LICENSE</aliasUrl>
+ <aliasUrl>https://raw.githubusercontent.com/perfmark/perfmark/v0.26.0/LICENSE</aliasUrl>
+ <aliasUrl>https://raw.githubusercontent.com/googleapis/google-http-java-client/v1.42.3/LICENSE</aliasUrl>
+ <aliasUrl>https://raw.githubusercontent.com/google/conscrypt/2.5.1/LICENSE</aliasUrl>
+ <aliasUrl>https://raw.githubusercontent.com/google/flogger/flogger-0.7.1/LICENSE</aliasUrl>
+ <aliasUrl>https://raw.githubusercontent.com/googleapis/google-api-java-client/v1.32.2/LICENSE</aliasUrl>
+ <aliasUrl>https://raw.githubusercontent.com/googleapis/google-api-java-client/v1.35.1/LICENSE</aliasUrl>
+ <aliasUrl>https://raw.githubusercontent.com/googleapis/java-core/v1.82.0/LICENSE</aliasUrl>
+ <aliasUrl>https://raw.githubusercontent.com/googleapis/java-monitoring/1.82.0/LICENSE</aliasUrl>
+ <aliasUrl>https://raw.githubusercontent.com/GoogleCloudDataproc/hadoop-connectors/v2.2.6/LICENSE</aliasUrl>
+ <aliasUrl>https://raw.githubusercontent.com/LMAX-Exchange/disruptor/3.4.2/LICENCE.txt</aliasUrl>
+ <aliasUrl>https://raw.githubusercontent.com/perfmark/perfmark/v0.23.0/LICENSE</aliasUrl>
+ <aliasUrl>https://raw.githubusercontent.com/grpc/grpc-java/v1.43.2/LICENSE</aliasUrl>
+ <aliasUrl>https://raw.githubusercontent.com/grpc/grpc-java/v1.47.0/LICENSE</aliasUrl>
+ <aliasUrl>https://raw.githubusercontent.com/googleapis/google-http-java-client/v1.42.0/LICENSE</aliasUrl>
+ <aliasUrl>https://raw.githubusercontent.com/googleapis/google-oauth-java-client/v1.34.1/LICENSE</aliasUrl>
+ <aliasUrl>https://raw.githubusercontent.com/googleapis/java-core/v2.9.4/LICENSE</aliasUrl>
+ <aliasUrl>https://raw.githubusercontent.com/google/gson/gson-parent-2.10.1/LICENSE</aliasUrl>
+ <aliasUrl>https://raw.githubusercontent.com/googleapis/gapic-generator-java/v2.13.0/java-common-protos/LICENSE</aliasUrl>
+ <aliasUrl>https://raw.githubusercontent.com/googleapis/google-api-java-client/v2.1.2/LICENSE</aliasUrl>
+ <aliasUrl>https://raw.githubusercontent.com/grpc/grpc-java/v1.52.1/LICENSE</aliasUrl>
+ <aliasUrl>https://raw.githubusercontent.com/googleapis/java-core/v2.8.0/LICENSE</aliasUrl>
+ <aliasUrl>https://raw.githubusercontent.com/google/gson/gson-parent-2.9.0/LICENSE</aliasUrl>
</aliasUrls>
<metric>1</metric>
</license>
@@ -537,15 +706,6 @@
<aliasUrls>http://creativecommons.org/publicdomain/zero/1.0/</aliasUrls>
</license>
<license>
- <displayName>The 2-Clause BSD License</displayName>
- <url>https://opensource.org/licenses/BSD-2-Clause</url>
- <contentFile>raw.githubusercontent.com_luben_zstd-jni_v1.4.9-1_LICENSE.txt</contentFile>
- <aliasUrls>
- <aliasURL>https://opensource.org/licenses/bsd-license.php</aliasURL>
- <aliasUrl>https://raw.githubusercontent.com/luben/zstd-jni/v1.4.9-1/LICENSE</aliasUrl>
- </aliasUrls>
- </license>
- <license>
<displayName>The Go license</displayName>
<url>http://golang.org/LICENSE</url>
<aliasUrls>
@@ -862,7 +1022,7 @@
<plugin>
<artifactId>jdeb</artifactId>
<groupId>org.vafer</groupId>
- <version>1.5</version>
+ <version>1.8</version>
<executions>
<execution>
<phase>package</phase>
@@ -872,26 +1032,36 @@
<configuration>
<dataSet>
<data>
- <src>${project.build.directory}/${project.build.finalName}-binary-assembly/apache-asterixdb-${project.version}/</src>
- <excludes>bin/**</excludes>
+ <src>${project.build.directory}/${project.build.finalName}-binary-assembly/apache-asterixdb-${project.version}</src>
<type>directory</type>
<mapper>
<type>perm</type>
- <prefix>/opt/apache-asterixdb-${project.version}/</prefix>
- <user>asterixdb</user>
- <group>asterixdb</group>
+ <prefix>/opt/apache-asterixdb/</prefix>
+ <user>root</user>
+ <group>root</group>
+ <filemode>755</filemode>
+ </mapper>
+ </data>
+ <data>
+ <type>file</type>
+ <src>src/deb/systemd/cc.conf</src>
+ <mapper>
+ <prefix>/opt/apache-asterixdb/</prefix>
+ <type>perm</type>
+ <user>root</user>
+ <group>root</group>
<filemode>644</filemode>
</mapper>
</data>
<data>
- <src>${project.build.directory}/${project.build.finalName}-binary-assembly/apache-asterixdb-${project.version}/bin</src>
- <type>directory</type>
+ <type>file</type>
+ <src>src/deb/udf_listener.py</src>
<mapper>
+ <prefix>/opt/apache-asterixdb/bin</prefix>
<type>perm</type>
- <prefix>/opt/apache-asterixdb-${project.version}/bin</prefix>
- <user>asterixdb</user>
- <group>asterixdb</group>
- <filemode>754</filemode>
+ <user>root</user>
+ <group>root</group>
+ <filemode>555</filemode>
</mapper>
</data>
<data>
@@ -914,6 +1084,39 @@
<group>root</group>
</mapper>
</data>
+ <data>
+ <type>file</type>
+ <src>src/deb/systemd/pyudf.socket</src>
+ <mapper>
+ <prefix>/lib/systemd/system</prefix>
+ <type>perm</type>
+ <user>root</user>
+ <group>root</group>
+ </mapper>
+ </data>
+ <data>
+ <type>file</type>
+ <src>src/deb/systemd/pyudf@.service</src>
+ <mapper>
+ <prefix>/lib/systemd/system</prefix>
+ <type>perm</type>
+ <user>root</user>
+ <group>root</group>
+ </mapper>
+ </data>
+ <data>
+ <type>template</type>
+ <paths>
+ <path>/opt/apache-asterixdb/logs</path>
+ <path>/opt/apache-asterixdb/data</path>
+ </paths>
+ <mapper>
+ <type>perm</type>
+ <user>asterixdb</user>
+ <group>asterixdb</group>
+ <filemode>750</filemode>
+ </mapper>
+ </data>
</dataSet>
</configuration>
</execution>
diff --git a/asterixdb/asterix-server/src/deb/control/control b/asterixdb/asterix-server/src/deb/control/control
index 1f6c213..77bbd1d 100644
--- a/asterixdb/asterix-server/src/deb/control/control
+++ b/asterixdb/asterix-server/src/deb/control/control
@@ -17,8 +17,7 @@
Section: databases
Priority: extra
Architecture: all
-Depends: jdk (>= 1.8)
+Depends: java17-runtime-headless
Maintainer: Ian Maxon <ian@maxons.email>
Description: Apache AsterixDB - a scalable, open source Big Data Management System (BDMS)
-Distribution: development
-Depends: default-jre | java8-runtime
+Distribution: development
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/deb/control/postinst b/asterixdb/asterix-server/src/deb/control/postinst
index 896ca28..fe5c912 100644
--- a/asterixdb/asterix-server/src/deb/control/postinst
+++ b/asterixdb/asterix-server/src/deb/control/postinst
@@ -13,5 +13,4 @@
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
-adduser --system --group --quiet --home /opt/apache-asterixdb/ \
---no-create-home --disabled-login --force-badname asterixdb
+chmod -R 755 /opt/apache-asterixdb/
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/deb/control/preinst b/asterixdb/asterix-server/src/deb/control/preinst
index 4509c90..8d14847 100644
--- a/asterixdb/asterix-server/src/deb/control/preinst
+++ b/asterixdb/asterix-server/src/deb/control/preinst
@@ -13,3 +13,7 @@
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
+adduser --system --group --quiet --home /opt/apache-asterixdb/ \
+--no-create-home --disabled-login --force-badname asterixdb
+adduser --system --group --quiet --home /opt/apache-asterixdb/ \
+--no-create-home --disabled-login --force-badname asterixdb-udf
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/deb/systemd/asterix-cc.service b/asterixdb/asterix-server/src/deb/systemd/asterix-cc.service
index 9711fba..2a52e2d 100644
--- a/asterixdb/asterix-server/src/deb/systemd/asterix-cc.service
+++ b/asterixdb/asterix-server/src/deb/systemd/asterix-cc.service
@@ -19,8 +19,9 @@
[Service]
Type=simple
User=asterixdb
-ExecStart=/opt/apache-asterixdb/bin/asterixcc --config-file /opt/apache-asterixdb/cc.conf
+ExecStart=/opt/apache-asterixdb/bin/asterixcc -config-file "/opt/apache-asterixdb/cc.conf"
Restart=on-abort
+WorkingDirectory=/opt/apache-asterixdb
[Install]
WantedBy=multi-user.target
diff --git a/asterixdb/asterix-server/src/deb/systemd/asterix-nc.service b/asterixdb/asterix-server/src/deb/systemd/asterix-nc.service
index bfe6296..e09d8e8 100644
--- a/asterixdb/asterix-server/src/deb/systemd/asterix-nc.service
+++ b/asterixdb/asterix-server/src/deb/systemd/asterix-nc.service
@@ -21,6 +21,7 @@
User=asterixdb
ExecStart=/opt/apache-asterixdb/bin/asterixncservice
Restart=on-abort
+WorkingDirectory=/opt/apache-asterixdb
[Install]
WantedBy=multi-user.target
diff --git a/asterixdb/asterix-server/src/deb/systemd/cc.conf b/asterixdb/asterix-server/src/deb/systemd/cc.conf
new file mode 100644
index 0000000..0af967a
--- /dev/null
+++ b/asterixdb/asterix-server/src/deb/systemd/cc.conf
@@ -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.
+
+[nc/asterix_nc1]
+txn.log.dir=/opt/apache-asterixdb/data/txnlog
+core.dump.dir=/opt/apache-asterixdb/logs/coredump
+iodevices=/opt/apache-asterixdb/data/
+nc.api.port=19004
+
+[nc]
+address=127.0.0.1
+command=asterixnc
+
+[cc]
+address = 127.0.0.1
+
+[common]
+log.level = INFO
+log.dir = /opt/apache-asterixdb/logs/
diff --git a/asterixdb/asterix-server/src/deb/systemd/pyudf.socket b/asterixdb/asterix-server/src/deb/systemd/pyudf.socket
new file mode 100644
index 0000000..4e731db
--- /dev/null
+++ b/asterixdb/asterix-server/src/deb/systemd/pyudf.socket
@@ -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.
+[Unit]
+Description=AsterixDB UDF Domain Socket
+PartOf=asterixdb_udf.service
+
+[Socket]
+ListenStream=/tmp/pyudf.socket
+SocketMode=0660
+SocketUser=asterixdb-udf
+SocketGroup=asterixdb
+Accept=true
+DeferAcceptSec=1
+
+[Install]
+WantedBy=sockets.target
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/deb/systemd/pyudf@.service b/asterixdb/asterix-server/src/deb/systemd/pyudf@.service
new file mode 100644
index 0000000..9856142
--- /dev/null
+++ b/asterixdb/asterix-server/src/deb/systemd/pyudf@.service
@@ -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.
+[Unit]
+Description=AsterixDB UDF Executor Service
+After=network.target pyudf.socket
+Requires=pyudf.socket
+
+[Service]
+User=asterixdb-udf
+Type=simple
+ExecStart=/usr/bin/python3 /opt/apache-asterixdb/bin/udf_listener.py
+TimeoutStopSec=5
+StandardError=journal
+StandardError=journal
+
+[Install]
+WantedBy=default.target
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/deb/udf_listener.py b/asterixdb/asterix-server/src/deb/udf_listener.py
new file mode 100644
index 0000000..03874b2
--- /dev/null
+++ b/asterixdb/asterix-server/src/deb/udf_listener.py
@@ -0,0 +1,283 @@
+#!/usr/bin/env python3
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import sys
+from systemd.daemon import listen_fds
+from os import chdir
+from os import getcwd
+from os import getpid
+from struct import *
+import signal
+import msgpack
+import socket
+import traceback
+from importlib import import_module
+from pathlib import Path
+from enum import IntEnum
+from io import BytesIO
+
+
+PROTO_VERSION = 1
+HEADER_SZ = 8 + 8 + 1
+REAL_HEADER_SZ = 4 + 8 + 8 + 1
+FRAMESZ = 32768
+
+
+class MessageType(IntEnum):
+ HELO = 0
+ QUIT = 1
+ INIT = 2
+ INIT_RSP = 3
+ CALL = 4
+ CALL_RSP = 5
+ ERROR = 6
+
+
+class MessageFlags(IntEnum):
+ NORMAL = 0
+ INITIAL_REQ = 1
+ INITIAL_ACK = 2
+ ERROR = 3
+
+
+class Wrapper(object):
+ wrapped_module = None
+ wrapped_class = None
+ wrapped_fn = None
+ sz = None
+ mid = None
+ rmid = None
+ flag = None
+ resp = None
+ unpacked_msg = None
+ msg_type = None
+ packer = msgpack.Packer(autoreset=False, use_bin_type=False)
+ unpacker = msgpack.Unpacker(raw=False)
+ response_buf = BytesIO()
+ stdin_buf = BytesIO()
+ wrapped_fns = {}
+ alive = True
+ readbuf = bytearray(FRAMESZ)
+ readview = memoryview(readbuf)
+
+
+ def init(self, module_name, class_name, fn_name):
+ self.wrapped_module = import_module(module_name)
+ # do not allow modules to be called that are not part of the uploaded module
+ wrapped_fn = None
+ if not self.check_module_path(self.wrapped_module):
+ self.wrapped_module = None
+ raise ImportError("Module was not found in library")
+ if class_name is not None:
+ self.wrapped_class = getattr(
+ import_module(module_name), class_name)()
+ if self.wrapped_class is not None:
+ wrapped_fn = getattr(self.wrapped_class, fn_name)
+ else:
+ wrapped_fn = getattr(import_module(module_name), fn_name)
+ if wrapped_fn is None:
+ raise ImportError(
+ "Could not find class or function in specified module")
+ self.wrapped_fns[self.mid] = wrapped_fn
+
+ def next_tuple(self, *args, key=None):
+ return self.wrapped_fns[key](*args)
+
+ def check_module_path(self, module):
+ cwd = Path('.').resolve()
+ module_path = Path(module.__file__).resolve()
+ return cwd in module_path.parents
+ return True
+
+ def read_header(self, readbuf):
+ self.sz, self.mid, self.rmid, self.flag = unpack(
+ "!iqqb", readbuf[0:REAL_HEADER_SZ])
+ return True
+
+ def write_header(self, response_buf, dlen):
+ total_len = dlen + HEADER_SZ
+ header = pack("!iqqb", total_len, int(-1), int(self.rmid), self.flag)
+ self.response_buf.write(header)
+ return total_len + 4
+
+ def get_ver_hlen(self, hlen):
+ return hlen + (PROTO_VERSION << 4)
+
+ def get_hlen(self):
+ return self.ver_hlen - (PROTO_VERSION << 4)
+
+ def init_remote_ipc(self):
+ self.response_buf.seek(0)
+ self.flag = MessageFlags.INITIAL_REQ
+ dlen = len(self.unpacked_msg[1])
+ resp_len = self.write_header(self.response_buf, dlen)
+ self.response_buf.write(self.unpacked_msg[1])
+ self.resp = self.response_buf.getbuffer()[0:resp_len]
+ self.send_msg()
+ self.packer.reset()
+
+ def cd(self, basedir):
+ chdir(basedir + "/site-packages")
+ sys.path.insert(0,getcwd())
+
+ def helo(self):
+ # need to ack the connection back before sending actual HELO
+ # self.init_remote_ipc()
+ self.cd(self.unpacked_msg[1][1])
+ self.flag = MessageFlags.NORMAL
+ self.response_buf.seek(0)
+ self.packer.pack(int(MessageType.HELO))
+ self.packer.pack(int(getpid()))
+ dlen = len(self.packer.bytes()) # tag(1) + body(4)
+ resp_len = self.write_header(self.response_buf, dlen)
+ self.response_buf.write(self.packer.bytes())
+ self.resp = self.response_buf.getbuffer()[0:resp_len]
+ self.send_msg()
+ self.packer.reset()
+ return True
+
+ def handle_init(self):
+ self.flag = MessageFlags.NORMAL
+ self.response_buf.seek(0)
+ args = self.unpacked_msg[1]
+ module = args[0]
+ if len(args) == 3:
+ clazz = args[1]
+ fn = args[2]
+ else:
+ clazz = None
+ fn = args[1]
+ self.init(module, clazz, fn)
+ self.packer.pack(int(MessageType.INIT_RSP))
+ dlen = 1 # just the tag.
+ resp_len = self.write_header(self.response_buf, dlen)
+ self.response_buf.write(self.packer.bytes())
+ self.resp = self.response_buf.getbuffer()[0:resp_len]
+ self.send_msg()
+ self.packer.reset()
+ return True
+
+ def quit(self):
+ self.alive = False
+ return True
+
+ def handle_call(self):
+ self.flag = MessageFlags.NORMAL
+ result = ([], [])
+ if len(self.unpacked_msg) > 1:
+ args = self.unpacked_msg[1]
+ if args is not None:
+ for arg in args:
+ try:
+ result[0].append(self.next_tuple(*arg, key=self.mid))
+ except BaseException as e:
+ result[1].append(traceback.format_exc())
+ self.packer.reset()
+ self.response_buf.seek(0)
+ body = msgpack.packb(result)
+ dlen = len(body) + 1 # 1 for tag
+ resp_len = self.write_header(self.response_buf, dlen)
+ self.packer.pack(int(MessageType.CALL_RSP))
+ self.response_buf.write(self.packer.bytes())
+ self.response_buf.write(body)
+ self.resp = self.response_buf.getbuffer()[0:resp_len]
+ self.send_msg()
+ self.packer.reset()
+ return True
+
+ def handle_error(self, e):
+ self.flag = MessageFlags.NORMAL
+ self.packer.reset()
+ self.response_buf.seek(0)
+ body = msgpack.packb(str(e))
+ dlen = len(body) + 1 # 1 for tag
+ resp_len = self.write_header(self.response_buf, dlen)
+ self.packer.pack(int(MessageType.ERROR))
+ self.response_buf.write(self.packer.bytes())
+ self.response_buf.write(body)
+ self.resp = self.response_buf.getbuffer()[0:resp_len]
+ self.send_msg()
+ self.packer.reset()
+ self.alive = False
+ return True
+
+ type_handler = {
+ MessageType.HELO: helo,
+ MessageType.QUIT: quit,
+ MessageType.INIT: handle_init,
+ MessageType.CALL: handle_call
+ }
+
+ def connect_sock(self):
+ self.sock = socket.fromfd(listen_fds()[0], socket.AF_UNIX, socket.SOCK_STREAM)
+
+ def disconnect_sock(self, *args):
+ self.sock.shutdown(socket.SHUT_RDWR)
+ self.sock.close()
+
+ def recv_msg(self):
+ while self.alive:
+ pos = self.sock.recv_into(self.readbuf)
+ if pos <= 0:
+ self.alive = False
+ return
+ try:
+ while pos < REAL_HEADER_SZ:
+ read = self.sock.recv_into(self.readview[pos:])
+ if read <= 0:
+ self.alive = False
+ return
+ pos += read
+ self.read_header(self.readview)
+ while pos < self.sz and len(self.readbuf) - pos > 0:
+ read = self.sock.recv_into(self.readview[pos:])
+ if read <= 0:
+ self.alive = False
+ return
+ pos += read
+ while pos < self.sz:
+ vszchunk = self.sock.recv(4096)
+ if len(vszchunk) == 0:
+ self.alive = False
+ return
+ self.readview.release()
+ self.readbuf.extend(vszchunk)
+ self.readview = memoryview(self.readbuf)
+ pos += len(vszchunk)
+ self.unpacker.feed(self.readview[REAL_HEADER_SZ:self.sz])
+ self.unpacked_msg = list(self.unpacker)
+ self.msg_type = MessageType(self.unpacked_msg[0])
+ self.type_handler[self.msg_type](self)
+ except BaseException as e:
+ self.handle_error(''.join(traceback.format_exc()))
+
+ def send_msg(self):
+ self.sock.sendall(self.resp)
+ self.resp = None
+ return
+
+ def recv_loop(self):
+ while self.alive:
+ self.recv_msg()
+ self.disconnect_sock()
+
+
+wrap = Wrapper()
+wrap.connect_sock()
+signal.signal(signal.SIGTERM, wrap.disconnect_sock)
+wrap.recv_loop()
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/pom.xml b/asterixdb/pom.xml
index f550b6a..1c9db43 100644
--- a/asterixdb/pom.xml
+++ b/asterixdb/pom.xml
@@ -84,16 +84,17 @@
<!-- Versions under dependencymanagement or used in many projects via properties -->
<algebricks.version>0.3.9-SNAPSHOT</algebricks.version>
<hyracks.version>0.3.9-SNAPSHOT</hyracks.version>
- <hadoop.version>3.3.2</hadoop.version>
+ <hadoop.version>3.3.4</hadoop.version>
<jacoco.version>0.7.6.201602180812</jacoco.version>
- <log4j.version>2.17.1</log4j.version>
- <awsjavasdk.version>2.17.116</awsjavasdk.version>
- <parquet.version>1.12.0</parquet.version>
- <hadoop-awsjavasdk.version>1.12.109</hadoop-awsjavasdk.version>
+ <log4j.version>2.19.0</log4j.version>
+ <awsjavasdk.version>2.17.218</awsjavasdk.version>
+ <parquet.version>1.12.3</parquet.version>
+ <hadoop-awsjavasdk.version>1.12.402</hadoop-awsjavasdk.version>
<azureblobjavasdk.version>12.14.2</azureblobjavasdk.version>
<azuredatalakejavasdk.version>12.7.2</azuredatalakejavasdk.version>
- <gcsjavasdk.version>2.3.0</gcsjavasdk.version>
+ <gcsjavasdk.version>2.17.2</gcsjavasdk.version>
<hadoop-azuresdk.version>8.6.6</hadoop-azuresdk.version>
+ <hadoop-gcs.version>hadoop3-2.2.6</hadoop-gcs.version>
<implementation.title>Apache AsterixDB - ${project.name}</implementation.title>
<implementation.url>https://asterixdb.apache.org/</implementation.url>
@@ -930,7 +931,7 @@
<module>asterix-test-framework</module>
<module>asterix-maven-plugins</module>
<module>asterix-server</module>
- <module>asterix-docker</module>
+ <module>asterix-podman</module>
<module>asterix-doc</module>
<module>asterix-fuzzyjoin</module>
<module>asterix-replication</module>
@@ -941,6 +942,7 @@
<module>asterix-license</module>
<module>asterix-geo</module>
<module>asterix-spidersilk</module>
+ <module>asterix-column</module>
</modules>
<dependencyManagement>
@@ -973,16 +975,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>
@@ -992,16 +990,21 @@
<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>
+ <groupId>ch.qos.reload4j</groupId>
+ <artifactId>reload4j</artifactId>
+ </exclusion>
+ <!-- TODO(htowaileb): Remove after updating to hadoop 3.3.4 -->
+ <exclusion>
+ <groupId>org.eclipse.jetty</groupId>
+ <artifactId>jetty-util</artifactId>
</exclusion>
<exclusion>
- <groupId>org.slf4j</groupId>
- <artifactId>slf4j-log4j12</artifactId>
+ <groupId>org.eclipse.jetty</groupId>
+ <artifactId>jetty-util-ajax</artifactId>
</exclusion>
</exclusions>
</dependency>
@@ -1035,16 +1038,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>
@@ -1066,6 +1065,15 @@
<groupId>javax.ws.rs</groupId>
<artifactId>jsr311-api</artifactId>
</exclusion>
+ <!-- TODO(htowaileb): Remove after updating to hadoop 3.3.4 -->
+ <exclusion>
+ <groupId>org.eclipse.jetty</groupId>
+ <artifactId>jetty-util</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.eclipse.jetty</groupId>
+ <artifactId>jetty-util-ajax</artifactId>
+ </exclusion>
</exclusions>
</dependency>
<dependency>
@@ -1074,16 +1082,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>
@@ -1098,16 +1102,21 @@
<artifactId>nimbus-jose-jwt</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>
+ <groupId>ch.qos.reload4j</groupId>
+ <artifactId>reload4j</artifactId>
+ </exclusion>
+ <!-- TODO(htowaileb): Remove after updating to hadoop 3.3.4 -->
+ <exclusion>
+ <groupId>org.eclipse.jetty</groupId>
+ <artifactId>jetty-util</artifactId>
</exclusion>
<exclusion>
- <groupId>org.slf4j</groupId>
- <artifactId>slf4j-log4j12</artifactId>
+ <groupId>org.eclipse.jetty</groupId>
+ <artifactId>jetty-util-ajax</artifactId>
</exclusion>
</exclusions>
</dependency>
@@ -1121,21 +1130,22 @@
<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>
<artifactId>jline</artifactId>
</exclusion>
+ <!-- TODO(htowaileb): Remove after updating to hadoop 3.3.4 -->
+ <exclusion>
+ <groupId>org.eclipse.jetty</groupId>
+ <artifactId>jetty-util</artifactId>
+ </exclusion>
</exclusions>
</dependency>
<dependency>
@@ -1144,16 +1154,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>
@@ -1163,16 +1169,21 @@
<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>
+ <groupId>ch.qos.reload4j</groupId>
+ <artifactId>reload4j</artifactId>
+ </exclusion>
+ <!-- TODO(htowaileb): Remove after updating to hadoop 3.3.4 -->
+ <exclusion>
+ <groupId>org.eclipse.jetty</groupId>
+ <artifactId>jetty-util</artifactId>
</exclusion>
<exclusion>
- <groupId>org.slf4j</groupId>
- <artifactId>slf4j-log4j12</artifactId>
+ <groupId>org.eclipse.jetty</groupId>
+ <artifactId>jetty-util-ajax</artifactId>
</exclusion>
</exclusions>
</dependency>
@@ -1187,23 +1198,28 @@
<artifactId>commons-logging</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>
+ <groupId>ch.qos.reload4j</groupId>
+ <artifactId>reload4j</artifactId>
+ </exclusion>
+ <!-- TODO(htowaileb): Remove after updating to hadoop 3.3.4 -->
+ <exclusion>
+ <groupId>org.eclipse.jetty</groupId>
+ <artifactId>jetty-util</artifactId>
</exclusion>
<exclusion>
- <groupId>org.slf4j</groupId>
- <artifactId>slf4j-log4j12</artifactId>
+ <groupId>org.eclipse.jetty</groupId>
+ <artifactId>jetty-util-ajax</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.apache.commons</groupId>
<artifactId>commons-compress</artifactId>
- <version>1.21</version>
+ <version>1.22</version>
</dependency>
<dependency>
<groupId>commons-logging</groupId>
@@ -1369,6 +1385,11 @@
</dependency>
<dependency>
<groupId>org.apache.hyracks</groupId>
+ <artifactId>hyracks-storage-am-lsm-btree-column</artifactId>
+ <version>${hyracks.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hyracks</groupId>
<artifactId>hyracks-storage-am-lsm-rtree</artifactId>
<version>${hyracks.version}</version>
</dependency>
@@ -1819,7 +1840,7 @@
<dependency>
<groupId>org.apache.tomcat</groupId>
<artifactId>tomcat-annotations-api</artifactId>
- <version>10.0.16</version>
+ <version>10.1.5</version>
</dependency>
<!-- Google Cloud Storage end -->
<!-- Azure Data Lake start -->
@@ -1882,6 +1903,28 @@
</exclusions>
</dependency>
<dependency>
+ <groupId>org.apache.parquet</groupId>
+ <artifactId>parquet-common</artifactId>
+ <version>${parquet.version}</version>
+ <exclusions>
+ <exclusion>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-api</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.parquet</groupId>
+ <artifactId>parquet-encoding</artifactId>
+ <version>${parquet.version}</version>
+ <exclusions>
+ <exclusion>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-api</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+ <dependency>
<groupId>org.kitesdk</groupId>
<artifactId>kite-data-core</artifactId>
<version>1.1.0</version>
@@ -1939,8 +1982,45 @@
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-azure</artifactId>
<version>${hadoop.version}</version>
+ <exclusions>
+ <!-- TODO(htowaileb): Remove after updating to hadoop 3.3.4 -->
+ <exclusion>
+ <groupId>org.eclipse.jetty</groupId>
+ <artifactId>jetty-util</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.eclipse.jetty</groupId>
+ <artifactId>jetty-util-ajax</artifactId>
+ </exclusion>
+ </exclusions>
</dependency>
<!-- Hadoop Azure end -->
+ <!-- Hadoop GCS start -->
+ <dependency>
+ <groupId>com.google.cloud.bigdataoss</groupId>
+ <artifactId>gcs-connector</artifactId>
+ <version>${hadoop-gcs.version}</version>
+ <exclusions>
+ <exclusion>
+ <groupId>org.checkerframework</groupId>
+ <artifactId>checker-compat-qual</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+ <!-- Hadoop GCS end -->
+
+ <!-- TODO(htowaileb): removed from hadoop transitively and added separately to avoid CVEs, can
+ be removed once upgraded to hadoop 3.3.4 as it addresses the CVEs -->
+ <dependency>
+ <groupId>org.eclipse.jetty</groupId>
+ <artifactId>jetty-util</artifactId>
+ <version>9.4.48.v20220622</version>
+ </dependency>
+ <dependency>
+ <groupId>org.eclipse.jetty</groupId>
+ <artifactId>jetty-util-ajax</artifactId>
+ <version>9.4.48.v20220622</version>
+ </dependency>
</dependencies>
</dependencyManagement>
diff --git a/asterixdb/src/main/appended-resources/supplemental-models.xml b/asterixdb/src/main/appended-resources/supplemental-models.xml
index fa6034b..ae9f9fd 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.73.Final</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>4.1.73.Final</license.ignoreMissingEmbeddedNotice>
- <license.ignoreNoticeOverride>4.1.73.Final</license.ignoreNoticeOverride>
+ <license.ignoreMissingEmbeddedLicense>4.1.87.Final</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>4.1.87.Final</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreNoticeOverride>4.1.87.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.72.Final</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>4.1.72.Final</license.ignoreMissingEmbeddedNotice>
- <license.ignoreNoticeOverride>4.1.712.Final</license.ignoreNoticeOverride>
+ <license.ignoreMissingEmbeddedLicense>4.1.87.Final</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>4.1.87.Final</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreNoticeOverride>4.1.87.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.72.Final,4.1.73.Final</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>4.1.72.Final,4.1.73.Final</license.ignoreMissingEmbeddedNotice>
- <license.ignoreNoticeOverride>4.1.72.Final,4.1.73.Final</license.ignoreNoticeOverride>
+ <license.ignoreMissingEmbeddedLicense>4.1.87.Final</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>4.1.87.Final</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreNoticeOverride>4.1.87.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.73.Final</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>4.1.73.Final</license.ignoreMissingEmbeddedNotice>
- <license.ignoreNoticeOverride>4.1.73.Final</license.ignoreNoticeOverride>
+ <license.ignoreMissingEmbeddedLicense>4.1.87.Final</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>4.1.87.Final</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreNoticeOverride>4.1.87.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.73.Final</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>4.1.73.Final</license.ignoreMissingEmbeddedNotice>
- <license.ignoreNoticeOverride>4.1.73.Final</license.ignoreNoticeOverride>
+ <license.ignoreMissingEmbeddedLicense>4.1.87.Final</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>4.1.87.Final</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreNoticeOverride>4.1.87.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.73.Final</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>4.1.73.Final</license.ignoreMissingEmbeddedNotice>
- <license.ignoreNoticeOverride>4.1.73.Final</license.ignoreNoticeOverride>
+ <license.ignoreMissingEmbeddedLicense>4.1.87.Final</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>4.1.87.Final</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreNoticeOverride>4.1.87.Final</license.ignoreNoticeOverride>
</properties>
</project>
</supplement>
@@ -235,9 +235,9 @@
<artifactId>netty-handler</artifactId>
<properties>
<!-- netty is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
- <license.ignoreMissingEmbeddedLicense>4.1.73.Final</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>4.1.73.Final</license.ignoreMissingEmbeddedNotice>
- <license.ignoreNoticeOverride>4.1.73.Final</license.ignoreNoticeOverride>
+ <license.ignoreMissingEmbeddedLicense>4.1.87.Final</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>4.1.87.Final</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreNoticeOverride>4.1.87.Final</license.ignoreNoticeOverride>
</properties>
</project>
</supplement>
@@ -247,9 +247,9 @@
<artifactId>netty-buffer</artifactId>
<properties>
<!-- netty is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
- <license.ignoreMissingEmbeddedLicense>4.1.73.Final</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>4.1.73.Final</license.ignoreMissingEmbeddedNotice>
- <license.ignoreNoticeOverride>4.1.73.Final</license.ignoreNoticeOverride>
+ <license.ignoreMissingEmbeddedLicense>4.1.87.Final</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>4.1.87.Final</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreNoticeOverride>4.1.87.Final</license.ignoreNoticeOverride>
</properties>
</project>
</supplement>
@@ -259,9 +259,9 @@
<artifactId>netty-common</artifactId>
<properties>
<!-- netty is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
- <license.ignoreMissingEmbeddedLicense>4.1.73.Final</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>4.1.73.Final</license.ignoreMissingEmbeddedNotice>
- <license.ignoreNoticeOverride>4.1.73.Final</license.ignoreNoticeOverride>
+ <license.ignoreMissingEmbeddedLicense>4.1.87.Final</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>4.1.87.Final</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreNoticeOverride>4.1.87.Final</license.ignoreNoticeOverride>
</properties>
</project>
</supplement>
@@ -271,9 +271,9 @@
<artifactId>netty-codec-http</artifactId>
<properties>
<!-- netty is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
- <license.ignoreMissingEmbeddedLicense>4.1.73.Final</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>4.1.73.Final</license.ignoreMissingEmbeddedNotice>
- <license.ignoreNoticeOverride>4.1.73.Final</license.ignoreNoticeOverride>
+ <license.ignoreMissingEmbeddedLicense>4.1.87.Final</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>4.1.87.Final</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreNoticeOverride>4.1.87.Final</license.ignoreNoticeOverride>
</properties>
</project>
</supplement>
@@ -283,9 +283,9 @@
<artifactId>netty-resolver</artifactId>
<properties>
<!-- netty is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
- <license.ignoreMissingEmbeddedLicense>4.1.73.Final</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>4.1.73.Final</license.ignoreMissingEmbeddedNotice>
- <license.ignoreNoticeOverride>4.1.73.Final</license.ignoreNoticeOverride>
+ <license.ignoreMissingEmbeddedLicense>4.1.87.Final</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>4.1.87.Final</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreNoticeOverride>4.1.87.Final</license.ignoreNoticeOverride>
</properties>
</project>
</supplement>
@@ -295,9 +295,21 @@
<artifactId>netty-resolver-dns</artifactId>
<properties>
<!-- netty is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
- <license.ignoreMissingEmbeddedLicense>4.1.73.Final</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>4.1.73.Final</license.ignoreMissingEmbeddedNotice>
- <license.ignoreNoticeOverride>4.1.73.Final</license.ignoreNoticeOverride>
+ <license.ignoreMissingEmbeddedLicense>4.1.87.Final</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>4.1.87.Final</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreNoticeOverride>4.1.87.Final</license.ignoreNoticeOverride>
+ </properties>
+ </project>
+ </supplement>
+ <supplement>
+ <project>
+ <groupId>io.netty</groupId>
+ <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>
</properties>
</project>
</supplement>
@@ -310,8 +322,8 @@
<!-- see https://github.com/google/guava/blob/v18.0/COPYING -->
<!-- see https://github.com/google/guava/tree/v28.[012]/COPYING -->
<!-- see https://github.com/google/guava/blob/v31.0.1/COPYING -->
- <license.ignoreMissingEmbeddedNotice>18.0,28.0-jre,28.1-jre,28.2-jre,30.1-jre,31.0.1-jre</license.ignoreMissingEmbeddedNotice>
- <license.ignoreMissingEmbeddedLicense>18.0,28.0-jre,28.1-jre,28.2-jre,30.1-jre,31.0.1-jre</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>18.0,28.0-jre,28.1-jre,28.2-jre,30.1-jre,31.0.1-jre,31.1-jre</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreMissingEmbeddedLicense>18.0,28.0-jre,28.1-jre,28.2-jre,30.1-jre,31.0.1-jre,31.1-jre</license.ignoreMissingEmbeddedLicense>
</properties>
</project>
</supplement>
@@ -334,8 +346,8 @@
<!-- see https://github.com/google/error-prone/tree/v2.3.[24]/ -->
<!-- see https://github.com/google/error-prone/blob/v2.7.1/COPYING -->
<properties>
- <license.ignoreMissingEmbeddedLicense>2.3.2,2.3.4,2.7.1</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>2.3.2,2.3.4,2.7.1</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreMissingEmbeddedLicense>2.3.2,2.3.4,2.7.1,2.11.0</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>2.3.2,2.3.4,2.7.1,2.11.0</license.ignoreMissingEmbeddedNotice>
</properties>
</project>
</supplement>
@@ -366,7 +378,7 @@
<artifactId>jackson-annotations</artifactId>
<properties>
<!-- jackson-annotations does not provide an embedded NOTICE file -->
- <license.ignoreMissingEmbeddedNotice>2.12.3,2.13.1,2.13.2</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreMissingEmbeddedNotice>2.12.3,2.13.1,2.13.2,2.13.4,2.14.1</license.ignoreMissingEmbeddedNotice>
</properties>
</project>
</supplement>
@@ -543,8 +555,8 @@
<groupId>software.amazon.awssdk</groupId>
<artifactId>third-party-jackson-core</artifactId>
<properties>
- <license.alternateNoticeFile>2.17.116:META-INF/NOTICE.txt</license.alternateNoticeFile>
- <license.alternateLicenseFile>2.17.116:META-INF/LICENSE.txt</license.alternateLicenseFile>
+ <license.alternateNoticeFile>2.17.218:META-INF/NOTICE.txt</license.alternateNoticeFile>
+ <license.alternateLicenseFile>2.17.218:META-INF/LICENSE.txt</license.alternateLicenseFile>
</properties>
</project>
</supplement>
@@ -568,14 +580,14 @@
<!-- AWS Hadoop SDK start -->
<!-- software.amazon.awssdk is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
<!-- license override not needed, ALv2 is specified in its pom.xml -->
- <!-- see https://github.com/aws/aws-sdk-java-v2/blob/master/LICENSE.txt -->
+ <!-- see https://github.com/aws/aws-sdk-java/blob/master/LICENSE.txt -->
<supplement>
<project>
<groupId>com.amazonaws</groupId>
<artifactId>aws-java-sdk-core</artifactId>
<properties>
- <license.ignoreMissingEmbeddedLicense>1.12.109</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>1.12.109</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreMissingEmbeddedLicense>1.12.402</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.12.402</license.ignoreMissingEmbeddedNotice>
</properties>
</project>
</supplement>
@@ -585,8 +597,8 @@
<groupId>com.amazonaws</groupId>
<artifactId>jmespath-java</artifactId>
<properties>
- <license.ignoreMissingEmbeddedLicense>1.12.109</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>1.12.109</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreMissingEmbeddedLicense>1.12.402</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.12.402</license.ignoreMissingEmbeddedNotice>
</properties>
</project>
</supplement>
@@ -596,8 +608,8 @@
<groupId>com.amazonaws</groupId>
<artifactId>aws-java-sdk-s3</artifactId>
<properties>
- <license.ignoreMissingEmbeddedLicense>1.12.109</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>1.12.109</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreMissingEmbeddedLicense>1.12.402</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.12.402</license.ignoreMissingEmbeddedNotice>
</properties>
</project>
</supplement>
@@ -607,8 +619,8 @@
<groupId>com.amazonaws</groupId>
<artifactId>aws-java-sdk-kms</artifactId>
<properties>
- <license.ignoreMissingEmbeddedLicense>1.12.109</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>1.12.109</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreMissingEmbeddedLicense>1.12.402</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.12.402</license.ignoreMissingEmbeddedNotice>
</properties>
</project>
</supplement>
@@ -618,8 +630,8 @@
<groupId>com.amazonaws</groupId>
<artifactId>aws-java-sdk-dynamodb</artifactId>
<properties>
- <license.ignoreMissingEmbeddedLicense>1.12.109</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>1.12.109</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreMissingEmbeddedLicense>1.12.402</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.12.402</license.ignoreMissingEmbeddedNotice>
</properties>
</project>
</supplement>
@@ -865,8 +877,8 @@
<groupId>com.fasterxml.jackson.dataformat</groupId>
<artifactId>jackson-dataformat-cbor</artifactId>
<properties>
- <license.ignoreMissingEmbeddedLicense>2.12.3</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>2.12.3</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreMissingEmbeddedLicense>2.12.6</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>2.12.6</license.ignoreMissingEmbeddedNotice>
</properties>
</project>
</supplement>
@@ -1115,9 +1127,9 @@
<groupId>com.google.api</groupId>
<artifactId>api-common</artifactId>
<properties>
- <license.ignoreMissingEmbeddedLicense>2.1.2</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>2.1.2</license.ignoreMissingEmbeddedNotice>
- <license.ignoreLicenseOverride>2.1.2</license.ignoreLicenseOverride>
+ <license.ignoreMissingEmbeddedLicense>2.5.0</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>2.5.0</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>2.5.0</license.ignoreLicenseOverride>
</properties>
</project>
</supplement>
@@ -1128,9 +1140,9 @@
<groupId>com.google.api</groupId>
<artifactId>gax-httpjson</artifactId>
<properties>
- <license.ignoreMissingEmbeddedLicense>0.93.1</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>0.93.1</license.ignoreMissingEmbeddedNotice>
- <license.ignoreLicenseOverride>0.93.1</license.ignoreLicenseOverride>
+ <license.ignoreMissingEmbeddedLicense>0.107.0</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>0.107.0</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>0.107.0</license.ignoreLicenseOverride>
</properties>
</project>
</supplement>
@@ -1141,9 +1153,9 @@
<groupId>com.google.api</groupId>
<artifactId>gax</artifactId>
<properties>
- <license.ignoreMissingEmbeddedLicense>2.8.1</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>2.8.1</license.ignoreMissingEmbeddedNotice>
- <license.ignoreLicenseOverride>2.8.1</license.ignoreLicenseOverride>
+ <license.ignoreMissingEmbeddedLicense>2.22.0</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>2.22.0</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>2.22.0</license.ignoreLicenseOverride>
</properties>
</project>
</supplement>
@@ -1154,9 +1166,59 @@
<groupId>com.google.api-client</groupId>
<artifactId>google-api-client</artifactId>
<properties>
- <license.ignoreMissingEmbeddedLicense>1.33.0</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>1.33.0</license.ignoreMissingEmbeddedNotice>
- <license.ignoreLicenseOverride>1.33.0</license.ignoreLicenseOverride>
+ <license.ignoreMissingEmbeddedLicense>2.1.2</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>2.1.2</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>2.1.2</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- io.opencensus uses ALv2 with no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>io.opencensus</groupId>
+ <artifactId>opencensus-proto</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>0.2.0</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>0.2.0</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>0.2.0</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.google.android:annotations has ASLv2 <license> in pom -->
+ <supplement>
+ <project>
+ <groupId>com.google.android</groupId>
+ <artifactId>annotations</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>4.1.1.4</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>4.1.1.4</license.ignoreMissingEmbeddedNotice>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- io.opencensus uses ALv2 with no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>io.opencensus</groupId>
+ <artifactId>opencensus-proto</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>0.2.0</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>0.2.0</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>0.2.0</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.google.android:annotations has ASLv2 <license> in pom -->
+ <supplement>
+ <project>
+ <groupId>com.google.android</groupId>
+ <artifactId>annotations</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>4.1.1.4</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>4.1.1.4</license.ignoreMissingEmbeddedNotice>
</properties>
</project>
</supplement>
@@ -1167,9 +1229,61 @@
<groupId>io.opencensus</groupId>
<artifactId>opencensus-api</artifactId>
<properties>
- <license.ignoreMissingEmbeddedLicense>0.28.0</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>0.28.0</license.ignoreMissingEmbeddedNotice>
- <license.ignoreLicenseOverride>0.28.0</license.ignoreLicenseOverride>
+ <license.ignoreMissingEmbeddedLicense>0.31.1</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>0.31.1</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>0.31.1</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- io.opencensus uses ALv2 with no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>io.opencensus</groupId>
+ <artifactId>opencensus-contrib-exemplar-util</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>0.31.0</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>0.31.0</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>0.31.0</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- io.opencensus uses ALv2 with no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>io.opencensus</groupId>
+ <artifactId>opencensus-contrib-grpc-metrics</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>0.31.0</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>0.31.0</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>0.31.0</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- io.opencensus uses ALv2 with no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>io.opencensus</groupId>
+ <artifactId>opencensus-contrib-exemplar-util</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>0.31.0</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>0.31.0</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>0.31.0</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- io.opencensus uses ALv2 with no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>io.opencensus</groupId>
+ <artifactId>opencensus-contrib-grpc-metrics</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>0.31.0</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>0.31.0</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>0.31.0</license.ignoreLicenseOverride>
</properties>
</project>
</supplement>
@@ -1180,9 +1294,357 @@
<groupId>io.opencensus</groupId>
<artifactId>opencensus-contrib-http-util</artifactId>
<properties>
- <license.ignoreMissingEmbeddedLicense>0.28.0</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>0.28.0</license.ignoreMissingEmbeddedNotice>
- <license.ignoreLicenseOverride>0.28.0</license.ignoreLicenseOverride>
+ <license.ignoreMissingEmbeddedLicense>0.31.1</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>0.31.1</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>0.31.1</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.google.api.grpc uses ALv2 with no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>com.google.api.grpc</groupId>
+ <artifactId>gapic-google-cloud-storage-v2</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>2.17.2-alpha</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>2.17.2-alpha</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>2.17.2-alpha</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.google.api.grpc uses ALv2 with no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>com.google.api.grpc</groupId>
+ <artifactId>grpc-google-cloud-storage-v2</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>2.17.2-alpha</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>2.17.2-alpha</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>2.17.2-alpha</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.google.api.grpc uses ALv2 with no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>com.google.api.grpc</groupId>
+ <artifactId>proto-google-cloud-storage-v2</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>2.17.2-alpha</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>2.17.2-alpha</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>2.17.2-alpha</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- org.conscrypt uses ALv2 LICENSE and has a NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>org.conscrypt</groupId>
+ <artifactId>conscrypt-openjdk-uber</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>2.5.2</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>2.5.2</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>2.5.2</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- io.perfmark uses ALv2 license -->
+ <supplement>
+ <project>
+ <groupId>io.perfmark</groupId>
+ <artifactId>perfmark-api</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>0.26.0</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>0.26.0</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>0.26.0</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- org.codehaus.mojo:animal-sniffer-annotations uses MIT license -->
+ <supplement>
+ <project>
+ <groupId>org.codehaus.mojo</groupId>
+ <artifactId>animal-sniffer-annotations</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.22</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.22</license.ignoreMissingEmbeddedNotice>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.google.apis:google-api-services-iamcredentials has embedded ASLv2 in pom.xml -->
+ <supplement>
+ <project>
+ <groupId>com.google.apis</groupId>
+ <artifactId>google-api-services-iamcredentials</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>v1-rev20210326-1.32.1</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>v1-rev20210326-1.32.1</license.ignoreMissingEmbeddedNotice>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.google.apis:google-api-services-iamcredentials has embedded ASLv2 in pom.xml -->
+ <supplement>
+ <project>
+ <groupId>com.google.api</groupId>
+ <artifactId>gax-grpc</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>2.22.0</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>2.22.0</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>2.22.0</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.google.cloud uses ALv2 with no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>com.google.cloud</groupId>
+ <artifactId>google-cloud-core-grpc</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>2.9.4</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>2.9.4</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>2.9.4</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- io.opencensus uses ALv2 with no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>io.opencensus</groupId>
+ <artifactId>opencensus-contrib-resource-util</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>0.31.0</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>0.31.0</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>0.31.0</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- io.opencensus uses ALv2 with no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>io.opencensus</groupId>
+ <artifactId>opencensus-exporter-metrics-util</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>0.31.0</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>0.31.0</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>0.31.0</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- io.opencensus uses ALv2 with no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>io.opencensus</groupId>
+ <artifactId>opencensus-exporter-stats-stackdriver</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>0.31.0</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>0.31.0</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>0.31.0</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- io.opencensus uses ALv2 with no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>io.opencensus</groupId>
+ <artifactId>opencensus-impl-core</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>0.31.0</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>0.31.0</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>0.31.0</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- io.opencensus uses ALv2 with no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>io.opencensus</groupId>
+ <artifactId>opencensus-impl</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>0.31.0</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>0.31.0</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>0.31.0</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.google.cloud.bigdataoss uses ALv2 with no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>com.google.cloud.bigdataoss</groupId>
+ <artifactId>gcs-connector</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>hadoop3-2.2.6</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>hadoop3-2.2.6</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>hadoop3-2.2.6</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.google.cloud.bigdataoss uses ALv2 with no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>com.google.cloud.bigdataoss</groupId>
+ <artifactId>gcsio</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>2.2.6</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>2.2.6</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>2.2.6</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.google.cloud.bigdataoss uses ALv2 with no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>com.google.cloud.bigdataoss</groupId>
+ <artifactId>util-hadoop</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>hadoop3-2.2.6</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>hadoop3-2.2.6</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>hadoop3-2.2.6</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.google.cloud.bigdataoss uses ALv2 with no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>com.google.cloud.bigdataoss</groupId>
+ <artifactId>util</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>2.2.6</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>2.2.6</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>2.2.6</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.google.flogger uses ALv2 with no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>com.google.flogger</groupId>
+ <artifactId>flogger</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>0.7.1</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>0.7.1</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>0.7.1</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.google.flogger uses ALv2 with no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>com.google.flogger</groupId>
+ <artifactId>google-extensions</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>0.7.1</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>0.7.1</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>0.7.1</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.google.flogger uses ALv2 with no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>com.google.flogger</groupId>
+ <artifactId>flogger-system-backend</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>0.7.1</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>0.7.1</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>0.7.1</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.google.api.grpc uses ALv2 with no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>com.google.api.grpc</groupId>
+ <artifactId>proto-google-cloud-monitoring-v3</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.64.0</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.64.0</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>1.64.0</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.lmax uses ALv2 with no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>com.lmax</groupId>
+ <artifactId>disruptor</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>3.4.2</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>3.4.2</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>3.4.2</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.google.cloud uses ALv2 with no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>com.google.cloud</groupId>
+ <artifactId>google-cloud-monitoring</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.82.0</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.82.0</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>1.82.0</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.google.api-client uses ALv2 LICENSE and has a NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>com.google.api-client</groupId>
+ <artifactId>google-api-client-jackson2</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.32.2</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.32.2</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>1.32.2</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- io.perfmark uses ALv2 license -->
+ <supplement>
+ <project>
+ <groupId>io.perfmark</groupId>
+ <artifactId>perfmark-api</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>0.23.0</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>0.23.0</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>0.23.0</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.google.apis:google-api-services-iamcredentials has embedded ASLv2 in pom.xml -->
+ <supplement>
+ <project>
+ <groupId>com.google.apis</groupId>
+ <artifactId>google-api-services-iamcredentials</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>v1-rev20210326-1.32.1</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>v1-rev20210326-1.32.1</license.ignoreMissingEmbeddedNotice>
</properties>
</project>
</supplement>
@@ -1193,9 +1655,9 @@
<groupId>com.google.cloud</groupId>
<artifactId>google-cloud-core</artifactId>
<properties>
- <license.ignoreMissingEmbeddedLicense>2.3.5</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>2.3.5</license.ignoreMissingEmbeddedNotice>
- <license.ignoreLicenseOverride>2.3.5</license.ignoreLicenseOverride>
+ <license.ignoreMissingEmbeddedLicense>2.9.4</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>2.9.4</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>2.9.4</license.ignoreLicenseOverride>
</properties>
</project>
</supplement>
@@ -1206,9 +1668,9 @@
<groupId>com.google.cloud</groupId>
<artifactId>google-cloud-core-http</artifactId>
<properties>
- <license.ignoreMissingEmbeddedLicense>2.3.5</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>2.3.5</license.ignoreMissingEmbeddedNotice>
- <license.ignoreLicenseOverride>2.3.5</license.ignoreLicenseOverride>
+ <license.ignoreMissingEmbeddedLicense>2.9.4</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>2.9.4</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>2.9.4</license.ignoreLicenseOverride>
</properties>
</project>
</supplement>
@@ -1219,9 +1681,9 @@
<groupId>com.google.auth</groupId>
<artifactId>google-auth-library-credentials</artifactId>
<properties>
- <license.ignoreMissingEmbeddedLicense>1.3.0</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>1.3.0</license.ignoreMissingEmbeddedNotice>
- <license.ignoreLicenseOverride>1.3.0</license.ignoreLicenseOverride>
+ <license.ignoreMissingEmbeddedLicense>1.14.0</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.14.0</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>1.14.0</license.ignoreLicenseOverride>
</properties>
</project>
</supplement>
@@ -1232,19 +1694,268 @@
<groupId>com.google.auth</groupId>
<artifactId>google-auth-library-oauth2-http</artifactId>
<properties>
- <license.ignoreMissingEmbeddedLicense>1.3.0</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>1.3.0</license.ignoreMissingEmbeddedNotice>
- <license.ignoreLicenseOverride>1.3.0</license.ignoreLicenseOverride>
+ <license.ignoreMissingEmbeddedLicense>1.14.0</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.14.0</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>1.14.0</license.ignoreLicenseOverride>
</properties>
</project>
</supplement>
- <!-- io.grpc:grpc-context uses non-fixed ALv2 and has no NOTICE -->
+ <!-- io.grpc uses ALv2 -->
+ <supplement>
+ <project>
+ <groupId>io.grpc</groupId>
+ <artifactId>grpc-alts</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.52.1</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.52.1</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>1.52.1</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- io.grpc uses ALv2 -->
+ <supplement>
+ <project>
+ <groupId>io.grpc</groupId>
+ <artifactId>grpc-api</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.43.2,1.52.1</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.43.2,1.52.1</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>1.43.2,1.52.1</license.ignoreLicenseOverride>
+ <license.ignoreNoticeOverride>1.43.2</license.ignoreNoticeOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- io.grpc uses ALv2 -->
+ <supplement>
+ <project>
+ <groupId>io.grpc</groupId>
+ <artifactId>grpc-auth</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.52.1</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.52.1</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>1.52.1</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- io.grpc uses ALv2 -->
+ <supplement>
+ <project>
+ <groupId>io.grpc</groupId>
+ <artifactId>grpc-census</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.43.2</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.43.2</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>1.43.2</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- io.grpc uses ALv2 -->
<supplement>
<project>
<groupId>io.grpc</groupId>
<artifactId>grpc-context</artifactId>
<properties>
+ <license.ignoreMissingEmbeddedLicense>1.52.1</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.52.1</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>1.52.1</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- io.grpc uses ALv2 -->
+ <supplement>
+ <project>
+ <groupId>io.grpc</groupId>
+ <artifactId>grpc-core</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.52.1</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.52.1</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>1.52.1</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- io.grpc uses ALv2 -->
+ <supplement>
+ <project>
+ <groupId>io.grpc</groupId>
+ <artifactId>grpc-googleapis</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.52.1</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.52.1</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>1.52.1</license.ignoreLicenseOverride>
+ <license.ignoreNoticeOverride>1.52.1</license.ignoreNoticeOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- io.grpc uses ALv2 -->
+ <supplement>
+ <project>
+ <groupId>io.grpc</groupId>
+ <artifactId>grpc-grpclb</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.52.1</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.52.1</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>1.52.1</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- io.grpc uses ALv2 -->
+ <supplement>
+ <project>
+ <groupId>io.grpc</groupId>
+ <artifactId>grpc-protobuf-lite</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.52.1</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.52.1</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>1.52.1</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- io.grpc uses ALv2 -->
+ <supplement>
+ <project>
+ <groupId>io.grpc</groupId>
+ <artifactId>grpc-protobuf</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.52.1</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.52.1</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>1.52.1</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- io.grpc uses ALv2 -->
+ <supplement>
+ <project>
+ <groupId>io.grpc</groupId>
+ <artifactId>grpc-services</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.52.1</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.52.1</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>1.52.1</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- io.grpc uses ALv2 -->
+ <supplement>
+ <project>
+ <groupId>io.grpc</groupId>
+ <artifactId>grpc-stub</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.52.1</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.52.1</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>1.52.1</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- io.grpc uses ALv2 -->
+ <supplement>
+ <project>
+ <groupId>io.grpc</groupId>
+ <artifactId>grpc-xds</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.52.1</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.52.1</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>1.52.1</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- io.grpc uses ALv2 -->
+ <supplement>
+ <project>
+ <groupId>io.grpc</groupId>
+ <artifactId>grpc-core</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.43.2</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.43.2</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>1.43.2</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- io.grpc uses ALv2 -->
+ <supplement>
+ <project>
+ <groupId>io.grpc</groupId>
+ <artifactId>grpc-grpclb</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.43.2</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.43.2</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>1.43.2</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- io.grpc uses ALv2 -->
+ <supplement>
+ <project>
+ <groupId>io.grpc</groupId>
+ <artifactId>grpc-protobuf-lite</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.43.2</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.43.2</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>1.43.2</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- io.grpc uses ALv2 -->
+ <supplement>
+ <project>
+ <groupId>io.grpc</groupId>
+ <artifactId>grpc-protobuf</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.43.2</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.43.2</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>1.43.2</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- io.grpc uses ALv2 -->
+ <supplement>
+ <project>
+ <groupId>io.grpc</groupId>
+ <artifactId>grpc-services</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.43.2</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.43.2</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>1.43.2</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- io.grpc uses ALv2 -->
+ <supplement>
+ <project>
+ <groupId>io.grpc</groupId>
+ <artifactId>grpc-stub</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.43.2</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.43.2</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>1.43.2</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- io.grpc uses ALv2 -->
+ <supplement>
+ <project>
+ <groupId>io.grpc</groupId>
+ <artifactId>grpc-xds</artifactId>
+ <properties>
<license.ignoreMissingEmbeddedLicense>1.43.2</license.ignoreMissingEmbeddedLicense>
<license.ignoreMissingEmbeddedNotice>1.43.2</license.ignoreMissingEmbeddedNotice>
<license.ignoreLicenseOverride>1.43.2</license.ignoreLicenseOverride>
@@ -1258,9 +1969,9 @@
<groupId>com.google.protobuf</groupId>
<artifactId>protobuf-java-util</artifactId>
<properties>
- <license.ignoreMissingEmbeddedLicense>3.19.2</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>3.19.2</license.ignoreMissingEmbeddedNotice>
- <license.ignoreLicenseOverride>3.19.2</license.ignoreLicenseOverride>
+ <license.ignoreMissingEmbeddedLicense>3.21.12</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>3.21.12</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>3.21.12</license.ignoreLicenseOverride>
</properties>
</project>
</supplement>
@@ -1271,9 +1982,9 @@
<groupId>com.google.protobuf</groupId>
<artifactId>protobuf-java</artifactId>
<properties>
- <license.ignoreMissingEmbeddedLicense>3.19.2</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>3.19.2</license.ignoreMissingEmbeddedNotice>
- <license.ignoreLicenseOverride>3.19.2</license.ignoreLicenseOverride>
+ <license.ignoreMissingEmbeddedLicense>3.21.12</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>3.21.12</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>3.21.12</license.ignoreLicenseOverride>
</properties>
</project>
</supplement>
@@ -1284,7 +1995,7 @@
<groupId>org.threeten</groupId>
<artifactId>threetenbp</artifactId>
<properties>
- <license.ignoreMissingEmbeddedNotice>1.5.2</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreMissingEmbeddedNotice>1.6.5</license.ignoreMissingEmbeddedNotice>
</properties>
</project>
</supplement>
@@ -1295,8 +2006,8 @@
<groupId>com.google.http-client</groupId>
<artifactId>google-http-client-jackson2</artifactId>
<properties>
- <license.ignoreMissingEmbeddedLicense>1.41.0</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>1.41.0</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreMissingEmbeddedLicense>1.42.3</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.42.3</license.ignoreMissingEmbeddedNotice>
</properties>
</project>
</supplement>
@@ -1307,8 +2018,8 @@
<groupId>com.google.http-client</groupId>
<artifactId>google-http-client</artifactId>
<properties>
- <license.ignoreMissingEmbeddedLicense>1.41.0</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>1.41.0</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreMissingEmbeddedLicense>1.42.3</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.42.3</license.ignoreMissingEmbeddedNotice>
</properties>
</project>
</supplement>
@@ -1319,8 +2030,8 @@
<groupId>com.google.http-client</groupId>
<artifactId>google-http-client-appengine</artifactId>
<properties>
- <license.ignoreMissingEmbeddedLicense>1.41.0</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>1.41.0</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreMissingEmbeddedLicense>1.42.3</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.42.3</license.ignoreMissingEmbeddedNotice>
</properties>
</project>
</supplement>
@@ -1331,8 +2042,8 @@
<groupId>com.google.http-client</groupId>
<artifactId>google-http-client-apache-v2</artifactId>
<properties>
- <license.ignoreMissingEmbeddedLicense>1.41.0</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>1.41.0</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreMissingEmbeddedLicense>1.42.3</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.42.3</license.ignoreMissingEmbeddedNotice>
</properties>
</project>
</supplement>
@@ -1343,8 +2054,8 @@
<groupId>com.google.http-client</groupId>
<artifactId>google-http-client-gson</artifactId>
<properties>
- <license.ignoreMissingEmbeddedLicense>1.41.0</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>1.41.0</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreMissingEmbeddedLicense>1.42.3</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.42.3</license.ignoreMissingEmbeddedNotice>
</properties>
</project>
</supplement>
@@ -1368,9 +2079,9 @@
<groupId>com.google.code.gson</groupId>
<artifactId>gson</artifactId>
<properties>
- <license.ignoreMissingEmbeddedLicense>2.8.9</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>2.8.9</license.ignoreMissingEmbeddedNotice>
- <license.ignoreLicenseOverride>2.8.9</license.ignoreLicenseOverride>
+ <license.ignoreMissingEmbeddedLicense>2.10.1</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>2.10.1</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>2.10.1</license.ignoreLicenseOverride>
</properties>
</project>
</supplement>
@@ -1388,13 +2099,25 @@
</project>
</supplement>
- <!-- org.checkerframework:checker-compat-qual has no NOTICE file -->
+ <!-- org.checkerframework:checker-qual has no NOTICE file -->
<supplement>
<project>
<groupId>org.checkerframework</groupId>
<artifactId>checker-qual</artifactId>
<properties>
- <license.ignoreMissingEmbeddedNotice>3.20.0</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreMissingEmbeddedNotice>3.29.0</license.ignoreMissingEmbeddedNotice>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.google.auto.value:auto-value-annotations has no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>com.google.auto.value</groupId>
+ <artifactId>auto-value</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.10.1</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.10.1</license.ignoreMissingEmbeddedNotice>
</properties>
</project>
</supplement>
@@ -1405,8 +2128,8 @@
<groupId>com.google.auto.value</groupId>
<artifactId>auto-value-annotations</artifactId>
<properties>
- <license.ignoreMissingEmbeddedLicense>1.9</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>1.9</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreMissingEmbeddedLicense>1.10.1</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.10.1</license.ignoreMissingEmbeddedNotice>
</properties>
</project>
</supplement>
@@ -1429,8 +2152,8 @@
<groupId>com.google.apis</groupId>
<artifactId>google-api-services-storage</artifactId>
<properties>
- <license.ignoreMissingEmbeddedLicense>v1-rev20211201-1.32.1</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>v1-rev20211201-1.32.1</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreMissingEmbeddedLicense>v1-rev20220705-2.0.0</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>v1-rev20220705-2.0.0</license.ignoreMissingEmbeddedNotice>
</properties>
</project>
</supplement>
@@ -1441,9 +2164,9 @@
<groupId>com.google.oauth-client</groupId>
<artifactId>google-oauth-client</artifactId>
<properties>
- <license.ignoreMissingEmbeddedLicense>1.32.1</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>1.32.1</license.ignoreMissingEmbeddedNotice>
- <license.ignoreLicenseOverride>1.32.1</license.ignoreLicenseOverride>
+ <license.ignoreMissingEmbeddedLicense>1.34.1</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.34.1</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>1.34.1</license.ignoreLicenseOverride>
</properties>
</project>
</supplement>
@@ -1454,8 +2177,8 @@
<groupId>com.google.api.grpc</groupId>
<artifactId>proto-google-iam-v1</artifactId>
<properties>
- <license.ignoreMissingEmbeddedLicense>1.2.0</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>1.2.0</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreMissingEmbeddedLicense>1.8.0</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.8.0</license.ignoreMissingEmbeddedNotice>
</properties>
</project>
</supplement>
@@ -1466,8 +2189,8 @@
<groupId>com.google.api.grpc</groupId>
<artifactId>proto-google-common-protos</artifactId>
<properties>
- <license.ignoreMissingEmbeddedLicense>2.7.1</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>2.7.1</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreMissingEmbeddedLicense>2.13.0</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>2.13.0</license.ignoreMissingEmbeddedNotice>
</properties>
</project>
</supplement>
@@ -1478,8 +2201,8 @@
<groupId>com.google.cloud</groupId>
<artifactId>google-cloud-storage</artifactId>
<properties>
- <license.ignoreMissingEmbeddedLicense>2.3.0</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>2.3.0</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreMissingEmbeddedLicense>2.17.2</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>2.17.2</license.ignoreMissingEmbeddedNotice>
</properties>
</project>
</supplement>
@@ -1516,7 +2239,7 @@
<groupId>com.fasterxml.woodstox</groupId>
<artifactId>woodstox-core</artifactId>
<properties>
- <license.ignoreMissingEmbeddedNotice>6.2.7</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreMissingEmbeddedNotice>6.2.7,6.4.0</license.ignoreMissingEmbeddedNotice>
</properties>
</project>
</supplement>
@@ -1652,9 +2375,21 @@
<groupId>com.github.luben</groupId>
<artifactId>zstd-jni</artifactId>
<properties>
- <license.ignoreMissingEmbeddedLicense>1.4.9-1</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>1.4.9-1</license.ignoreMissingEmbeddedNotice>
- <license.ignoreLicenseOverride>1.4.9-1</license.ignoreLicenseOverride>
+ <license.ignoreMissingEmbeddedLicense>1.5.0-1</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.5.0-1</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>1.5.0-1</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <supplement>
+ <project>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-reload4j</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.7.36</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.7.36</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>1.7.36</license.ignoreLicenseOverride>
</properties>
</project>
</supplement>
@@ -1664,8 +2399,8 @@
<groupId>org.apache.parquet</groupId>
<artifactId>parquet-format-structures</artifactId>
<properties>
- <license.ignoreMissingEmbeddedLicense>1.12.0</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>1.12.0</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreMissingEmbeddedLicense>1.12.3</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.12.3</license.ignoreMissingEmbeddedNotice>
</properties>
</project>
</supplement>
@@ -1675,8 +2410,8 @@
<groupId>org.apache.parquet</groupId>
<artifactId>parquet-common</artifactId>
<properties>
- <license.ignoreMissingEmbeddedLicense>1.12.0</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>1.12.0</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreMissingEmbeddedLicense>1.12.3</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.12.3</license.ignoreMissingEmbeddedNotice>
</properties>
</project>
</supplement>
@@ -1687,7 +2422,7 @@
<groupId>org.apache.parquet</groupId>
<artifactId>parquet-column</artifactId>
<properties>
- <license.ignoreMissingEmbeddedNotice>1.12.0</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreMissingEmbeddedNotice>1.12.3</license.ignoreMissingEmbeddedNotice>
</properties>
</project>
</supplement>
@@ -1698,7 +2433,7 @@
<groupId>org.apache.parquet</groupId>
<artifactId>parquet-encoding</artifactId>
<properties>
- <license.ignoreMissingEmbeddedNotice>1.12.0</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreMissingEmbeddedNotice>1.12.3</license.ignoreMissingEmbeddedNotice>
</properties>
</project>
</supplement>
@@ -1709,7 +2444,31 @@
<groupId>org.apache.parquet</groupId>
<artifactId>parquet-hadoop</artifactId>
<properties>
- <license.ignoreMissingEmbeddedNotice>1.12.0</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreMissingEmbeddedNotice>1.12.3</license.ignoreMissingEmbeddedNotice>
+ </properties>
+ </project>
+ </supplement>
+
+ <supplement>
+ <project>
+ <groupId>org.bouncycastle</groupId>
+ <artifactId>bcpkix-jdk15on</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedNotice>1.60</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreMissingEmbeddedLicense>1.60</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreLicenseOverride>1.60</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <supplement>
+ <project>
+ <groupId>org.bouncycastle</groupId>
+ <artifactId>bcprov-jdk15on</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedNotice>1.60</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreMissingEmbeddedLicense>1.60</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreLicenseOverride>1.60</license.ignoreLicenseOverride>
</properties>
</project>
</supplement>
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_aws_aws-sdk-java_1.12.109_NOTICE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_aws_aws-sdk-java_1.12.402_NOTICE.txt
similarity index 100%
rename from asterixdb/src/main/licenses/content/raw.githubusercontent.com_aws_aws-sdk-java_1.12.109_NOTICE.txt
rename to asterixdb/src/main/licenses/content/raw.githubusercontent.com_aws_aws-sdk-java_1.12.402_NOTICE.txt
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_bcgit_bc-java_r1rv60_LICENSE.html.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_bcgit_bc-java_r1rv60_LICENSE.html.txt
new file mode 100644
index 0000000..b8f46c5
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_bcgit_bc-java_r1rv60_LICENSE.html.txt
@@ -0,0 +1,17 @@
+Copyright (c) 2000-2018 The Legion of the Bouncy Castle Inc. (http://www.bouncycastle.org)
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of this software
+and associated documentation files (the "Software"), to deal in the Software without restriction,
+including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense,
+and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so,
+subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all copies or substantial
+portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED,
+INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR
+PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
+LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR
+OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
+DEALINGS IN THE SOFTWARE.
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_census-instrumentation_opencensus-java_v0.28.0_LICENSE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_census-instrumentation_opencensus-java_v0.28.0_LICENSE.txt
deleted file mode 100644
index 7a4a3ea..0000000
--- a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_census-instrumentation_opencensus-java_v0.28.0_LICENSE.txt
+++ /dev/null
@@ -1,202 +0,0 @@
-
- Apache License
- Version 2.0, January 2004
- http://www.apache.org/licenses/
-
- TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
-
- 1. Definitions.
-
- "License" shall mean the terms and conditions for use, reproduction,
- and distribution as defined by Sections 1 through 9 of this document.
-
- "Licensor" shall mean the copyright owner or entity authorized by
- the copyright owner that is granting the License.
-
- "Legal Entity" shall mean the union of the acting entity and all
- other entities that control, are controlled by, or are under common
- control with that entity. For the purposes of this definition,
- "control" means (i) the power, direct or indirect, to cause the
- direction or management of such entity, whether by contract or
- otherwise, or (ii) ownership of fifty percent (50%) or more of the
- outstanding shares, or (iii) beneficial ownership of such entity.
-
- "You" (or "Your") shall mean an individual or Legal Entity
- exercising permissions granted by this License.
-
- "Source" form shall mean the preferred form for making modifications,
- including but not limited to software source code, documentation
- source, and configuration files.
-
- "Object" form shall mean any form resulting from mechanical
- transformation or translation of a Source form, including but
- not limited to compiled object code, generated documentation,
- and conversions to other media types.
-
- "Work" shall mean the work of authorship, whether in Source or
- Object form, made available under the License, as indicated by a
- copyright notice that is included in or attached to the work
- (an example is provided in the Appendix below).
-
- "Derivative Works" shall mean any work, whether in Source or Object
- form, that is based on (or derived from) the Work and for which the
- editorial revisions, annotations, elaborations, or other modifications
- represent, as a whole, an original work of authorship. For the purposes
- of this License, Derivative Works shall not include works that remain
- separable from, or merely link (or bind by name) to the interfaces of,
- the Work and Derivative Works thereof.
-
- "Contribution" shall mean any work of authorship, including
- the original version of the Work and any modifications or additions
- to that Work or Derivative Works thereof, that is intentionally
- submitted to Licensor for inclusion in the Work by the copyright owner
- or by an individual or Legal Entity authorized to submit on behalf of
- the copyright owner. For the purposes of this definition, "submitted"
- means any form of electronic, verbal, or written communication sent
- to the Licensor or its representatives, including but not limited to
- communication on electronic mailing lists, source code control systems,
- and issue tracking systems that are managed by, or on behalf of, the
- Licensor for the purpose of discussing and improving the Work, but
- excluding communication that is conspicuously marked or otherwise
- designated in writing by the copyright owner as "Not a Contribution."
-
- "Contributor" shall mean Licensor and any individual or Legal Entity
- on behalf of whom a Contribution has been received by Licensor and
- subsequently incorporated within the Work.
-
- 2. Grant of Copyright License. Subject to the terms and conditions of
- this License, each Contributor hereby grants to You a perpetual,
- worldwide, non-exclusive, no-charge, royalty-free, irrevocable
- copyright license to reproduce, prepare Derivative Works of,
- publicly display, publicly perform, sublicense, and distribute the
- Work and such Derivative Works in Source or Object form.
-
- 3. Grant of Patent License. Subject to the terms and conditions of
- this License, each Contributor hereby grants to You a perpetual,
- worldwide, non-exclusive, no-charge, royalty-free, irrevocable
- (except as stated in this section) patent license to make, have made,
- use, offer to sell, sell, import, and otherwise transfer the Work,
- where such license applies only to those patent claims licensable
- by such Contributor that are necessarily infringed by their
- Contribution(s) alone or by combination of their Contribution(s)
- with the Work to which such Contribution(s) was submitted. If You
- institute patent litigation against any entity (including a
- cross-claim or counterclaim in a lawsuit) alleging that the Work
- or a Contribution incorporated within the Work constitutes direct
- or contributory patent infringement, then any patent licenses
- granted to You under this License for that Work shall terminate
- as of the date such litigation is filed.
-
- 4. Redistribution. You may reproduce and distribute copies of the
- Work or Derivative Works thereof in any medium, with or without
- modifications, and in Source or Object form, provided that You
- meet the following conditions:
-
- (a) You must give any other recipients of the Work or
- Derivative Works a copy of this License; and
-
- (b) You must cause any modified files to carry prominent notices
- stating that You changed the files; and
-
- (c) You must retain, in the Source form of any Derivative Works
- that You distribute, all copyright, patent, trademark, and
- attribution notices from the Source form of the Work,
- excluding those notices that do not pertain to any part of
- the Derivative Works; and
-
- (d) If the Work includes a "NOTICE" text file as part of its
- distribution, then any Derivative Works that You distribute must
- include a readable copy of the attribution notices contained
- within such NOTICE file, excluding those notices that do not
- pertain to any part of the Derivative Works, in at least one
- of the following places: within a NOTICE text file distributed
- as part of the Derivative Works; within the Source form or
- documentation, if provided along with the Derivative Works; or,
- within a display generated by the Derivative Works, if and
- wherever such third-party notices normally appear. The contents
- of the NOTICE file are for informational purposes only and
- do not modify the License. You may add Your own attribution
- notices within Derivative Works that You distribute, alongside
- or as an addendum to the NOTICE text from the Work, provided
- that such additional attribution notices cannot be construed
- as modifying the License.
-
- You may add Your own copyright statement to Your modifications and
- may provide additional or different license terms and conditions
- for use, reproduction, or distribution of Your modifications, or
- for any such Derivative Works as a whole, provided Your use,
- reproduction, and distribution of the Work otherwise complies with
- the conditions stated in this License.
-
- 5. Submission of Contributions. Unless You explicitly state otherwise,
- any Contribution intentionally submitted for inclusion in the Work
- by You to the Licensor shall be under the terms and conditions of
- this License, without any additional terms or conditions.
- Notwithstanding the above, nothing herein shall supersede or modify
- the terms of any separate license agreement you may have executed
- with Licensor regarding such Contributions.
-
- 6. Trademarks. This License does not grant permission to use the trade
- names, trademarks, service marks, or product names of the Licensor,
- except as required for reasonable and customary use in describing the
- origin of the Work and reproducing the content of the NOTICE file.
-
- 7. Disclaimer of Warranty. Unless required by applicable law or
- agreed to in writing, Licensor provides the Work (and each
- Contributor provides its Contributions) on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
- implied, including, without limitation, any warranties or conditions
- of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
- PARTICULAR PURPOSE. You are solely responsible for determining the
- appropriateness of using or redistributing the Work and assume any
- risks associated with Your exercise of permissions under this License.
-
- 8. Limitation of Liability. In no event and under no legal theory,
- whether in tort (including negligence), contract, or otherwise,
- unless required by applicable law (such as deliberate and grossly
- negligent acts) or agreed to in writing, shall any Contributor be
- liable to You for damages, including any direct, indirect, special,
- incidental, or consequential damages of any character arising as a
- result of this License or out of the use or inability to use the
- Work (including but not limited to damages for loss of goodwill,
- work stoppage, computer failure or malfunction, or any and all
- other commercial damages or losses), even if such Contributor
- has been advised of the possibility of such damages.
-
- 9. Accepting Warranty or Additional Liability. While redistributing
- the Work or Derivative Works thereof, You may choose to offer,
- and charge a fee for, acceptance of support, warranty, indemnity,
- or other liability obligations and/or rights consistent with this
- License. However, in accepting such obligations, You may act only
- on Your own behalf and on Your sole responsibility, not on behalf
- of any other Contributor, and only if You agree to indemnify,
- defend, and hold each Contributor harmless for any liability
- incurred by, or claims asserted against, such Contributor by reason
- of your accepting any such warranty or additional liability.
-
- END OF TERMS AND CONDITIONS
-
- APPENDIX: How to apply the Apache License to your work.
-
- To apply the Apache License to your work, attach the following
- boilerplate notice, with the fields enclosed by brackets "[]"
- replaced with your own identifying information. (Don't include
- the brackets!) The text should be enclosed in the appropriate
- comment syntax for the file format. We also recommend that a
- file or class name and description of purpose be included on the
- same "printed page" as the copyright notice for easier
- identification within third-party archives.
-
- Copyright [yyyy] [name of copyright owner]
-
- Licensed under the Apache License, Version 2.0 (the "License");
- you may not use this file except in compliance with the License.
- You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_google_conscrypt_2.5.1_NOTICE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_google_conscrypt_2.5.1_NOTICE.txt
new file mode 100644
index 0000000..80715a6
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_google_conscrypt_2.5.1_NOTICE.txt
@@ -0,0 +1,30 @@
+Copyright 2016 The Android Open Source Project
+
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+-----------------------------------------------------------------------
+This product contains a modified portion of `Netty`, a configurable network
+stack in Java, which can be obtained at:
+
+ * LICENSE:
+ * licenses/LICENSE.netty.txt (Apache License 2.0)
+ * HOMEPAGE:
+ * http://netty.io/
+
+This product contains a modified portion of `Apache Harmony`, modular Java runtime,
+which can be obtained at:
+
+ * LICENSE:
+ * licenses/LICENSE.harmony.txt (Apache License 2.0)
+ * HOMEPAGE:
+ * https://harmony.apache.org/
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_google_conscrypt_2.5.2_NOTICE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_google_conscrypt_2.5.2_NOTICE.txt
new file mode 100644
index 0000000..80715a6
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_google_conscrypt_2.5.2_NOTICE.txt
@@ -0,0 +1,30 @@
+Copyright 2016 The Android Open Source Project
+
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+-----------------------------------------------------------------------
+This product contains a modified portion of `Netty`, a configurable network
+stack in Java, which can be obtained at:
+
+ * LICENSE:
+ * licenses/LICENSE.netty.txt (Apache License 2.0)
+ * HOMEPAGE:
+ * http://netty.io/
+
+This product contains a modified portion of `Apache Harmony`, modular Java runtime,
+which can be obtained at:
+
+ * LICENSE:
+ * licenses/LICENSE.harmony.txt (Apache License 2.0)
+ * HOMEPAGE:
+ * https://harmony.apache.org/
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_google_gson_gson-parent-2.8.9_LICENSE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_google_gson_gson-parent-2.8.9_LICENSE.txt
deleted file mode 100644
index 7a4a3ea..0000000
--- a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_google_gson_gson-parent-2.8.9_LICENSE.txt
+++ /dev/null
@@ -1,202 +0,0 @@
-
- Apache License
- Version 2.0, January 2004
- http://www.apache.org/licenses/
-
- TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
-
- 1. Definitions.
-
- "License" shall mean the terms and conditions for use, reproduction,
- and distribution as defined by Sections 1 through 9 of this document.
-
- "Licensor" shall mean the copyright owner or entity authorized by
- the copyright owner that is granting the License.
-
- "Legal Entity" shall mean the union of the acting entity and all
- other entities that control, are controlled by, or are under common
- control with that entity. For the purposes of this definition,
- "control" means (i) the power, direct or indirect, to cause the
- direction or management of such entity, whether by contract or
- otherwise, or (ii) ownership of fifty percent (50%) or more of the
- outstanding shares, or (iii) beneficial ownership of such entity.
-
- "You" (or "Your") shall mean an individual or Legal Entity
- exercising permissions granted by this License.
-
- "Source" form shall mean the preferred form for making modifications,
- including but not limited to software source code, documentation
- source, and configuration files.
-
- "Object" form shall mean any form resulting from mechanical
- transformation or translation of a Source form, including but
- not limited to compiled object code, generated documentation,
- and conversions to other media types.
-
- "Work" shall mean the work of authorship, whether in Source or
- Object form, made available under the License, as indicated by a
- copyright notice that is included in or attached to the work
- (an example is provided in the Appendix below).
-
- "Derivative Works" shall mean any work, whether in Source or Object
- form, that is based on (or derived from) the Work and for which the
- editorial revisions, annotations, elaborations, or other modifications
- represent, as a whole, an original work of authorship. For the purposes
- of this License, Derivative Works shall not include works that remain
- separable from, or merely link (or bind by name) to the interfaces of,
- the Work and Derivative Works thereof.
-
- "Contribution" shall mean any work of authorship, including
- the original version of the Work and any modifications or additions
- to that Work or Derivative Works thereof, that is intentionally
- submitted to Licensor for inclusion in the Work by the copyright owner
- or by an individual or Legal Entity authorized to submit on behalf of
- the copyright owner. For the purposes of this definition, "submitted"
- means any form of electronic, verbal, or written communication sent
- to the Licensor or its representatives, including but not limited to
- communication on electronic mailing lists, source code control systems,
- and issue tracking systems that are managed by, or on behalf of, the
- Licensor for the purpose of discussing and improving the Work, but
- excluding communication that is conspicuously marked or otherwise
- designated in writing by the copyright owner as "Not a Contribution."
-
- "Contributor" shall mean Licensor and any individual or Legal Entity
- on behalf of whom a Contribution has been received by Licensor and
- subsequently incorporated within the Work.
-
- 2. Grant of Copyright License. Subject to the terms and conditions of
- this License, each Contributor hereby grants to You a perpetual,
- worldwide, non-exclusive, no-charge, royalty-free, irrevocable
- copyright license to reproduce, prepare Derivative Works of,
- publicly display, publicly perform, sublicense, and distribute the
- Work and such Derivative Works in Source or Object form.
-
- 3. Grant of Patent License. Subject to the terms and conditions of
- this License, each Contributor hereby grants to You a perpetual,
- worldwide, non-exclusive, no-charge, royalty-free, irrevocable
- (except as stated in this section) patent license to make, have made,
- use, offer to sell, sell, import, and otherwise transfer the Work,
- where such license applies only to those patent claims licensable
- by such Contributor that are necessarily infringed by their
- Contribution(s) alone or by combination of their Contribution(s)
- with the Work to which such Contribution(s) was submitted. If You
- institute patent litigation against any entity (including a
- cross-claim or counterclaim in a lawsuit) alleging that the Work
- or a Contribution incorporated within the Work constitutes direct
- or contributory patent infringement, then any patent licenses
- granted to You under this License for that Work shall terminate
- as of the date such litigation is filed.
-
- 4. Redistribution. You may reproduce and distribute copies of the
- Work or Derivative Works thereof in any medium, with or without
- modifications, and in Source or Object form, provided that You
- meet the following conditions:
-
- (a) You must give any other recipients of the Work or
- Derivative Works a copy of this License; and
-
- (b) You must cause any modified files to carry prominent notices
- stating that You changed the files; and
-
- (c) You must retain, in the Source form of any Derivative Works
- that You distribute, all copyright, patent, trademark, and
- attribution notices from the Source form of the Work,
- excluding those notices that do not pertain to any part of
- the Derivative Works; and
-
- (d) If the Work includes a "NOTICE" text file as part of its
- distribution, then any Derivative Works that You distribute must
- include a readable copy of the attribution notices contained
- within such NOTICE file, excluding those notices that do not
- pertain to any part of the Derivative Works, in at least one
- of the following places: within a NOTICE text file distributed
- as part of the Derivative Works; within the Source form or
- documentation, if provided along with the Derivative Works; or,
- within a display generated by the Derivative Works, if and
- wherever such third-party notices normally appear. The contents
- of the NOTICE file are for informational purposes only and
- do not modify the License. You may add Your own attribution
- notices within Derivative Works that You distribute, alongside
- or as an addendum to the NOTICE text from the Work, provided
- that such additional attribution notices cannot be construed
- as modifying the License.
-
- You may add Your own copyright statement to Your modifications and
- may provide additional or different license terms and conditions
- for use, reproduction, or distribution of Your modifications, or
- for any such Derivative Works as a whole, provided Your use,
- reproduction, and distribution of the Work otherwise complies with
- the conditions stated in this License.
-
- 5. Submission of Contributions. Unless You explicitly state otherwise,
- any Contribution intentionally submitted for inclusion in the Work
- by You to the Licensor shall be under the terms and conditions of
- this License, without any additional terms or conditions.
- Notwithstanding the above, nothing herein shall supersede or modify
- the terms of any separate license agreement you may have executed
- with Licensor regarding such Contributions.
-
- 6. Trademarks. This License does not grant permission to use the trade
- names, trademarks, service marks, or product names of the Licensor,
- except as required for reasonable and customary use in describing the
- origin of the Work and reproducing the content of the NOTICE file.
-
- 7. Disclaimer of Warranty. Unless required by applicable law or
- agreed to in writing, Licensor provides the Work (and each
- Contributor provides its Contributions) on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
- implied, including, without limitation, any warranties or conditions
- of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
- PARTICULAR PURPOSE. You are solely responsible for determining the
- appropriateness of using or redistributing the Work and assume any
- risks associated with Your exercise of permissions under this License.
-
- 8. Limitation of Liability. In no event and under no legal theory,
- whether in tort (including negligence), contract, or otherwise,
- unless required by applicable law (such as deliberate and grossly
- negligent acts) or agreed to in writing, shall any Contributor be
- liable to You for damages, including any direct, indirect, special,
- incidental, or consequential damages of any character arising as a
- result of this License or out of the use or inability to use the
- Work (including but not limited to damages for loss of goodwill,
- work stoppage, computer failure or malfunction, or any and all
- other commercial damages or losses), even if such Contributor
- has been advised of the possibility of such damages.
-
- 9. Accepting Warranty or Additional Liability. While redistributing
- the Work or Derivative Works thereof, You may choose to offer,
- and charge a fee for, acceptance of support, warranty, indemnity,
- or other liability obligations and/or rights consistent with this
- License. However, in accepting such obligations, You may act only
- on Your own behalf and on Your sole responsibility, not on behalf
- of any other Contributor, and only if You agree to indemnify,
- defend, and hold each Contributor harmless for any liability
- incurred by, or claims asserted against, such Contributor by reason
- of your accepting any such warranty or additional liability.
-
- END OF TERMS AND CONDITIONS
-
- APPENDIX: How to apply the Apache License to your work.
-
- To apply the Apache License to your work, attach the following
- boilerplate notice, with the fields enclosed by brackets "[]"
- replaced with your own identifying information. (Don't include
- the brackets!) The text should be enclosed in the appropriate
- comment syntax for the file format. We also recommend that a
- file or class name and description of purpose be included on the
- same "printed page" as the copyright notice for easier
- identification within third-party archives.
-
- Copyright [yyyy] [name of copyright owner]
-
- Licensed under the Apache License, Version 2.0 (the "License");
- you may not use this file except in compliance with the License.
- You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_api-common-java_v2.1.2_LICENSE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_api-common-java_v2.1.2_LICENSE.txt
deleted file mode 100644
index 4e19437..0000000
--- a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_api-common-java_v2.1.2_LICENSE.txt
+++ /dev/null
@@ -1,27 +0,0 @@
-Copyright 2016, Google Inc.
-All rights reserved.
-Redistribution and use in source and binary forms, with or without
-modification, are permitted provided that the following conditions are
-met:
-
- * Redistributions of source code must retain the above copyright
-notice, this list of conditions and the following disclaimer.
- * Redistributions in binary form must reproduce the above
-copyright notice, this list of conditions and the following disclaimer
-in the documentation and/or other materials provided with the
-distribution.
- * Neither the name of Google Inc. nor the names of its
-contributors may be used to endorse or promote products derived from
-this software without specific prior written permission.
-
-THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
-"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
-LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
-A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
-OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
-SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
-LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
-DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
-THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
-(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
-OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_gapic-generator-java_v2.13.0_api-common-java_LICENSE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_gapic-generator-java_v2.13.0_api-common-java_LICENSE.txt
new file mode 100644
index 0000000..b442fc4
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_gapic-generator-java_v2.13.0_api-common-java_LICENSE.txt
@@ -0,0 +1,26 @@
+Copyright 2016, Google Inc.
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+ * Redistributions of source code must retain the above copyright
+notice, this list of conditions and the following disclaimer.
+ * Redistributions in binary form must reproduce the above
+copyright notice, this list of conditions and the following disclaimer
+in the documentation and/or other materials provided with the
+distribution.
+ * Neither the name of Google Inc. nor the names of its
+contributors may be used to endorse or promote products derived from
+this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_gax-java_v2.8.1_LICENSE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_gapic-generator-java_v2.13.0_gax-java_LICENSE.txt
similarity index 100%
rename from asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_gax-java_v2.8.1_LICENSE.txt
rename to asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_gapic-generator-java_v2.13.0_gax-java_LICENSE.txt
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_gax-java_v2.7.1_LICENSE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_gax-java_v2.7.1_LICENSE.txt
new file mode 100644
index 0000000..6874140
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_gax-java_v2.7.1_LICENSE.txt
@@ -0,0 +1,27 @@
+Copyright 2016, Google Inc. All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+ * Redistributions of source code must retain the above copyright
+notice, this list of conditions and the following disclaimer.
+ * Redistributions in binary form must reproduce the above
+copyright notice, this list of conditions and the following disclaimer
+in the documentation and/or other materials provided with the
+distribution.
+ * Neither the name of Google Inc. nor the names of its
+contributors may be used to endorse or promote products derived from
+this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_google-api-java-client_v1.33.0_LICENSE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_google-api-java-client_v1.33.0_LICENSE.txt
deleted file mode 100644
index f49a4e1..0000000
--- a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_google-api-java-client_v1.33.0_LICENSE.txt
+++ /dev/null
@@ -1,201 +0,0 @@
- Apache License
- Version 2.0, January 2004
- http://www.apache.org/licenses/
-
- TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
-
- 1. Definitions.
-
- "License" shall mean the terms and conditions for use, reproduction,
- and distribution as defined by Sections 1 through 9 of this document.
-
- "Licensor" shall mean the copyright owner or entity authorized by
- the copyright owner that is granting the License.
-
- "Legal Entity" shall mean the union of the acting entity and all
- other entities that control, are controlled by, or are under common
- control with that entity. For the purposes of this definition,
- "control" means (i) the power, direct or indirect, to cause the
- direction or management of such entity, whether by contract or
- otherwise, or (ii) ownership of fifty percent (50%) or more of the
- outstanding shares, or (iii) beneficial ownership of such entity.
-
- "You" (or "Your") shall mean an individual or Legal Entity
- exercising permissions granted by this License.
-
- "Source" form shall mean the preferred form for making modifications,
- including but not limited to software source code, documentation
- source, and configuration files.
-
- "Object" form shall mean any form resulting from mechanical
- transformation or translation of a Source form, including but
- not limited to compiled object code, generated documentation,
- and conversions to other media types.
-
- "Work" shall mean the work of authorship, whether in Source or
- Object form, made available under the License, as indicated by a
- copyright notice that is included in or attached to the work
- (an example is provided in the Appendix below).
-
- "Derivative Works" shall mean any work, whether in Source or Object
- form, that is based on (or derived from) the Work and for which the
- editorial revisions, annotations, elaborations, or other modifications
- represent, as a whole, an original work of authorship. For the purposes
- of this License, Derivative Works shall not include works that remain
- separable from, or merely link (or bind by name) to the interfaces of,
- the Work and Derivative Works thereof.
-
- "Contribution" shall mean any work of authorship, including
- the original version of the Work and any modifications or additions
- to that Work or Derivative Works thereof, that is intentionally
- submitted to Licensor for inclusion in the Work by the copyright owner
- or by an individual or Legal Entity authorized to submit on behalf of
- the copyright owner. For the purposes of this definition, "submitted"
- means any form of electronic, verbal, or written communication sent
- to the Licensor or its representatives, including but not limited to
- communication on electronic mailing lists, source code control systems,
- and issue tracking systems that are managed by, or on behalf of, the
- Licensor for the purpose of discussing and improving the Work, but
- excluding communication that is conspicuously marked or otherwise
- designated in writing by the copyright owner as "Not a Contribution."
-
- "Contributor" shall mean Licensor and any individual or Legal Entity
- on behalf of whom a Contribution has been received by Licensor and
- subsequently incorporated within the Work.
-
- 2. Grant of Copyright License. Subject to the terms and conditions of
- this License, each Contributor hereby grants to You a perpetual,
- worldwide, non-exclusive, no-charge, royalty-free, irrevocable
- copyright license to reproduce, prepare Derivative Works of,
- publicly display, publicly perform, sublicense, and distribute the
- Work and such Derivative Works in Source or Object form.
-
- 3. Grant of Patent License. Subject to the terms and conditions of
- this License, each Contributor hereby grants to You a perpetual,
- worldwide, non-exclusive, no-charge, royalty-free, irrevocable
- (except as stated in this section) patent license to make, have made,
- use, offer to sell, sell, import, and otherwise transfer the Work,
- where such license applies only to those patent claims licensable
- by such Contributor that are necessarily infringed by their
- Contribution(s) alone or by combination of their Contribution(s)
- with the Work to which such Contribution(s) was submitted. If You
- institute patent litigation against any entity (including a
- cross-claim or counterclaim in a lawsuit) alleging that the Work
- or a Contribution incorporated within the Work constitutes direct
- or contributory patent infringement, then any patent licenses
- granted to You under this License for that Work shall terminate
- as of the date such litigation is filed.
-
- 4. Redistribution. You may reproduce and distribute copies of the
- Work or Derivative Works thereof in any medium, with or without
- modifications, and in Source or Object form, provided that You
- meet the following conditions:
-
- (a) You must give any other recipients of the Work or
- Derivative Works a copy of this License; and
-
- (b) You must cause any modified files to carry prominent notices
- stating that You changed the files; and
-
- (c) You must retain, in the Source form of any Derivative Works
- that You distribute, all copyright, patent, trademark, and
- attribution notices from the Source form of the Work,
- excluding those notices that do not pertain to any part of
- the Derivative Works; and
-
- (d) If the Work includes a "NOTICE" text file as part of its
- distribution, then any Derivative Works that You distribute must
- include a readable copy of the attribution notices contained
- within such NOTICE file, excluding those notices that do not
- pertain to any part of the Derivative Works, in at least one
- of the following places: within a NOTICE text file distributed
- as part of the Derivative Works; within the Source form or
- documentation, if provided along with the Derivative Works; or,
- within a display generated by the Derivative Works, if and
- wherever such third-party notices normally appear. The contents
- of the NOTICE file are for informational purposes only and
- do not modify the License. You may add Your own attribution
- notices within Derivative Works that You distribute, alongside
- or as an addendum to the NOTICE text from the Work, provided
- that such additional attribution notices cannot be construed
- as modifying the License.
-
- You may add Your own copyright statement to Your modifications and
- may provide additional or different license terms and conditions
- for use, reproduction, or distribution of Your modifications, or
- for any such Derivative Works as a whole, provided Your use,
- reproduction, and distribution of the Work otherwise complies with
- the conditions stated in this License.
-
- 5. Submission of Contributions. Unless You explicitly state otherwise,
- any Contribution intentionally submitted for inclusion in the Work
- by You to the Licensor shall be under the terms and conditions of
- this License, without any additional terms or conditions.
- Notwithstanding the above, nothing herein shall supersede or modify
- the terms of any separate license agreement you may have executed
- with Licensor regarding such Contributions.
-
- 6. Trademarks. This License does not grant permission to use the trade
- names, trademarks, service marks, or product names of the Licensor,
- except as required for reasonable and customary use in describing the
- origin of the Work and reproducing the content of the NOTICE file.
-
- 7. Disclaimer of Warranty. Unless required by applicable law or
- agreed to in writing, Licensor provides the Work (and each
- Contributor provides its Contributions) on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
- implied, including, without limitation, any warranties or conditions
- of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
- PARTICULAR PURPOSE. You are solely responsible for determining the
- appropriateness of using or redistributing the Work and assume any
- risks associated with Your exercise of permissions under this License.
-
- 8. Limitation of Liability. In no event and under no legal theory,
- whether in tort (including negligence), contract, or otherwise,
- unless required by applicable law (such as deliberate and grossly
- negligent acts) or agreed to in writing, shall any Contributor be
- liable to You for damages, including any direct, indirect, special,
- incidental, or consequential damages of any character arising as a
- result of this License or out of the use or inability to use the
- Work (including but not limited to damages for loss of goodwill,
- work stoppage, computer failure or malfunction, or any and all
- other commercial damages or losses), even if such Contributor
- has been advised of the possibility of such damages.
-
- 9. Accepting Warranty or Additional Liability. While redistributing
- the Work or Derivative Works thereof, You may choose to offer,
- and charge a fee for, acceptance of support, warranty, indemnity,
- or other liability obligations and/or rights consistent with this
- License. However, in accepting such obligations, You may act only
- on Your own behalf and on Your sole responsibility, not on behalf
- of any other Contributor, and only if You agree to indemnify,
- defend, and hold each Contributor harmless for any liability
- incurred by, or claims asserted against, such Contributor by reason
- of your accepting any such warranty or additional liability.
-
- END OF TERMS AND CONDITIONS
-
- APPENDIX: How to apply the Apache License to your work.
-
- To apply the Apache License to your work, attach the following
- boilerplate notice, with the fields enclosed by brackets "[]"
- replaced with your own identifying information. (Don't include
- the brackets!) The text should be enclosed in the appropriate
- comment syntax for the file format. We also recommend that a
- file or class name and description of purpose be included on the
- same "printed page" as the copyright notice for easier
- identification within third-party archives.
-
- Copyright [yyyy] [name of copyright owner]
-
- Licensed under the Apache License, Version 2.0 (the "License");
- you may not use this file except in compliance with the License.
- You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_google-auth-library-java_v1.3.0_LICENSE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_google-auth-library-java_v1.14.0_LICENSE.txt
similarity index 100%
rename from asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_google-auth-library-java_v1.3.0_LICENSE.txt
rename to asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_google-auth-library-java_v1.14.0_LICENSE.txt
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_google-http-java-client_v1.41.0_LICENSE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_google-http-java-client_v1.41.0_LICENSE.txt
deleted file mode 100644
index f49a4e1..0000000
--- a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_google-http-java-client_v1.41.0_LICENSE.txt
+++ /dev/null
@@ -1,201 +0,0 @@
- Apache License
- Version 2.0, January 2004
- http://www.apache.org/licenses/
-
- TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
-
- 1. Definitions.
-
- "License" shall mean the terms and conditions for use, reproduction,
- and distribution as defined by Sections 1 through 9 of this document.
-
- "Licensor" shall mean the copyright owner or entity authorized by
- the copyright owner that is granting the License.
-
- "Legal Entity" shall mean the union of the acting entity and all
- other entities that control, are controlled by, or are under common
- control with that entity. For the purposes of this definition,
- "control" means (i) the power, direct or indirect, to cause the
- direction or management of such entity, whether by contract or
- otherwise, or (ii) ownership of fifty percent (50%) or more of the
- outstanding shares, or (iii) beneficial ownership of such entity.
-
- "You" (or "Your") shall mean an individual or Legal Entity
- exercising permissions granted by this License.
-
- "Source" form shall mean the preferred form for making modifications,
- including but not limited to software source code, documentation
- source, and configuration files.
-
- "Object" form shall mean any form resulting from mechanical
- transformation or translation of a Source form, including but
- not limited to compiled object code, generated documentation,
- and conversions to other media types.
-
- "Work" shall mean the work of authorship, whether in Source or
- Object form, made available under the License, as indicated by a
- copyright notice that is included in or attached to the work
- (an example is provided in the Appendix below).
-
- "Derivative Works" shall mean any work, whether in Source or Object
- form, that is based on (or derived from) the Work and for which the
- editorial revisions, annotations, elaborations, or other modifications
- represent, as a whole, an original work of authorship. For the purposes
- of this License, Derivative Works shall not include works that remain
- separable from, or merely link (or bind by name) to the interfaces of,
- the Work and Derivative Works thereof.
-
- "Contribution" shall mean any work of authorship, including
- the original version of the Work and any modifications or additions
- to that Work or Derivative Works thereof, that is intentionally
- submitted to Licensor for inclusion in the Work by the copyright owner
- or by an individual or Legal Entity authorized to submit on behalf of
- the copyright owner. For the purposes of this definition, "submitted"
- means any form of electronic, verbal, or written communication sent
- to the Licensor or its representatives, including but not limited to
- communication on electronic mailing lists, source code control systems,
- and issue tracking systems that are managed by, or on behalf of, the
- Licensor for the purpose of discussing and improving the Work, but
- excluding communication that is conspicuously marked or otherwise
- designated in writing by the copyright owner as "Not a Contribution."
-
- "Contributor" shall mean Licensor and any individual or Legal Entity
- on behalf of whom a Contribution has been received by Licensor and
- subsequently incorporated within the Work.
-
- 2. Grant of Copyright License. Subject to the terms and conditions of
- this License, each Contributor hereby grants to You a perpetual,
- worldwide, non-exclusive, no-charge, royalty-free, irrevocable
- copyright license to reproduce, prepare Derivative Works of,
- publicly display, publicly perform, sublicense, and distribute the
- Work and such Derivative Works in Source or Object form.
-
- 3. Grant of Patent License. Subject to the terms and conditions of
- this License, each Contributor hereby grants to You a perpetual,
- worldwide, non-exclusive, no-charge, royalty-free, irrevocable
- (except as stated in this section) patent license to make, have made,
- use, offer to sell, sell, import, and otherwise transfer the Work,
- where such license applies only to those patent claims licensable
- by such Contributor that are necessarily infringed by their
- Contribution(s) alone or by combination of their Contribution(s)
- with the Work to which such Contribution(s) was submitted. If You
- institute patent litigation against any entity (including a
- cross-claim or counterclaim in a lawsuit) alleging that the Work
- or a Contribution incorporated within the Work constitutes direct
- or contributory patent infringement, then any patent licenses
- granted to You under this License for that Work shall terminate
- as of the date such litigation is filed.
-
- 4. Redistribution. You may reproduce and distribute copies of the
- Work or Derivative Works thereof in any medium, with or without
- modifications, and in Source or Object form, provided that You
- meet the following conditions:
-
- (a) You must give any other recipients of the Work or
- Derivative Works a copy of this License; and
-
- (b) You must cause any modified files to carry prominent notices
- stating that You changed the files; and
-
- (c) You must retain, in the Source form of any Derivative Works
- that You distribute, all copyright, patent, trademark, and
- attribution notices from the Source form of the Work,
- excluding those notices that do not pertain to any part of
- the Derivative Works; and
-
- (d) If the Work includes a "NOTICE" text file as part of its
- distribution, then any Derivative Works that You distribute must
- include a readable copy of the attribution notices contained
- within such NOTICE file, excluding those notices that do not
- pertain to any part of the Derivative Works, in at least one
- of the following places: within a NOTICE text file distributed
- as part of the Derivative Works; within the Source form or
- documentation, if provided along with the Derivative Works; or,
- within a display generated by the Derivative Works, if and
- wherever such third-party notices normally appear. The contents
- of the NOTICE file are for informational purposes only and
- do not modify the License. You may add Your own attribution
- notices within Derivative Works that You distribute, alongside
- or as an addendum to the NOTICE text from the Work, provided
- that such additional attribution notices cannot be construed
- as modifying the License.
-
- You may add Your own copyright statement to Your modifications and
- may provide additional or different license terms and conditions
- for use, reproduction, or distribution of Your modifications, or
- for any such Derivative Works as a whole, provided Your use,
- reproduction, and distribution of the Work otherwise complies with
- the conditions stated in this License.
-
- 5. Submission of Contributions. Unless You explicitly state otherwise,
- any Contribution intentionally submitted for inclusion in the Work
- by You to the Licensor shall be under the terms and conditions of
- this License, without any additional terms or conditions.
- Notwithstanding the above, nothing herein shall supersede or modify
- the terms of any separate license agreement you may have executed
- with Licensor regarding such Contributions.
-
- 6. Trademarks. This License does not grant permission to use the trade
- names, trademarks, service marks, or product names of the Licensor,
- except as required for reasonable and customary use in describing the
- origin of the Work and reproducing the content of the NOTICE file.
-
- 7. Disclaimer of Warranty. Unless required by applicable law or
- agreed to in writing, Licensor provides the Work (and each
- Contributor provides its Contributions) on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
- implied, including, without limitation, any warranties or conditions
- of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
- PARTICULAR PURPOSE. You are solely responsible for determining the
- appropriateness of using or redistributing the Work and assume any
- risks associated with Your exercise of permissions under this License.
-
- 8. Limitation of Liability. In no event and under no legal theory,
- whether in tort (including negligence), contract, or otherwise,
- unless required by applicable law (such as deliberate and grossly
- negligent acts) or agreed to in writing, shall any Contributor be
- liable to You for damages, including any direct, indirect, special,
- incidental, or consequential damages of any character arising as a
- result of this License or out of the use or inability to use the
- Work (including but not limited to damages for loss of goodwill,
- work stoppage, computer failure or malfunction, or any and all
- other commercial damages or losses), even if such Contributor
- has been advised of the possibility of such damages.
-
- 9. Accepting Warranty or Additional Liability. While redistributing
- the Work or Derivative Works thereof, You may choose to offer,
- and charge a fee for, acceptance of support, warranty, indemnity,
- or other liability obligations and/or rights consistent with this
- License. However, in accepting such obligations, You may act only
- on Your own behalf and on Your sole responsibility, not on behalf
- of any other Contributor, and only if You agree to indemnify,
- defend, and hold each Contributor harmless for any liability
- incurred by, or claims asserted against, such Contributor by reason
- of your accepting any such warranty or additional liability.
-
- END OF TERMS AND CONDITIONS
-
- APPENDIX: How to apply the Apache License to your work.
-
- To apply the Apache License to your work, attach the following
- boilerplate notice, with the fields enclosed by brackets "[]"
- replaced with your own identifying information. (Don't include
- the brackets!) The text should be enclosed in the appropriate
- comment syntax for the file format. We also recommend that a
- file or class name and description of purpose be included on the
- same "printed page" as the copyright notice for easier
- identification within third-party archives.
-
- Copyright [yyyy] [name of copyright owner]
-
- Licensed under the Apache License, Version 2.0 (the "License");
- you may not use this file except in compliance with the License.
- You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_google-oauth-java-client_v1.32.1_LICENSE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_google-oauth-java-client_v1.32.1_LICENSE.txt
deleted file mode 100644
index f49a4e1..0000000
--- a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_google-oauth-java-client_v1.32.1_LICENSE.txt
+++ /dev/null
@@ -1,201 +0,0 @@
- Apache License
- Version 2.0, January 2004
- http://www.apache.org/licenses/
-
- TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
-
- 1. Definitions.
-
- "License" shall mean the terms and conditions for use, reproduction,
- and distribution as defined by Sections 1 through 9 of this document.
-
- "Licensor" shall mean the copyright owner or entity authorized by
- the copyright owner that is granting the License.
-
- "Legal Entity" shall mean the union of the acting entity and all
- other entities that control, are controlled by, or are under common
- control with that entity. For the purposes of this definition,
- "control" means (i) the power, direct or indirect, to cause the
- direction or management of such entity, whether by contract or
- otherwise, or (ii) ownership of fifty percent (50%) or more of the
- outstanding shares, or (iii) beneficial ownership of such entity.
-
- "You" (or "Your") shall mean an individual or Legal Entity
- exercising permissions granted by this License.
-
- "Source" form shall mean the preferred form for making modifications,
- including but not limited to software source code, documentation
- source, and configuration files.
-
- "Object" form shall mean any form resulting from mechanical
- transformation or translation of a Source form, including but
- not limited to compiled object code, generated documentation,
- and conversions to other media types.
-
- "Work" shall mean the work of authorship, whether in Source or
- Object form, made available under the License, as indicated by a
- copyright notice that is included in or attached to the work
- (an example is provided in the Appendix below).
-
- "Derivative Works" shall mean any work, whether in Source or Object
- form, that is based on (or derived from) the Work and for which the
- editorial revisions, annotations, elaborations, or other modifications
- represent, as a whole, an original work of authorship. For the purposes
- of this License, Derivative Works shall not include works that remain
- separable from, or merely link (or bind by name) to the interfaces of,
- the Work and Derivative Works thereof.
-
- "Contribution" shall mean any work of authorship, including
- the original version of the Work and any modifications or additions
- to that Work or Derivative Works thereof, that is intentionally
- submitted to Licensor for inclusion in the Work by the copyright owner
- or by an individual or Legal Entity authorized to submit on behalf of
- the copyright owner. For the purposes of this definition, "submitted"
- means any form of electronic, verbal, or written communication sent
- to the Licensor or its representatives, including but not limited to
- communication on electronic mailing lists, source code control systems,
- and issue tracking systems that are managed by, or on behalf of, the
- Licensor for the purpose of discussing and improving the Work, but
- excluding communication that is conspicuously marked or otherwise
- designated in writing by the copyright owner as "Not a Contribution."
-
- "Contributor" shall mean Licensor and any individual or Legal Entity
- on behalf of whom a Contribution has been received by Licensor and
- subsequently incorporated within the Work.
-
- 2. Grant of Copyright License. Subject to the terms and conditions of
- this License, each Contributor hereby grants to You a perpetual,
- worldwide, non-exclusive, no-charge, royalty-free, irrevocable
- copyright license to reproduce, prepare Derivative Works of,
- publicly display, publicly perform, sublicense, and distribute the
- Work and such Derivative Works in Source or Object form.
-
- 3. Grant of Patent License. Subject to the terms and conditions of
- this License, each Contributor hereby grants to You a perpetual,
- worldwide, non-exclusive, no-charge, royalty-free, irrevocable
- (except as stated in this section) patent license to make, have made,
- use, offer to sell, sell, import, and otherwise transfer the Work,
- where such license applies only to those patent claims licensable
- by such Contributor that are necessarily infringed by their
- Contribution(s) alone or by combination of their Contribution(s)
- with the Work to which such Contribution(s) was submitted. If You
- institute patent litigation against any entity (including a
- cross-claim or counterclaim in a lawsuit) alleging that the Work
- or a Contribution incorporated within the Work constitutes direct
- or contributory patent infringement, then any patent licenses
- granted to You under this License for that Work shall terminate
- as of the date such litigation is filed.
-
- 4. Redistribution. You may reproduce and distribute copies of the
- Work or Derivative Works thereof in any medium, with or without
- modifications, and in Source or Object form, provided that You
- meet the following conditions:
-
- (a) You must give any other recipients of the Work or
- Derivative Works a copy of this License; and
-
- (b) You must cause any modified files to carry prominent notices
- stating that You changed the files; and
-
- (c) You must retain, in the Source form of any Derivative Works
- that You distribute, all copyright, patent, trademark, and
- attribution notices from the Source form of the Work,
- excluding those notices that do not pertain to any part of
- the Derivative Works; and
-
- (d) If the Work includes a "NOTICE" text file as part of its
- distribution, then any Derivative Works that You distribute must
- include a readable copy of the attribution notices contained
- within such NOTICE file, excluding those notices that do not
- pertain to any part of the Derivative Works, in at least one
- of the following places: within a NOTICE text file distributed
- as part of the Derivative Works; within the Source form or
- documentation, if provided along with the Derivative Works; or,
- within a display generated by the Derivative Works, if and
- wherever such third-party notices normally appear. The contents
- of the NOTICE file are for informational purposes only and
- do not modify the License. You may add Your own attribution
- notices within Derivative Works that You distribute, alongside
- or as an addendum to the NOTICE text from the Work, provided
- that such additional attribution notices cannot be construed
- as modifying the License.
-
- You may add Your own copyright statement to Your modifications and
- may provide additional or different license terms and conditions
- for use, reproduction, or distribution of Your modifications, or
- for any such Derivative Works as a whole, provided Your use,
- reproduction, and distribution of the Work otherwise complies with
- the conditions stated in this License.
-
- 5. Submission of Contributions. Unless You explicitly state otherwise,
- any Contribution intentionally submitted for inclusion in the Work
- by You to the Licensor shall be under the terms and conditions of
- this License, without any additional terms or conditions.
- Notwithstanding the above, nothing herein shall supersede or modify
- the terms of any separate license agreement you may have executed
- with Licensor regarding such Contributions.
-
- 6. Trademarks. This License does not grant permission to use the trade
- names, trademarks, service marks, or product names of the Licensor,
- except as required for reasonable and customary use in describing the
- origin of the Work and reproducing the content of the NOTICE file.
-
- 7. Disclaimer of Warranty. Unless required by applicable law or
- agreed to in writing, Licensor provides the Work (and each
- Contributor provides its Contributions) on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
- implied, including, without limitation, any warranties or conditions
- of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
- PARTICULAR PURPOSE. You are solely responsible for determining the
- appropriateness of using or redistributing the Work and assume any
- risks associated with Your exercise of permissions under this License.
-
- 8. Limitation of Liability. In no event and under no legal theory,
- whether in tort (including negligence), contract, or otherwise,
- unless required by applicable law (such as deliberate and grossly
- negligent acts) or agreed to in writing, shall any Contributor be
- liable to You for damages, including any direct, indirect, special,
- incidental, or consequential damages of any character arising as a
- result of this License or out of the use or inability to use the
- Work (including but not limited to damages for loss of goodwill,
- work stoppage, computer failure or malfunction, or any and all
- other commercial damages or losses), even if such Contributor
- has been advised of the possibility of such damages.
-
- 9. Accepting Warranty or Additional Liability. While redistributing
- the Work or Derivative Works thereof, You may choose to offer,
- and charge a fee for, acceptance of support, warranty, indemnity,
- or other liability obligations and/or rights consistent with this
- License. However, in accepting such obligations, You may act only
- on Your own behalf and on Your sole responsibility, not on behalf
- of any other Contributor, and only if You agree to indemnify,
- defend, and hold each Contributor harmless for any liability
- incurred by, or claims asserted against, such Contributor by reason
- of your accepting any such warranty or additional liability.
-
- END OF TERMS AND CONDITIONS
-
- APPENDIX: How to apply the Apache License to your work.
-
- To apply the Apache License to your work, attach the following
- boilerplate notice, with the fields enclosed by brackets "[]"
- replaced with your own identifying information. (Don't include
- the brackets!) The text should be enclosed in the appropriate
- comment syntax for the file format. We also recommend that a
- file or class name and description of purpose be included on the
- same "printed page" as the copyright notice for easier
- identification within third-party archives.
-
- Copyright [yyyy] [name of copyright owner]
-
- Licensed under the Apache License, Version 2.0 (the "License");
- you may not use this file except in compliance with the License.
- You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_java-core_v2.3.5_LICENSE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_java-core_v2.3.5_LICENSE.txt
deleted file mode 100644
index f49a4e1..0000000
--- a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_java-core_v2.3.5_LICENSE.txt
+++ /dev/null
@@ -1,201 +0,0 @@
- Apache License
- Version 2.0, January 2004
- http://www.apache.org/licenses/
-
- TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
-
- 1. Definitions.
-
- "License" shall mean the terms and conditions for use, reproduction,
- and distribution as defined by Sections 1 through 9 of this document.
-
- "Licensor" shall mean the copyright owner or entity authorized by
- the copyright owner that is granting the License.
-
- "Legal Entity" shall mean the union of the acting entity and all
- other entities that control, are controlled by, or are under common
- control with that entity. For the purposes of this definition,
- "control" means (i) the power, direct or indirect, to cause the
- direction or management of such entity, whether by contract or
- otherwise, or (ii) ownership of fifty percent (50%) or more of the
- outstanding shares, or (iii) beneficial ownership of such entity.
-
- "You" (or "Your") shall mean an individual or Legal Entity
- exercising permissions granted by this License.
-
- "Source" form shall mean the preferred form for making modifications,
- including but not limited to software source code, documentation
- source, and configuration files.
-
- "Object" form shall mean any form resulting from mechanical
- transformation or translation of a Source form, including but
- not limited to compiled object code, generated documentation,
- and conversions to other media types.
-
- "Work" shall mean the work of authorship, whether in Source or
- Object form, made available under the License, as indicated by a
- copyright notice that is included in or attached to the work
- (an example is provided in the Appendix below).
-
- "Derivative Works" shall mean any work, whether in Source or Object
- form, that is based on (or derived from) the Work and for which the
- editorial revisions, annotations, elaborations, or other modifications
- represent, as a whole, an original work of authorship. For the purposes
- of this License, Derivative Works shall not include works that remain
- separable from, or merely link (or bind by name) to the interfaces of,
- the Work and Derivative Works thereof.
-
- "Contribution" shall mean any work of authorship, including
- the original version of the Work and any modifications or additions
- to that Work or Derivative Works thereof, that is intentionally
- submitted to Licensor for inclusion in the Work by the copyright owner
- or by an individual or Legal Entity authorized to submit on behalf of
- the copyright owner. For the purposes of this definition, "submitted"
- means any form of electronic, verbal, or written communication sent
- to the Licensor or its representatives, including but not limited to
- communication on electronic mailing lists, source code control systems,
- and issue tracking systems that are managed by, or on behalf of, the
- Licensor for the purpose of discussing and improving the Work, but
- excluding communication that is conspicuously marked or otherwise
- designated in writing by the copyright owner as "Not a Contribution."
-
- "Contributor" shall mean Licensor and any individual or Legal Entity
- on behalf of whom a Contribution has been received by Licensor and
- subsequently incorporated within the Work.
-
- 2. Grant of Copyright License. Subject to the terms and conditions of
- this License, each Contributor hereby grants to You a perpetual,
- worldwide, non-exclusive, no-charge, royalty-free, irrevocable
- copyright license to reproduce, prepare Derivative Works of,
- publicly display, publicly perform, sublicense, and distribute the
- Work and such Derivative Works in Source or Object form.
-
- 3. Grant of Patent License. Subject to the terms and conditions of
- this License, each Contributor hereby grants to You a perpetual,
- worldwide, non-exclusive, no-charge, royalty-free, irrevocable
- (except as stated in this section) patent license to make, have made,
- use, offer to sell, sell, import, and otherwise transfer the Work,
- where such license applies only to those patent claims licensable
- by such Contributor that are necessarily infringed by their
- Contribution(s) alone or by combination of their Contribution(s)
- with the Work to which such Contribution(s) was submitted. If You
- institute patent litigation against any entity (including a
- cross-claim or counterclaim in a lawsuit) alleging that the Work
- or a Contribution incorporated within the Work constitutes direct
- or contributory patent infringement, then any patent licenses
- granted to You under this License for that Work shall terminate
- as of the date such litigation is filed.
-
- 4. Redistribution. You may reproduce and distribute copies of the
- Work or Derivative Works thereof in any medium, with or without
- modifications, and in Source or Object form, provided that You
- meet the following conditions:
-
- (a) You must give any other recipients of the Work or
- Derivative Works a copy of this License; and
-
- (b) You must cause any modified files to carry prominent notices
- stating that You changed the files; and
-
- (c) You must retain, in the Source form of any Derivative Works
- that You distribute, all copyright, patent, trademark, and
- attribution notices from the Source form of the Work,
- excluding those notices that do not pertain to any part of
- the Derivative Works; and
-
- (d) If the Work includes a "NOTICE" text file as part of its
- distribution, then any Derivative Works that You distribute must
- include a readable copy of the attribution notices contained
- within such NOTICE file, excluding those notices that do not
- pertain to any part of the Derivative Works, in at least one
- of the following places: within a NOTICE text file distributed
- as part of the Derivative Works; within the Source form or
- documentation, if provided along with the Derivative Works; or,
- within a display generated by the Derivative Works, if and
- wherever such third-party notices normally appear. The contents
- of the NOTICE file are for informational purposes only and
- do not modify the License. You may add Your own attribution
- notices within Derivative Works that You distribute, alongside
- or as an addendum to the NOTICE text from the Work, provided
- that such additional attribution notices cannot be construed
- as modifying the License.
-
- You may add Your own copyright statement to Your modifications and
- may provide additional or different license terms and conditions
- for use, reproduction, or distribution of Your modifications, or
- for any such Derivative Works as a whole, provided Your use,
- reproduction, and distribution of the Work otherwise complies with
- the conditions stated in this License.
-
- 5. Submission of Contributions. Unless You explicitly state otherwise,
- any Contribution intentionally submitted for inclusion in the Work
- by You to the Licensor shall be under the terms and conditions of
- this License, without any additional terms or conditions.
- Notwithstanding the above, nothing herein shall supersede or modify
- the terms of any separate license agreement you may have executed
- with Licensor regarding such Contributions.
-
- 6. Trademarks. This License does not grant permission to use the trade
- names, trademarks, service marks, or product names of the Licensor,
- except as required for reasonable and customary use in describing the
- origin of the Work and reproducing the content of the NOTICE file.
-
- 7. Disclaimer of Warranty. Unless required by applicable law or
- agreed to in writing, Licensor provides the Work (and each
- Contributor provides its Contributions) on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
- implied, including, without limitation, any warranties or conditions
- of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
- PARTICULAR PURPOSE. You are solely responsible for determining the
- appropriateness of using or redistributing the Work and assume any
- risks associated with Your exercise of permissions under this License.
-
- 8. Limitation of Liability. In no event and under no legal theory,
- whether in tort (including negligence), contract, or otherwise,
- unless required by applicable law (such as deliberate and grossly
- negligent acts) or agreed to in writing, shall any Contributor be
- liable to You for damages, including any direct, indirect, special,
- incidental, or consequential damages of any character arising as a
- result of this License or out of the use or inability to use the
- Work (including but not limited to damages for loss of goodwill,
- work stoppage, computer failure or malfunction, or any and all
- other commercial damages or losses), even if such Contributor
- has been advised of the possibility of such damages.
-
- 9. Accepting Warranty or Additional Liability. While redistributing
- the Work or Derivative Works thereof, You may choose to offer,
- and charge a fee for, acceptance of support, warranty, indemnity,
- or other liability obligations and/or rights consistent with this
- License. However, in accepting such obligations, You may act only
- on Your own behalf and on Your sole responsibility, not on behalf
- of any other Contributor, and only if You agree to indemnify,
- defend, and hold each Contributor harmless for any liability
- incurred by, or claims asserted against, such Contributor by reason
- of your accepting any such warranty or additional liability.
-
- END OF TERMS AND CONDITIONS
-
- APPENDIX: How to apply the Apache License to your work.
-
- To apply the Apache License to your work, attach the following
- boilerplate notice, with the fields enclosed by brackets "[]"
- replaced with your own identifying information. (Don't include
- the brackets!) The text should be enclosed in the appropriate
- comment syntax for the file format. We also recommend that a
- file or class name and description of purpose be included on the
- same "printed page" as the copyright notice for easier
- identification within third-party archives.
-
- Copyright [yyyy] [name of copyright owner]
-
- Licensed under the Apache License, Version 2.0 (the "License");
- you may not use this file except in compliance with the License.
- You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_grpc_grpc-java_v1.43.2_LICENSE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_grpc_grpc-java_v1.43.2_LICENSE.txt
deleted file mode 100644
index 7a4a3ea..0000000
--- a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_grpc_grpc-java_v1.43.2_LICENSE.txt
+++ /dev/null
@@ -1,202 +0,0 @@
-
- Apache License
- Version 2.0, January 2004
- http://www.apache.org/licenses/
-
- TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
-
- 1. Definitions.
-
- "License" shall mean the terms and conditions for use, reproduction,
- and distribution as defined by Sections 1 through 9 of this document.
-
- "Licensor" shall mean the copyright owner or entity authorized by
- the copyright owner that is granting the License.
-
- "Legal Entity" shall mean the union of the acting entity and all
- other entities that control, are controlled by, or are under common
- control with that entity. For the purposes of this definition,
- "control" means (i) the power, direct or indirect, to cause the
- direction or management of such entity, whether by contract or
- otherwise, or (ii) ownership of fifty percent (50%) or more of the
- outstanding shares, or (iii) beneficial ownership of such entity.
-
- "You" (or "Your") shall mean an individual or Legal Entity
- exercising permissions granted by this License.
-
- "Source" form shall mean the preferred form for making modifications,
- including but not limited to software source code, documentation
- source, and configuration files.
-
- "Object" form shall mean any form resulting from mechanical
- transformation or translation of a Source form, including but
- not limited to compiled object code, generated documentation,
- and conversions to other media types.
-
- "Work" shall mean the work of authorship, whether in Source or
- Object form, made available under the License, as indicated by a
- copyright notice that is included in or attached to the work
- (an example is provided in the Appendix below).
-
- "Derivative Works" shall mean any work, whether in Source or Object
- form, that is based on (or derived from) the Work and for which the
- editorial revisions, annotations, elaborations, or other modifications
- represent, as a whole, an original work of authorship. For the purposes
- of this License, Derivative Works shall not include works that remain
- separable from, or merely link (or bind by name) to the interfaces of,
- the Work and Derivative Works thereof.
-
- "Contribution" shall mean any work of authorship, including
- the original version of the Work and any modifications or additions
- to that Work or Derivative Works thereof, that is intentionally
- submitted to Licensor for inclusion in the Work by the copyright owner
- or by an individual or Legal Entity authorized to submit on behalf of
- the copyright owner. For the purposes of this definition, "submitted"
- means any form of electronic, verbal, or written communication sent
- to the Licensor or its representatives, including but not limited to
- communication on electronic mailing lists, source code control systems,
- and issue tracking systems that are managed by, or on behalf of, the
- Licensor for the purpose of discussing and improving the Work, but
- excluding communication that is conspicuously marked or otherwise
- designated in writing by the copyright owner as "Not a Contribution."
-
- "Contributor" shall mean Licensor and any individual or Legal Entity
- on behalf of whom a Contribution has been received by Licensor and
- subsequently incorporated within the Work.
-
- 2. Grant of Copyright License. Subject to the terms and conditions of
- this License, each Contributor hereby grants to You a perpetual,
- worldwide, non-exclusive, no-charge, royalty-free, irrevocable
- copyright license to reproduce, prepare Derivative Works of,
- publicly display, publicly perform, sublicense, and distribute the
- Work and such Derivative Works in Source or Object form.
-
- 3. Grant of Patent License. Subject to the terms and conditions of
- this License, each Contributor hereby grants to You a perpetual,
- worldwide, non-exclusive, no-charge, royalty-free, irrevocable
- (except as stated in this section) patent license to make, have made,
- use, offer to sell, sell, import, and otherwise transfer the Work,
- where such license applies only to those patent claims licensable
- by such Contributor that are necessarily infringed by their
- Contribution(s) alone or by combination of their Contribution(s)
- with the Work to which such Contribution(s) was submitted. If You
- institute patent litigation against any entity (including a
- cross-claim or counterclaim in a lawsuit) alleging that the Work
- or a Contribution incorporated within the Work constitutes direct
- or contributory patent infringement, then any patent licenses
- granted to You under this License for that Work shall terminate
- as of the date such litigation is filed.
-
- 4. Redistribution. You may reproduce and distribute copies of the
- Work or Derivative Works thereof in any medium, with or without
- modifications, and in Source or Object form, provided that You
- meet the following conditions:
-
- (a) You must give any other recipients of the Work or
- Derivative Works a copy of this License; and
-
- (b) You must cause any modified files to carry prominent notices
- stating that You changed the files; and
-
- (c) You must retain, in the Source form of any Derivative Works
- that You distribute, all copyright, patent, trademark, and
- attribution notices from the Source form of the Work,
- excluding those notices that do not pertain to any part of
- the Derivative Works; and
-
- (d) If the Work includes a "NOTICE" text file as part of its
- distribution, then any Derivative Works that You distribute must
- include a readable copy of the attribution notices contained
- within such NOTICE file, excluding those notices that do not
- pertain to any part of the Derivative Works, in at least one
- of the following places: within a NOTICE text file distributed
- as part of the Derivative Works; within the Source form or
- documentation, if provided along with the Derivative Works; or,
- within a display generated by the Derivative Works, if and
- wherever such third-party notices normally appear. The contents
- of the NOTICE file are for informational purposes only and
- do not modify the License. You may add Your own attribution
- notices within Derivative Works that You distribute, alongside
- or as an addendum to the NOTICE text from the Work, provided
- that such additional attribution notices cannot be construed
- as modifying the License.
-
- You may add Your own copyright statement to Your modifications and
- may provide additional or different license terms and conditions
- for use, reproduction, or distribution of Your modifications, or
- for any such Derivative Works as a whole, provided Your use,
- reproduction, and distribution of the Work otherwise complies with
- the conditions stated in this License.
-
- 5. Submission of Contributions. Unless You explicitly state otherwise,
- any Contribution intentionally submitted for inclusion in the Work
- by You to the Licensor shall be under the terms and conditions of
- this License, without any additional terms or conditions.
- Notwithstanding the above, nothing herein shall supersede or modify
- the terms of any separate license agreement you may have executed
- with Licensor regarding such Contributions.
-
- 6. Trademarks. This License does not grant permission to use the trade
- names, trademarks, service marks, or product names of the Licensor,
- except as required for reasonable and customary use in describing the
- origin of the Work and reproducing the content of the NOTICE file.
-
- 7. Disclaimer of Warranty. Unless required by applicable law or
- agreed to in writing, Licensor provides the Work (and each
- Contributor provides its Contributions) on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
- implied, including, without limitation, any warranties or conditions
- of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
- PARTICULAR PURPOSE. You are solely responsible for determining the
- appropriateness of using or redistributing the Work and assume any
- risks associated with Your exercise of permissions under this License.
-
- 8. Limitation of Liability. In no event and under no legal theory,
- whether in tort (including negligence), contract, or otherwise,
- unless required by applicable law (such as deliberate and grossly
- negligent acts) or agreed to in writing, shall any Contributor be
- liable to You for damages, including any direct, indirect, special,
- incidental, or consequential damages of any character arising as a
- result of this License or out of the use or inability to use the
- Work (including but not limited to damages for loss of goodwill,
- work stoppage, computer failure or malfunction, or any and all
- other commercial damages or losses), even if such Contributor
- has been advised of the possibility of such damages.
-
- 9. Accepting Warranty or Additional Liability. While redistributing
- the Work or Derivative Works thereof, You may choose to offer,
- and charge a fee for, acceptance of support, warranty, indemnity,
- or other liability obligations and/or rights consistent with this
- License. However, in accepting such obligations, You may act only
- on Your own behalf and on Your sole responsibility, not on behalf
- of any other Contributor, and only if You agree to indemnify,
- defend, and hold each Contributor harmless for any liability
- incurred by, or claims asserted against, such Contributor by reason
- of your accepting any such warranty or additional liability.
-
- END OF TERMS AND CONDITIONS
-
- APPENDIX: How to apply the Apache License to your work.
-
- To apply the Apache License to your work, attach the following
- boilerplate notice, with the fields enclosed by brackets "[]"
- replaced with your own identifying information. (Don't include
- the brackets!) The text should be enclosed in the appropriate
- comment syntax for the file format. We also recommend that a
- file or class name and description of purpose be included on the
- same "printed page" as the copyright notice for easier
- identification within third-party archives.
-
- Copyright [yyyy] [name of copyright owner]
-
- Licensed under the Apache License, Version 2.0 (the "License");
- you may not use this file except in compliance with the License.
- You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_grpc_grpc-java_v1.50.2_NOTICE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_grpc_grpc-java_v1.50.2_NOTICE.txt
new file mode 100644
index 0000000..c5d3ec2
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_grpc_grpc-java_v1.50.2_NOTICE.txt
@@ -0,0 +1,62 @@
+Copyright 2014 The gRPC 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.
+
+-----------------------------------------------------------------------
+
+This product contains a modified portion of 'OkHttp', an open source
+HTTP & SPDY client for Android and Java applications, which can be obtained
+at:
+
+ * LICENSE:
+ * okhttp/third_party/okhttp/LICENSE (Apache License 2.0)
+ * HOMEPAGE:
+ * https://github.com/square/okhttp
+ * LOCATION_IN_GRPC:
+ * okhttp/third_party/okhttp
+
+This product contains a modified portion of 'Envoy', an open source
+cloud-native high-performance edge/middle/service proxy, which can be
+obtained at:
+
+ * LICENSE:
+ * xds/third_party/envoy/LICENSE (Apache License 2.0)
+ * NOTICE:
+ * xds/third_party/envoy/NOTICE
+ * HOMEPAGE:
+ * https://www.envoyproxy.io
+ * LOCATION_IN_GRPC:
+ * xds/third_party/envoy
+
+This product contains a modified portion of 'protoc-gen-validate (PGV)',
+an open source protoc plugin to generate polyglot message validators,
+which can be obtained at:
+
+ * LICENSE:
+ * xds/third_party/protoc-gen-validate/LICENSE (Apache License 2.0)
+ * NOTICE:
+ * xds/third_party/protoc-gen-validate/NOTICE
+ * HOMEPAGE:
+ * https://github.com/envoyproxy/protoc-gen-validate
+ * LOCATION_IN_GRPC:
+ * xds/third_party/protoc-gen-validate
+
+This product contains a modified portion of 'udpa',
+an open source universal data plane API, which can be obtained at:
+
+ * LICENSE:
+ * xds/third_party/udpa/LICENSE (Apache License 2.0)
+ * HOMEPAGE:
+ * https://github.com/cncf/udpa
+ * LOCATION_IN_GRPC:
+ * xds/third_party/udpa
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_grpc_grpc-java_v1.52.1_NOTICE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_grpc_grpc-java_v1.52.1_NOTICE.txt
new file mode 100644
index 0000000..c5d3ec2
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_grpc_grpc-java_v1.52.1_NOTICE.txt
@@ -0,0 +1,62 @@
+Copyright 2014 The gRPC 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.
+
+-----------------------------------------------------------------------
+
+This product contains a modified portion of 'OkHttp', an open source
+HTTP & SPDY client for Android and Java applications, which can be obtained
+at:
+
+ * LICENSE:
+ * okhttp/third_party/okhttp/LICENSE (Apache License 2.0)
+ * HOMEPAGE:
+ * https://github.com/square/okhttp
+ * LOCATION_IN_GRPC:
+ * okhttp/third_party/okhttp
+
+This product contains a modified portion of 'Envoy', an open source
+cloud-native high-performance edge/middle/service proxy, which can be
+obtained at:
+
+ * LICENSE:
+ * xds/third_party/envoy/LICENSE (Apache License 2.0)
+ * NOTICE:
+ * xds/third_party/envoy/NOTICE
+ * HOMEPAGE:
+ * https://www.envoyproxy.io
+ * LOCATION_IN_GRPC:
+ * xds/third_party/envoy
+
+This product contains a modified portion of 'protoc-gen-validate (PGV)',
+an open source protoc plugin to generate polyglot message validators,
+which can be obtained at:
+
+ * LICENSE:
+ * xds/third_party/protoc-gen-validate/LICENSE (Apache License 2.0)
+ * NOTICE:
+ * xds/third_party/protoc-gen-validate/NOTICE
+ * HOMEPAGE:
+ * https://github.com/envoyproxy/protoc-gen-validate
+ * LOCATION_IN_GRPC:
+ * xds/third_party/protoc-gen-validate
+
+This product contains a modified portion of 'udpa',
+an open source universal data plane API, which can be obtained at:
+
+ * LICENSE:
+ * xds/third_party/udpa/LICENSE (Apache License 2.0)
+ * HOMEPAGE:
+ * https://github.com/cncf/udpa
+ * LOCATION_IN_GRPC:
+ * xds/third_party/udpa
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_luben_zstd-jni_v1.4.9-1_LICENSE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_luben_zstd-jni_v1.5.0-1_LICENSE.txt
similarity index 100%
rename from asterixdb/src/main/licenses/content/raw.githubusercontent.com_luben_zstd-jni_v1.4.9-1_LICENSE.txt
rename to asterixdb/src/main/licenses/content/raw.githubusercontent.com_luben_zstd-jni_v1.5.0-1_LICENSE.txt
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_netty_netty_netty-4.1.73.Final_NOTICE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_netty_netty_netty-4.1.87.Final_NOTICE.txt
similarity index 100%
rename from asterixdb/src/main/licenses/content/raw.githubusercontent.com_netty_netty_netty-4.1.73.Final_NOTICE.txt
rename to asterixdb/src/main/licenses/content/raw.githubusercontent.com_netty_netty_netty-4.1.87.Final_NOTICE.txt
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_perfmark_perfmark_v0.23.0_NOTICE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_perfmark_perfmark_v0.23.0_NOTICE.txt
new file mode 100644
index 0000000..3ed2d12
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_perfmark_perfmark_v0.23.0_NOTICE.txt
@@ -0,0 +1,32 @@
+
+Copyright 2019 Google LLC
+
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+-----------------------------------------------------------------------
+
+This product contains a modified portion of 'Catapult', an open source
+Trace Event viewer for Chome, Linux, and Android applications, which can
+be obtained at:
+
+ * LICENSE:
+ * traceviewer/src/main/resources/io/perfmark/traceviewer/third_party/catapult/LICENSE (New BSD License)
+ * HOMEPAGE:
+ * https://github.com/catapult-project/catapult
+
+This product contains a modified portion of 'Polymer', a library for Web
+Components, which can be obtained at:
+ * LICENSE:
+ * traceviewer/src/main/resources/io/perfmark/traceviewer/third_party/polymer/LICENSE (New BSD License)
+ * HOMEPAGE:
+ * https://github.com/Polymer/polymer
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_perfmark_perfmark_v0.26.0_NOTICE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_perfmark_perfmark_v0.26.0_NOTICE.txt
new file mode 100644
index 0000000..7349754
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_perfmark_perfmark_v0.26.0_NOTICE.txt
@@ -0,0 +1,40 @@
+Copyright 2019 Google LLC
+
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+-----------------------------------------------------------------------
+
+This product contains a modified portion of 'Catapult', an open source
+Trace Event viewer for Chome, Linux, and Android applications, which can
+be obtained at:
+
+ * LICENSE:
+ * traceviewer/src/main/resources/io/perfmark/traceviewer/third_party/catapult/LICENSE (New BSD License)
+ * HOMEPAGE:
+ * https://github.com/catapult-project/catapult
+
+This product contains a modified portion of 'Polymer', a library for Web
+Components, which can be obtained at:
+ * LICENSE:
+ * traceviewer/src/main/resources/io/perfmark/traceviewer/third_party/polymer/LICENSE (New BSD License)
+ * HOMEPAGE:
+ * https://github.com/Polymer/polymer
+
+
+This product contains a modified portion of 'ASM', an open source
+Java Bytecode library, which can be obtained at:
+
+ * LICENSE:
+ * agent/src/main/resources/io/perfmark/agent/third_party/asm/LICENSE (BSD style License)
+ * HOMEPAGE:
+ * https://asm.ow2.io/
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_protocolbuffers_protobuf_v3.19.2_LICENSE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_protocolbuffers_protobuf_v3.21.12_LICENSE.txt
similarity index 100%
rename from asterixdb/src/main/licenses/content/raw.githubusercontent.com_protocolbuffers_protobuf_v3.19.2_LICENSE.txt
rename to asterixdb/src/main/licenses/content/raw.githubusercontent.com_protocolbuffers_protobuf_v3.21.12_LICENSE.txt
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_qos-ch_slf4j_v_1.7.36_LICENSE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_qos-ch_slf4j_v_1.7.36_LICENSE.txt
new file mode 100644
index 0000000..f687729
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_qos-ch_slf4j_v_1.7.36_LICENSE.txt
@@ -0,0 +1,21 @@
+Copyright (c) 2004-2022 QOS.ch Sarl (Switzerland)
+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.
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_typetools_checker-framework_checker-framework-2.5.3_LICENSE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_typetools_checker-framework_checker-framework-2.5.3_LICENSE.txt
new file mode 100644
index 0000000..fe8c705
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_typetools_checker-framework_checker-framework-2.5.3_LICENSE.txt
@@ -0,0 +1,408 @@
+Most of the Checker Framework is licensed under the GNU General Public
+License, version 2 (GPL2), with the classpath exception. The text of this
+license appears below. This is the same license used for OpenJDK.
+
+A few parts of the Checker Framework have more permissive licenses.
+
+ * The annotations are licensed under the MIT License. (The text of this
+ license appears below.) More specifically, all the parts of the Checker
+ Framework that you might want to include with your own program use the
+ MIT License. This is the checker-qual.jar file and all the files that
+ appear in it: every file in a qual/ directory, plus utility files such
+ as NullnessUtil.java, RegexUtil.java, SignednessUtil.java, etc.
+ In addition, the cleanroom implementations of third-party annotations,
+ which the Checker Framework recognizes as aliases for its own
+ annotations, are licensed under the MIT License.
+
+Some external libraries that are included with the Checker Framework have
+different licenses.
+
+ * javaparser is dual licensed under the LGPL or the Apache license -- you
+ may use it under whichever one you want. (The javaparser source code
+ contains a file with the text of the GPL, but it is not clear why, since
+ javaparser does not use the GPL.) See file stubparser/LICENSE
+ and the source code of all its files.
+
+ * JUnit is licensed under the Common Public License v1.0 (see
+ http://www.junit.org/license), with parts (Hamcrest) licensed under the
+ BSD License (see http://hamcrest.org/JavaHamcrest/).
+
+ * plume-lib is licensed under the MIT License.
+
+The Checker Framework includes annotations for the JDK in directory
+checker/jdk/, and for some other libraries. Each annotated library uses
+the same license as the unannotated version of the library.
+
+===========================================================================
+
+The GNU General Public License (GPL)
+
+Version 2, June 1991
+
+Copyright (C) 1989, 1991 Free Software Foundation, Inc.
+59 Temple Place, Suite 330, Boston, MA 02111-1307 USA
+
+Everyone is permitted to copy and distribute verbatim copies of this license
+document, but changing it is not allowed.
+
+Preamble
+
+The licenses for most software are designed to take away your freedom to share
+and change it. By contrast, the GNU General Public License is intended to
+guarantee your freedom to share and change free software--to make sure the
+software is free for all its users. This General Public License applies to
+most of the Free Software Foundation's software and to any other program whose
+authors commit to using it. (Some other Free Software Foundation software is
+covered by the GNU Library General Public License instead.) You can apply it to
+your programs, too.
+
+When we speak of free software, we are referring to freedom, not price. Our
+General Public Licenses are designed to make sure that you have the freedom to
+distribute copies of free software (and charge for this service if you wish),
+that you receive source code or can get it if you want it, that you can change
+the software or use pieces of it in new free programs; and that you know you
+can do these things.
+
+To protect your rights, we need to make restrictions that forbid anyone to deny
+you these rights or to ask you to surrender the rights. These restrictions
+translate to certain responsibilities for you if you distribute copies of the
+software, or if you modify it.
+
+For example, if you distribute copies of such a program, whether gratis or for
+a fee, you must give the recipients all the rights that you have. You must
+make sure that they, too, receive or can get the source code. And you must
+show them these terms so they know their rights.
+
+We protect your rights with two steps: (1) copyright the software, and (2)
+offer you this license which gives you legal permission to copy, distribute
+and/or modify the software.
+
+Also, for each author's protection and ours, we want to make certain that
+everyone understands that there is no warranty for this free software. If the
+software is modified by someone else and passed on, we want its recipients to
+know that what they have is not the original, so that any problems introduced
+by others will not reflect on the original authors' reputations.
+
+Finally, any free program is threatened constantly by software patents. We
+wish to avoid the danger that redistributors of a free program will
+individually obtain patent licenses, in effect making the program proprietary.
+To prevent this, we have made it clear that any patent must be licensed for
+everyone's free use or not licensed at all.
+
+The precise terms and conditions for copying, distribution and modification
+follow.
+
+TERMS AND CONDITIONS FOR COPYING, DISTRIBUTION AND MODIFICATION
+
+0. This License applies to any program or other work which contains a notice
+placed by the copyright holder saying it may be distributed under the terms of
+this General Public License. The "Program", below, refers to any such program
+or work, and a "work based on the Program" means either the Program or any
+derivative work under copyright law: that is to say, a work containing the
+Program or a portion of it, either verbatim or with modifications and/or
+translated into another language. (Hereinafter, translation is included
+without limitation in the term "modification".) Each licensee is addressed as
+"you".
+
+Activities other than copying, distribution and modification are not covered by
+this License; they are outside its scope. The act of running the Program is
+not restricted, and the output from the Program is covered only if its contents
+constitute a work based on the Program (independent of having been made by
+running the Program). Whether that is true depends on what the Program does.
+
+1. You may copy and distribute verbatim copies of the Program's source code as
+you receive it, in any medium, provided that you conspicuously and
+appropriately publish on each copy an appropriate copyright notice and
+disclaimer of warranty; keep intact all the notices that refer to this License
+and to the absence of any warranty; and give any other recipients of the
+Program a copy of this License along with the Program.
+
+You may charge a fee for the physical act of transferring a copy, and you may
+at your option offer warranty protection in exchange for a fee.
+
+2. You may modify your copy or copies of the Program or any portion of it, thus
+forming a work based on the Program, and copy and distribute such modifications
+or work under the terms of Section 1 above, provided that you also meet all of
+these conditions:
+
+ a) You must cause the modified files to carry prominent notices stating
+ that you changed the files and the date of any change.
+
+ b) You must cause any work that you distribute or publish, that in whole or
+ in part contains or is derived from the Program or any part thereof, to be
+ licensed as a whole at no charge to all third parties under the terms of
+ this License.
+
+ c) If the modified program normally reads commands interactively when run,
+ you must cause it, when started running for such interactive use in the
+ most ordinary way, to print or display an announcement including an
+ appropriate copyright notice and a notice that there is no warranty (or
+ else, saying that you provide a warranty) and that users may redistribute
+ the program under these conditions, and telling the user how to view a copy
+ of this License. (Exception: if the Program itself is interactive but does
+ not normally print such an announcement, your work based on the Program is
+ not required to print an announcement.)
+
+These requirements apply to the modified work as a whole. If identifiable
+sections of that work are not derived from the Program, and can be reasonably
+considered independent and separate works in themselves, then this License, and
+its terms, do not apply to those sections when you distribute them as separate
+works. But when you distribute the same sections as part of a whole which is a
+work based on the Program, the distribution of the whole must be on the terms
+of this License, whose permissions for other licensees extend to the entire
+whole, and thus to each and every part regardless of who wrote it.
+
+Thus, it is not the intent of this section to claim rights or contest your
+rights to work written entirely by you; rather, the intent is to exercise the
+right to control the distribution of derivative or collective works based on
+the Program.
+
+In addition, mere aggregation of another work not based on the Program with the
+Program (or with a work based on the Program) on a volume of a storage or
+distribution medium does not bring the other work under the scope of this
+License.
+
+3. You may copy and distribute the Program (or a work based on it, under
+Section 2) in object code or executable form under the terms of Sections 1 and
+2 above provided that you also do one of the following:
+
+ a) Accompany it with the complete corresponding machine-readable source
+ code, which must be distributed under the terms of Sections 1 and 2 above
+ on a medium customarily used for software interchange; or,
+
+ b) Accompany it with a written offer, valid for at least three years, to
+ give any third party, for a charge no more than your cost of physically
+ performing source distribution, a complete machine-readable copy of the
+ corresponding source code, to be distributed under the terms of Sections 1
+ and 2 above on a medium customarily used for software interchange; or,
+
+ c) Accompany it with the information you received as to the offer to
+ distribute corresponding source code. (This alternative is allowed only
+ for noncommercial distribution and only if you received the program in
+ object code or executable form with such an offer, in accord with
+ Subsection b above.)
+
+The source code for a work means the preferred form of the work for making
+modifications to it. For an executable work, complete source code means all
+the source code for all modules it contains, plus any associated interface
+definition files, plus the scripts used to control compilation and installation
+of the executable. However, as a special exception, the source code
+distributed need not include anything that is normally distributed (in either
+source or binary form) with the major components (compiler, kernel, and so on)
+of the operating system on which the executable runs, unless that component
+itself accompanies the executable.
+
+If distribution of executable or object code is made by offering access to copy
+from a designated place, then offering equivalent access to copy the source
+code from the same place counts as distribution of the source code, even though
+third parties are not compelled to copy the source along with the object code.
+
+4. You may not copy, modify, sublicense, or distribute the Program except as
+expressly provided under this License. Any attempt otherwise to copy, modify,
+sublicense or distribute the Program is void, and will automatically terminate
+your rights under this License. However, parties who have received copies, or
+rights, from you under this License will not have their licenses terminated so
+long as such parties remain in full compliance.
+
+5. You are not required to accept this License, since you have not signed it.
+However, nothing else grants you permission to modify or distribute the Program
+or its derivative works. These actions are prohibited by law if you do not
+accept this License. Therefore, by modifying or distributing the Program (or
+any work based on the Program), you indicate your acceptance of this License to
+do so, and all its terms and conditions for copying, distributing or modifying
+the Program or works based on it.
+
+6. Each time you redistribute the Program (or any work based on the Program),
+the recipient automatically receives a license from the original licensor to
+copy, distribute or modify the Program subject to these terms and conditions.
+You may not impose any further restrictions on the recipients' exercise of the
+rights granted herein. You are not responsible for enforcing compliance by
+third parties to this License.
+
+7. If, as a consequence of a court judgment or allegation of patent
+infringement or for any other reason (not limited to patent issues), conditions
+are imposed on you (whether by court order, agreement or otherwise) that
+contradict the conditions of this License, they do not excuse you from the
+conditions of this License. If you cannot distribute so as to satisfy
+simultaneously your obligations under this License and any other pertinent
+obligations, then as a consequence you may not distribute the Program at all.
+For example, if a patent license would not permit royalty-free redistribution
+of the Program by all those who receive copies directly or indirectly through
+you, then the only way you could satisfy both it and this License would be to
+refrain entirely from distribution of the Program.
+
+If any portion of this section is held invalid or unenforceable under any
+particular circumstance, the balance of the section is intended to apply and
+the section as a whole is intended to apply in other circumstances.
+
+It is not the purpose of this section to induce you to infringe any patents or
+other property right claims or to contest validity of any such claims; this
+section has the sole purpose of protecting the integrity of the free software
+distribution system, which is implemented by public license practices. Many
+people have made generous contributions to the wide range of software
+distributed through that system in reliance on consistent application of that
+system; it is up to the author/donor to decide if he or she is willing to
+distribute software through any other system and a licensee cannot impose that
+choice.
+
+This section is intended to make thoroughly clear what is believed to be a
+consequence of the rest of this License.
+
+8. If the distribution and/or use of the Program is restricted in certain
+countries either by patents or by copyrighted interfaces, the original
+copyright holder who places the Program under this License may add an explicit
+geographical distribution limitation excluding those countries, so that
+distribution is permitted only in or among countries not thus excluded. In
+such case, this License incorporates the limitation as if written in the body
+of this License.
+
+9. The Free Software Foundation may publish revised and/or new versions of the
+General Public License from time to time. Such new versions will be similar in
+spirit to the present version, but may differ in detail to address new problems
+or concerns.
+
+Each version is given a distinguishing version number. If the Program
+specifies a version number of this License which applies to it and "any later
+version", you have the option of following the terms and conditions either of
+that version or of any later version published by the Free Software Foundation.
+If the Program does not specify a version number of this License, you may
+choose any version ever published by the Free Software Foundation.
+
+10. If you wish to incorporate parts of the Program into other free programs
+whose distribution conditions are different, write to the author to ask for
+permission. For software which is copyrighted by the Free Software Foundation,
+write to the Free Software Foundation; we sometimes make exceptions for this.
+Our decision will be guided by the two goals of preserving the free status of
+all derivatives of our free software and of promoting the sharing and reuse of
+software generally.
+
+NO WARRANTY
+
+11. BECAUSE THE PROGRAM IS LICENSED FREE OF CHARGE, THERE IS NO WARRANTY FOR
+THE PROGRAM, TO THE EXTENT PERMITTED BY APPLICABLE LAW. EXCEPT WHEN OTHERWISE
+STATED IN WRITING THE COPYRIGHT HOLDERS AND/OR OTHER PARTIES PROVIDE THE
+PROGRAM "AS IS" WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED,
+INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND
+FITNESS FOR A PARTICULAR PURPOSE. THE ENTIRE RISK AS TO THE QUALITY AND
+PERFORMANCE OF THE PROGRAM IS WITH YOU. SHOULD THE PROGRAM PROVE DEFECTIVE,
+YOU ASSUME THE COST OF ALL NECESSARY SERVICING, REPAIR OR CORRECTION.
+
+12. IN NO EVENT UNLESS REQUIRED BY APPLICABLE LAW OR AGREED TO IN WRITING WILL
+ANY COPYRIGHT HOLDER, OR ANY OTHER PARTY WHO MAY MODIFY AND/OR REDISTRIBUTE THE
+PROGRAM AS PERMITTED ABOVE, BE LIABLE TO YOU FOR DAMAGES, INCLUDING ANY
+GENERAL, SPECIAL, INCIDENTAL OR CONSEQUENTIAL DAMAGES ARISING OUT OF THE USE OR
+INABILITY TO USE THE PROGRAM (INCLUDING BUT NOT LIMITED TO LOSS OF DATA OR DATA
+BEING RENDERED INACCURATE OR LOSSES SUSTAINED BY YOU OR THIRD PARTIES OR A
+FAILURE OF THE PROGRAM TO OPERATE WITH ANY OTHER PROGRAMS), EVEN IF SUCH HOLDER
+OR OTHER PARTY HAS BEEN ADVISED OF THE POSSIBILITY OF SUCH DAMAGES.
+
+END OF TERMS AND CONDITIONS
+
+How to Apply These Terms to Your New Programs
+
+If you develop a new program, and you want it to be of the greatest possible
+use to the public, the best way to achieve this is to make it free software
+which everyone can redistribute and change under these terms.
+
+To do so, attach the following notices to the program. It is safest to attach
+them to the start of each source file to most effectively convey the exclusion
+of warranty; and each file should have at least the "copyright" line and a
+pointer to where the full notice is found.
+
+ One line to give the program's name and a brief idea of what it does.
+
+ Copyright (C) <year> <name of author>
+
+ This program is free software; you can redistribute it and/or modify it
+ under the terms of the GNU General Public License as published by the Free
+ Software Foundation; either version 2 of the License, or (at your option)
+ any later version.
+
+ This program is distributed in the hope that it will be useful, but WITHOUT
+ ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
+ FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License for
+ more details.
+
+ You should have received a copy of the GNU General Public License along
+ with this program; if not, write to the Free Software Foundation, Inc., 59
+ Temple Place, Suite 330, Boston, MA 02111-1307 USA
+
+Also add information on how to contact you by electronic and paper mail.
+
+If the program is interactive, make it output a short notice like this when it
+starts in an interactive mode:
+
+ Gnomovision version 69, Copyright (C) year name of author Gnomovision comes
+ with ABSOLUTELY NO WARRANTY; for details type 'show w'. This is free
+ software, and you are welcome to redistribute it under certain conditions;
+ type 'show c' for details.
+
+The hypothetical commands 'show w' and 'show c' should show the appropriate
+parts of the General Public License. Of course, the commands you use may be
+called something other than 'show w' and 'show c'; they could even be
+mouse-clicks or menu items--whatever suits your program.
+
+You should also get your employer (if you work as a programmer) or your school,
+if any, to sign a "copyright disclaimer" for the program, if necessary. Here
+is a sample; alter the names:
+
+ Yoyodyne, Inc., hereby disclaims all copyright interest in the program
+ 'Gnomovision' (which makes passes at compilers) written by James Hacker.
+
+ signature of Ty Coon, 1 April 1989
+
+ Ty Coon, President of Vice
+
+This General Public License does not permit incorporating your program into
+proprietary programs. If your program is a subroutine library, you may
+consider it more useful to permit linking proprietary applications with the
+library. If this is what you want to do, use the GNU Library General Public
+License instead of this License.
+
+
+"CLASSPATH" EXCEPTION TO THE GPL
+
+Certain source files distributed by Oracle America and/or its affiliates are
+subject to the following clarification and special exception to the GPL, but
+only where Oracle has expressly included in the particular source file's header
+the words "Oracle designates this particular file as subject to the "Classpath"
+exception as provided by Oracle in the LICENSE file that accompanied this code."
+
+ Linking this library statically or dynamically with other modules is making
+ a combined work based on this library. Thus, the terms and conditions of
+ the GNU General Public License cover the whole combination.
+
+ As a special exception, the copyright holders of this library give you
+ permission to link this library with independent modules to produce an
+ executable, regardless of the license terms of these independent modules,
+ and to copy and distribute the resulting executable under terms of your
+ choice, provided that you also meet, for each linked independent module,
+ the terms and conditions of the license of that module. An independent
+ module is a module which is not derived from or based on this library. If
+ you modify this library, you may extend this exception to your version of
+ the library, but you are not obligated to do so. If you do not wish to do
+ so, delete this exception statement from your version.
+
+===========================================================================
+
+MIT License:
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in
+all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+THE SOFTWARE.
+
+===========================================================================
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/templates/asterix-license.ftl b/asterixdb/src/main/licenses/templates/asterix-license.ftl
index 381d400..f9c969e 100644
--- a/asterixdb/src/main/licenses/templates/asterix-license.ftl
+++ b/asterixdb/src/main/licenses/templates/asterix-license.ftl
@@ -34,8 +34,11 @@
<#list licenseMap as e>
<#assign licenseUrl = e.getKey()/>
<#assign entry = e.getValue()/>
- <#assign projects = entry.projects/>
+ <#assign projects = entry.nonShadowedProjects/>
<#assign license = entry.getLicense()/>
+ <#if projects?size == 0>
+ <#continue/>
+ </#if>
<#if projects?size == 1>
Component:
<#assign isare = "is"/>
diff --git a/asterixdb/src/main/licenses/templates/asterix-notice.ftl b/asterixdb/src/main/licenses/templates/asterix-notice.ftl
index c825397..5090cb0 100644
--- a/asterixdb/src/main/licenses/templates/asterix-notice.ftl
+++ b/asterixdb/src/main/licenses/templates/asterix-notice.ftl
@@ -34,6 +34,9 @@
<#assign projects = e.getValue()/>
<#list projects as p>
+ <#if p.shadowed>
+ <#continue/>
+ </#if>
${p.name} (${p.groupId}:${p.artifactId}:${p.version})
<#list p.locations as loc>
- ${loc}${p.jarName}
diff --git a/hyracks-fullstack/NOTICE b/hyracks-fullstack/NOTICE
index 2e33eed..6e9c46b 100644
--- a/hyracks-fullstack/NOTICE
+++ b/hyracks-fullstack/NOTICE
@@ -1,5 +1,5 @@
Apache Hyracks and Algebricks
-Copyright 2015-2022 The Apache Software Foundation
+Copyright 2015-2023 The Apache Software Foundation
This product includes software developed at
The Apache Software Foundation (http://www.apache.org/).
diff --git a/hyracks-fullstack/algebricks/algebricks-compiler/pom.xml b/hyracks-fullstack/algebricks/algebricks-compiler/pom.xml
index caa2a69..5381b19 100644
--- a/hyracks-fullstack/algebricks/algebricks-compiler/pom.xml
+++ b/hyracks-fullstack/algebricks/algebricks-compiler/pom.xml
@@ -59,6 +59,11 @@
</dependency>
<dependency>
<groupId>org.apache.hyracks</groupId>
+ <artifactId>algebricks-runtime</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hyracks</groupId>
<artifactId>hyracks-api</artifactId>
<version>${project.version}</version>
</dependency>
diff --git a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/AbstractCompilerFactoryBuilder.java b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/AbstractCompilerFactoryBuilder.java
index c22d54d..3a2fe3b 100644
--- a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/AbstractCompilerFactoryBuilder.java
+++ b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/AbstractCompilerFactoryBuilder.java
@@ -19,6 +19,8 @@
package org.apache.hyracks.algebricks.compiler.api;
import java.util.List;
+import java.util.function.Function;
+import java.util.function.Supplier;
import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
@@ -32,7 +34,9 @@
import org.apache.hyracks.algebricks.core.algebra.expressions.IPartialAggregationTypeComputer;
import org.apache.hyracks.algebricks.core.rewriter.base.AbstractRuleController;
import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.algebricks.core.rewriter.base.IRuleSetKind;
import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
+import org.apache.hyracks.algebricks.data.IAWriterFactory;
import org.apache.hyracks.algebricks.data.IBinaryBooleanInspectorFactory;
import org.apache.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
import org.apache.hyracks.algebricks.data.IBinaryHashFunctionFactoryProvider;
@@ -40,6 +44,7 @@
import org.apache.hyracks.algebricks.data.IBinaryIntegerInspectorFactory;
import org.apache.hyracks.algebricks.data.INormalizedKeyComputerFactoryProvider;
import org.apache.hyracks.algebricks.data.IPrinterFactoryProvider;
+import org.apache.hyracks.algebricks.data.IResultSerializerFactoryProvider;
import org.apache.hyracks.algebricks.data.ISerializerDeserializerProvider;
import org.apache.hyracks.algebricks.data.ITypeTraitProvider;
import org.apache.hyracks.algebricks.data.IUnnestingPositionWriterFactory;
@@ -49,8 +54,9 @@
public abstract class AbstractCompilerFactoryBuilder {
- protected List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> logicalRewrites;
- protected List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> physicalRewrites;
+ protected Supplier<List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>>> logicalRewrites;
+ protected Function<IRuleSetKind, List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>>> logicalRewritesByKind;
+ protected Supplier<List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>>> physicalRewrites;
protected ITypeTraitProvider typeTraitProvider;
protected ISerializerDeserializerProvider serializerDeserializerProvider;
protected IBinaryHashFunctionFactoryProvider hashFunctionFactoryProvider;
@@ -59,6 +65,8 @@
protected IBinaryBooleanInspectorFactory binaryBooleanInspectorFactory;
protected IBinaryIntegerInspectorFactory binaryIntegerInspectorFactory;
protected IPrinterFactoryProvider printerProvider;
+ protected IAWriterFactory writerFactory;
+ protected IResultSerializerFactoryProvider resultSerializerFactoryProvider;
protected IPredicateEvaluatorFactoryProvider predEvaluatorFactoryProvider;
protected IExpressionRuntimeProvider expressionRuntimeProvider;
protected IExpressionTypeComputer expressionTypeComputer;
@@ -78,11 +86,18 @@
public abstract ICompilerFactory create();
- public void setLogicalRewrites(List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> logicalRewrites) {
+ public void setLogicalRewrites(
+ Supplier<List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>>> logicalRewrites) {
this.logicalRewrites = logicalRewrites;
}
- public void setPhysicalRewrites(List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> physicalRewrites) {
+ public void setLogicalRewritesByKind(
+ Function<IRuleSetKind, List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>>> logicalRewritesByKind) {
+ this.logicalRewritesByKind = logicalRewritesByKind;
+ }
+
+ public void setPhysicalRewrites(
+ Supplier<List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>>> physicalRewrites) {
this.physicalRewrites = physicalRewrites;
}
@@ -158,6 +173,22 @@
return printerProvider;
}
+ public void setWriterFactory(IAWriterFactory writerFactory) {
+ this.writerFactory = writerFactory;
+ }
+
+ public IAWriterFactory getWriterFactory() {
+ return writerFactory;
+ }
+
+ public void setResultSerializerFactoryProvider(IResultSerializerFactoryProvider resultSerializerFactoryProvider) {
+ this.resultSerializerFactoryProvider = resultSerializerFactoryProvider;
+ }
+
+ public IResultSerializerFactoryProvider getResultSerializerFactoryProvider() {
+ return resultSerializerFactoryProvider;
+ }
+
public void setExpressionRuntimeProvider(IExpressionRuntimeProvider expressionRuntimeProvider) {
this.expressionRuntimeProvider = expressionRuntimeProvider;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java
index 891980f..e35a539 100644
--- a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java
+++ b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java
@@ -18,8 +18,11 @@
*/
package org.apache.hyracks.algebricks.compiler.api;
+import java.util.List;
+
import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalPlan;
import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
import org.apache.hyracks.algebricks.core.algebra.expressions.IConflictingTypeResolver;
@@ -33,10 +36,15 @@
import org.apache.hyracks.algebricks.core.config.AlgebricksConfig;
import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
import org.apache.hyracks.algebricks.core.jobgen.impl.PlanCompiler;
+import org.apache.hyracks.algebricks.core.rewriter.base.AbstractRuleController;
import org.apache.hyracks.algebricks.core.rewriter.base.AlgebricksOptimizationContext;
import org.apache.hyracks.algebricks.core.rewriter.base.HeuristicOptimizer;
+import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
import org.apache.hyracks.algebricks.core.rewriter.base.IOptimizationContextFactory;
+import org.apache.hyracks.algebricks.core.rewriter.base.IRuleSetKind;
import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
+import org.apache.hyracks.algebricks.data.IAWriterFactory;
+import org.apache.hyracks.algebricks.runtime.writers.SerializedDataWriterFactory;
import org.apache.hyracks.api.exceptions.IWarningCollector;
import org.apache.hyracks.api.job.IJobletEventListenerFactory;
import org.apache.hyracks.api.job.JobSpecification;
@@ -58,14 +66,19 @@
IConflictingTypeResolver conflictingTypeResolver, PhysicalOptimizationConfig physicalOptimizationConfig,
AlgebricksPartitionConstraint clusterLocations, IWarningCollector warningCollector) {
IPlanPrettyPrinter prettyPrinter = PlanPrettyPrinter.createStringPlanPrettyPrinter();
- return new AlgebricksOptimizationContext(varCounter, expressionEvalSizeComputer,
+ return new AlgebricksOptimizationContext(this, varCounter, expressionEvalSizeComputer,
mergeAggregationExpressionFactory, expressionTypeComputer, missableTypeComputer,
conflictingTypeResolver, physicalOptimizationConfig, clusterLocations, prettyPrinter,
warningCollector);
}
+
+ @Override
+ public IOptimizationContext cloneOptimizationContext(IOptimizationContext oc) {
+ return new AlgebricksOptimizationContext((AlgebricksOptimizationContext) oc);
+ }
}
- private IOptimizationContextFactory optCtxFactory;
+ private final IOptimizationContextFactory optCtxFactory;
public HeuristicCompilerFactoryBuilder() {
this.optCtxFactory = DefaultOptimizationContextFactory.INSTANCE;
@@ -77,42 +90,85 @@
@Override
public ICompilerFactory create() {
- return new ICompilerFactory() {
- @Override
- public ICompiler createCompiler(final ILogicalPlan plan, final IMetadataProvider<?, ?> metadata,
- int varCounter) {
- final IOptimizationContext oc = optCtxFactory.createOptimizationContext(varCounter,
- expressionEvalSizeComputer, mergeAggregationExpressionFactory, expressionTypeComputer,
- missableTypeComputer, conflictingTypeResolver, physicalOptimizationConfig, clusterLocations,
- warningCollector);
- oc.setMetadataDeclarations(metadata);
- final HeuristicOptimizer opt = new HeuristicOptimizer(plan, logicalRewrites, physicalRewrites, oc);
- return new ICompiler() {
-
- @Override
- public void optimize() throws AlgebricksException {
- opt.optimize();
- }
-
- @Override
- public JobSpecification createJob(Object appContext,
- IJobletEventListenerFactory jobEventListenerFactory) throws AlgebricksException {
- AlgebricksConfig.ALGEBRICKS_LOGGER.trace("Starting Job Generation.\n");
- JobGenContext context = new JobGenContext(null, metadata, appContext,
- serializerDeserializerProvider, hashFunctionFactoryProvider, hashFunctionFamilyProvider,
- comparatorFactoryProvider, typeTraitProvider, binaryBooleanInspectorFactory,
- binaryIntegerInspectorFactory, printerProvider, missingWriterFactory, nullWriterFactory,
- unnestingPositionWriterFactory, normalizedKeyComputerFactoryProvider,
- expressionRuntimeProvider, expressionTypeComputer, oc, expressionEvalSizeComputer,
- partialAggregationTypeComputer, predEvaluatorFactoryProvider,
- physicalOptimizationConfig.getFrameSize(), clusterLocations, warningCollector,
- maxWarnings, physicalOptimizationConfig);
- PlanCompiler pc = new PlanCompiler(context);
- return pc.compilePlan(plan, jobEventListenerFactory);
- }
- };
- }
- };
+ return new CompilerFactoryImpl();
}
+ private class CompilerFactoryImpl implements ICompilerFactory {
+ @Override
+ public ICompiler createCompiler(ILogicalPlan plan, IMetadataProvider<?, ?> metadata, int varCounter) {
+ IOptimizationContext optContext =
+ optCtxFactory.createOptimizationContext(varCounter, expressionEvalSizeComputer,
+ mergeAggregationExpressionFactory, expressionTypeComputer, missableTypeComputer,
+ conflictingTypeResolver, physicalOptimizationConfig, clusterLocations, warningCollector);
+ optContext.setMetadataDeclarations(metadata);
+ optContext.setCompilerFactory(this);
+ return new CompilerImpl(this, plan, optContext, logicalRewrites.get(), physicalRewrites.get(),
+ writerFactory);
+ }
+
+ @Override
+ public ICompiler createCompiler(ILogicalPlan plan, IOptimizationContext newOptContext,
+ IRuleSetKind ruleSetKind) {
+ if (newOptContext.getCompilerFactory() != this) {
+ throw new IllegalStateException();
+ }
+ return new CompilerImpl(this, plan, newOptContext, logicalRewritesByKind.apply(ruleSetKind),
+ physicalRewrites.get(), SerializedDataWriterFactory.WITHOUT_RECORD_DESCRIPTOR);
+ }
+
+ private PlanCompiler createPlanCompiler(IOptimizationContext oc, Object appContext,
+ IAWriterFactory writerFactory) {
+ JobGenContext context = new JobGenContext(null, oc.getMetadataProvider(), appContext,
+ serializerDeserializerProvider, hashFunctionFactoryProvider, hashFunctionFamilyProvider,
+ comparatorFactoryProvider, typeTraitProvider, binaryBooleanInspectorFactory,
+ binaryIntegerInspectorFactory, printerProvider, writerFactory, resultSerializerFactoryProvider,
+ missingWriterFactory, nullWriterFactory, unnestingPositionWriterFactory,
+ normalizedKeyComputerFactoryProvider, expressionRuntimeProvider, expressionTypeComputer, oc,
+ expressionEvalSizeComputer, partialAggregationTypeComputer, predEvaluatorFactoryProvider,
+ physicalOptimizationConfig.getFrameSize(), clusterLocations, warningCollector, maxWarnings,
+ physicalOptimizationConfig);
+ return new PlanCompiler(context);
+ }
+ }
+
+ private static class CompilerImpl implements ICompiler {
+
+ private final CompilerFactoryImpl factory;
+
+ private final ILogicalPlan plan;
+
+ private final IOptimizationContext oc;
+
+ private final List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> logicalRewrites;
+
+ private final List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> physicalRewrites;
+
+ private final IAWriterFactory writerFactory;
+
+ private CompilerImpl(CompilerFactoryImpl factory, ILogicalPlan plan, IOptimizationContext oc,
+ List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> logicalRewrites,
+ List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> physicalRewrites,
+ IAWriterFactory writerFactory) {
+ this.factory = factory;
+ this.plan = plan;
+ this.oc = oc;
+ this.logicalRewrites = logicalRewrites;
+ this.physicalRewrites = physicalRewrites;
+ this.writerFactory = writerFactory;
+ }
+
+ @Override
+ public void optimize() throws AlgebricksException {
+ HeuristicOptimizer opt = new HeuristicOptimizer(plan, logicalRewrites, physicalRewrites, oc);
+ opt.optimize();
+ }
+
+ @Override
+ public JobSpecification createJob(Object appContext, IJobletEventListenerFactory jobEventListenerFactory)
+ throws AlgebricksException {
+ AlgebricksConfig.ALGEBRICKS_LOGGER.trace("Starting Job Generation.\n");
+ PlanCompiler pc = factory.createPlanCompiler(oc, appContext, writerFactory);
+ return pc.compilePlan(plan, jobEventListenerFactory);
+ }
+ }
}
diff --git a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/ICompilerFactory.java b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/ICompilerFactory.java
index 7c138ea..07a8034 100644
--- a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/ICompilerFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/ICompilerFactory.java
@@ -19,8 +19,12 @@
package org.apache.hyracks.algebricks.compiler.api;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+import org.apache.hyracks.algebricks.core.rewriter.base.IRuleSetKind;
public interface ICompilerFactory {
ICompiler createCompiler(ILogicalPlan plan, IMetadataProvider<?, ?> metadata, int varCounter);
+
+ ICompiler createCompiler(ILogicalPlan plan, IOptimizationContext newOptContext, IRuleSetKind ruleSetKind);
}
diff --git a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/rewriter/rulecontrollers/SequentialFirstRuleCheckFixpointRuleController.java b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/rewriter/rulecontrollers/SequentialFirstRuleCheckFixpointRuleController.java
index 29c178a..79ec0fa 100644
--- a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/rewriter/rulecontrollers/SequentialFirstRuleCheckFixpointRuleController.java
+++ b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/rewriter/rulecontrollers/SequentialFirstRuleCheckFixpointRuleController.java
@@ -72,7 +72,7 @@
do {
anyChange = false;
for (int i = 0; i < rules.size(); i++) {
- boolean ruleFired = rewriteOperatorRef(root, rules.get(i), true, fullDfs);
+ boolean ruleFired = rewriteOperatorRef(root, rules.get(i), true, fullDfs, false);
// If the first rule returns false in the first iteration, stops applying the rules at all.
if (!firstRuleChecked && i == 0 && !ruleFired) {
return ruleFired;
diff --git a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/rewriter/rulecontrollers/SequentialFixpointRuleController.java b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/rewriter/rulecontrollers/SequentialFixpointRuleController.java
index 1fef33e..bbe281d 100644
--- a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/rewriter/rulecontrollers/SequentialFixpointRuleController.java
+++ b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/rewriter/rulecontrollers/SequentialFixpointRuleController.java
@@ -49,7 +49,7 @@
do {
anyChange = false;
for (IAlgebraicRewriteRule rule : ruleCollection) {
- boolean ruleFired = rewriteOperatorRef(root, rule, true, fullDfs);
+ boolean ruleFired = rewriteOperatorRef(root, rule, true, fullDfs, false);
if (ruleFired) {
anyChange = true;
anyRuleFired = true;
diff --git a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/rewriter/rulecontrollers/SequentialOnceRuleController.java b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/rewriter/rulecontrollers/SequentialOnceRuleController.java
index bcbc207..1090fe1 100644
--- a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/rewriter/rulecontrollers/SequentialOnceRuleController.java
+++ b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/rewriter/rulecontrollers/SequentialOnceRuleController.java
@@ -40,7 +40,7 @@
throws AlgebricksException {
boolean fired = false;
for (IAlgebraicRewriteRule rule : rules) {
- if (rewriteOperatorRef(root, rule, enterNestedPlans, true)) {
+ if (rewriteOperatorRef(root, rule, enterNestedPlans, true, false)) {
fired = true;
}
}
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..69ec210 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/IOptimizationContext.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/IOptimizationContext.java
@@ -33,6 +33,7 @@
import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
import org.apache.hyracks.algebricks.core.algebra.typing.ITypingContext;
import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.algebricks.core.rewriter.base.IOptimizationContextFactory;
import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
import org.apache.hyracks.api.exceptions.IWarningCollector;
@@ -93,4 +94,10 @@
public PlanStructureVerifier getPlanStructureVerifier();
public PlanStabilityVerifier getPlanStabilityVerifier();
+
+ void setCompilerFactory(Object factory);
+
+ Object getCompilerFactory();
+
+ IOptimizationContextFactory getOptimizationContextFactory();
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java
index 4466408..9c41cb9 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java
@@ -46,6 +46,7 @@
SELECT,
SINK,
SPLIT,
+ SWITCH,
SUBPLAN,
TOKENIZE,
UNIONALL,
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/base/PhysicalOperatorTag.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
index 4a900af..c052d58 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
@@ -68,6 +68,7 @@
SORT_MERGE_EXCHANGE,
SPATIAL_JOIN,
SPLIT,
+ SWITCH,
STABLE_SORT,
STATS,
STREAM_LIMIT,
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..7f7f65f 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
@@ -129,7 +130,7 @@
@Override
public String toString() {
- return "NULL";
+ return "null";
}
@Override
@@ -166,7 +167,7 @@
@Override
public String toString() {
- return "MISSING";
+ return "missing";
}
@Override
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/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/functions/IFunctionInfo.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/functions/IFunctionInfo.java
index 6e1ccef..34ed225 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/functions/IFunctionInfo.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/functions/IFunctionInfo.java
@@ -38,6 +38,11 @@
return true;
}
+ default boolean isExternal() {
+ // A function is not external by default.
+ return false;
+ }
+
/**
* @param args,
* the arguments.
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IDataSource.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IDataSource.java
index e15b699..5b75cd9 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,19 @@
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();
+
+ default boolean compareProperties() {
+ return false;
+ }
}
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/AbstractUnnestMapOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractUnnestMapOperator.java
index 5fc69b2..56e2e22 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractUnnestMapOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractUnnestMapOperator.java
@@ -25,10 +25,10 @@
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
import org.apache.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
+import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
public abstract class AbstractUnnestMapOperator extends AbstractUnnestOperator {
- protected final Mutable<ILogicalExpression> expression;
protected final List<Object> variableTypes;
protected boolean propagateInput;
protected List<Mutable<ILogicalExpression>> additionalFilteringExpressions;
@@ -42,7 +42,6 @@
public AbstractUnnestMapOperator(List<LogicalVariable> variables, Mutable<ILogicalExpression> expression,
List<Object> variableTypes, boolean propagateInput) {
super(variables, expression);
- this.expression = expression;
this.variableTypes = variableTypes;
this.propagateInput = propagateInput;
this.propagateIndexFilter = false;
@@ -90,6 +89,18 @@
};
}
+ @Override
+ public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform visitor) throws AlgebricksException {
+ boolean changed = super.acceptExpressionTransform(visitor);
+
+ if (additionalFilteringExpressions != null) {
+ for (Mutable<ILogicalExpression> filterExpr : additionalFilteringExpressions) {
+ changed |= visitor.transform(filterExpr);
+ }
+ }
+ return changed;
+ }
+
public boolean propagatesInput() {
return propagateInput;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/DataSourceScanOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/DataSourceScanOperator.java
index bb18014..9f73113 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/DataSourceScanOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/DataSourceScanOperator.java
@@ -57,7 +57,7 @@
}
public DataSourceScanOperator(List<LogicalVariable> variables, IDataSource<?> dataSource,
- Mutable<ILogicalExpression> selectCondition, long outputLimit, IProjectionInfo projectionInfo) {
+ Mutable<ILogicalExpression> selectCondition, long outputLimit, IProjectionInfo<?> projectionInfo) {
super(variables, dataSource);
projectVars = new ArrayList<>();
this.selectCondition = selectCondition;
@@ -77,7 +77,13 @@
@Override
public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform visitor) throws AlgebricksException {
- return false;
+ boolean changed = selectCondition != null && visitor.transform(selectCondition);
+ if (additionalFilteringExpressions != null) {
+ for (Mutable<ILogicalExpression> filterExpr : additionalFilteringExpressions) {
+ changed |= visitor.transform(filterExpr);
+ }
+ }
+ return changed;
}
@Override
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/SwitchOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/SwitchOperator.java
new file mode 100644
index 0000000..4b1f260
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/SwitchOperator.java
@@ -0,0 +1,67 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * 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 from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.operators.logical;
+
+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.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
+import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+
+/**
+ * Switch Operator receives an expression and an output mapping. We evaluate the expression during runtime and look up
+ * the result in the output mapping. Based on this, we propagate each tuple to the corresponding output branch(es).
+ */
+public class SwitchOperator extends AbstractReplicateOperator {
+
+ // Expression containing the index of the relevant field
+ private final Mutable<ILogicalExpression> branchingExpression;
+
+ // The supplied mapping from field values to arrays of output branch numbers
+ private final Map<Integer, int[]> outputMapping;
+
+ public SwitchOperator(int outputArity, Mutable<ILogicalExpression> branchingExpression,
+ Map<Integer, int[]> outputMapping) {
+ super(outputArity);
+ this.branchingExpression = branchingExpression;
+ this.outputMapping = outputMapping;
+ }
+
+ @Override
+ public LogicalOperatorTag getOperatorTag() {
+ return LogicalOperatorTag.SWITCH;
+ }
+
+ @Override
+ public <R, T> R accept(ILogicalOperatorVisitor<R, T> visitor, T arg) throws AlgebricksException {
+ return visitor.visitSwitchOperator(this, arg);
+ }
+
+ public Mutable<ILogicalExpression> getBranchingExpression() {
+ return branchingExpression;
+ }
+
+ public Map<Integer, int[]> getOutputMapping() {
+ return outputMapping;
+ }
+
+ @Override
+ public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform visitor) throws AlgebricksException {
+ return visitor.transform(branchingExpression);
+ }
+}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/UnnestMapOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/UnnestMapOperator.java
index c4bcc52..f8d07b8 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/UnnestMapOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/UnnestMapOperator.java
@@ -28,6 +28,7 @@
import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
import org.apache.hyracks.algebricks.core.algebra.typing.ITypingContext;
import org.apache.hyracks.algebricks.core.algebra.typing.NonPropagatingTypeEnvironment;
+import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
public class UnnestMapOperator extends AbstractUnnestMapOperator {
@@ -61,6 +62,12 @@
return visitor.visitUnnestMapOperator(this, arg);
}
+ @Override
+ public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform visitor) throws AlgebricksException {
+ boolean changed = super.acceptExpressionTransform(visitor);
+ return selectCondition != null && visitor.transform(selectCondition) || changed;
+ }
+
// When propagateInput is true,
// this operator propagates all input variables.
@Override
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/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/CardinalityInferenceVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/CardinalityInferenceVisitor.java
index 6ed90a5..14221d6 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/CardinalityInferenceVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/CardinalityInferenceVisitor.java
@@ -58,6 +58,7 @@
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.SwitchOperator;
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;
@@ -196,6 +197,11 @@
}
@Override
+ public Long visitSwitchOperator(SwitchOperator op, Void arg) throws AlgebricksException {
+ return op.getInputs().get(0).getValue().accept(this, arg);
+ }
+
+ @Override
public Long visitMaterializeOperator(MaterializeOperator op, Void arg) throws AlgebricksException {
return op.getInputs().get(0).getValue().accept(this, arg);
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
index f3717c8..5937d9d 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
@@ -74,6 +74,7 @@
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.SwitchOperator;
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;
@@ -427,6 +428,12 @@
}
@Override
+ public Void visitSwitchOperator(SwitchOperator op, IOptimizationContext ctx) throws AlgebricksException {
+ propagateFDsAndEquivClasses(op, ctx);
+ return null;
+ }
+
+ @Override
public Void visitMaterializeOperator(MaterializeOperator op, IOptimizationContext ctx) throws AlgebricksException {
propagateFDsAndEquivClasses(op, ctx);
return null;
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 e5d0241..cfae695 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
@@ -27,6 +27,7 @@
import org.apache.commons.lang3.mutable.Mutable;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
import org.apache.hyracks.algebricks.common.utils.Pair;
import org.apache.hyracks.algebricks.common.utils.Triple;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
@@ -34,6 +35,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;
@@ -65,6 +67,7 @@
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.SwitchOperator;
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;
@@ -249,8 +252,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
@@ -309,6 +312,12 @@
}
@Override
+ public Boolean visitSwitchOperator(SwitchOperator op, ILogicalOperator arg) throws AlgebricksException {
+ // TODO (GLENN): Implement this logic
+ throw new NotImplementedException();
+ }
+
+ @Override
public Boolean visitMaterializeOperator(MaterializeOperator op, ILogicalOperator arg) throws AlgebricksException {
AbstractLogicalOperator aop = (AbstractLogicalOperator) arg;
if (aop.getOperatorTag() != LogicalOperatorTag.MATERIALIZE) {
@@ -476,6 +485,12 @@
if (!isomorphic) {
return Boolean.FALSE;
}
+ IDataSource<?> dataSource = op.getDataSource();
+ IDataSource<?> argDataSource = argScan.getDataSource();
+ if (dataSource.compareProperties() && argDataSource.compareProperties()
+ && !Objects.equals(dataSource.getProperties(), argDataSource.getProperties())) {
+ return Boolean.FALSE;
+ }
DataSourceScanOperator scanOpArg = (DataSourceScanOperator) copyAndSubstituteVar(op, arg);
ILogicalExpression opCondition = op.getSelectCondition() != null ? op.getSelectCondition().getValue() : null;
ILogicalExpression argCondition =
@@ -522,6 +537,9 @@
if (!partProp.getPartitioningType().equals(partPropArg.getPartitioningType())) {
return Boolean.FALSE;
}
+ if (!partProp.getNodeDomain().sameAs(partPropArg.getNodeDomain())) {
+ return Boolean.FALSE;
+ }
List<LogicalVariable> columns = new ArrayList<LogicalVariable>();
partProp.getColumns(columns);
List<LogicalVariable> columnsArg = new ArrayList<LogicalVariable>();
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java
index 8ca2b83..e4d6586 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java
@@ -65,6 +65,7 @@
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.SwitchOperator;
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;
@@ -205,6 +206,12 @@
}
@Override
+ public Void visitSwitchOperator(SwitchOperator op, ILogicalOperator arg) throws AlgebricksException {
+ mapVariablesStandard(op, arg);
+ return null;
+ }
+
+ @Override
public Void visitMaterializeOperator(MaterializeOperator op, ILogicalOperator arg) throws AlgebricksException {
mapVariablesStandard(op, arg);
return null;
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java
index e242531..08acf13 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.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.exceptions.NotImplementedException;
import org.apache.hyracks.algebricks.common.utils.Pair;
import org.apache.hyracks.algebricks.common.utils.Triple;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
@@ -64,6 +65,7 @@
import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
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.SwitchOperator;
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;
@@ -447,6 +449,12 @@
}
@Override
+ public ILogicalOperator visitSwitchOperator(SwitchOperator op, ILogicalOperator arg) throws AlgebricksException {
+ // TODO (GLENN): Implement this logic
+ throw new NotImplementedException();
+ }
+
+ @Override
public ILogicalOperator visitMaterializeOperator(MaterializeOperator op, ILogicalOperator arg)
throws AlgebricksException {
MaterializeOperator opCopy = new MaterializeOperator();
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java
index 5a566ee..3a88d2c 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java
@@ -57,6 +57,7 @@
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.SwitchOperator;
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;
@@ -194,6 +195,11 @@
}
@Override
+ public Void visitSwitchOperator(SwitchOperator op, IOptimizationContext arg) throws AlgebricksException {
+ return null;
+ }
+
+ @Override
public Void visitMaterializeOperator(MaterializeOperator op, IOptimizationContext arg) throws AlgebricksException {
// TODO Auto-generated method stub
return null;
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java
index c2ee661..8ef0b5b 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java
@@ -20,6 +20,7 @@
package org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors;
import java.util.ArrayList;
+import java.util.HashMap;
import java.util.List;
import org.apache.commons.lang3.mutable.Mutable;
@@ -63,6 +64,7 @@
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.SwitchOperator;
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;
@@ -176,7 +178,13 @@
@Override
public ILogicalOperator visitSplitOperator(SplitOperator op, Void arg) throws AlgebricksException {
- return new SplitOperator(op.getOutputArity(), op.getBranchingExpression());
+ return new SplitOperator(op.getOutputArity(), deepCopyExpressionRef(op.getBranchingExpression()));
+ }
+
+ @Override
+ public ILogicalOperator visitSwitchOperator(SwitchOperator op, Void arg) throws AlgebricksException {
+ return new SwitchOperator(op.getOutputArity(), deepCopyExpressionRef(op.getBranchingExpression()),
+ new HashMap<>(op.getOutputMapping()));
}
@Override
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/PrimaryKeyVariablesVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/PrimaryKeyVariablesVisitor.java
index eb90288..8a3a885 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/PrimaryKeyVariablesVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/PrimaryKeyVariablesVisitor.java
@@ -58,6 +58,7 @@
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.SwitchOperator;
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;
@@ -186,6 +187,11 @@
}
@Override
+ public Void visitSwitchOperator(SwitchOperator op, IOptimizationContext arg) throws AlgebricksException {
+ return null;
+ }
+
+ @Override
public Void visitMaterializeOperator(MaterializeOperator op, IOptimizationContext ctx) throws AlgebricksException {
return null;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java
index 5d9d7895..ff50994 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java
@@ -62,6 +62,7 @@
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.SwitchOperator;
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;
@@ -271,6 +272,11 @@
}
@Override
+ public Void visitSwitchOperator(SwitchOperator op, Void arg) throws AlgebricksException {
+ return null;
+ }
+
+ @Override
public Void visitMaterializeOperator(MaterializeOperator op, Void arg) throws AlgebricksException {
return null;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java
index 44bb7e2..e7d6a92 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java
@@ -60,6 +60,7 @@
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.SwitchOperator;
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;
@@ -290,6 +291,12 @@
}
@Override
+ public Void visitSwitchOperator(SwitchOperator op, Void arg) throws AlgebricksException {
+ standardLayout(op);
+ return null;
+ }
+
+ @Override
public Void visitMaterializeOperator(MaterializeOperator op, Void arg) throws AlgebricksException {
standardLayout(op);
return null;
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
index 439e493..cf8196c 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
@@ -23,6 +23,7 @@
import org.apache.commons.lang3.mutable.Mutable;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
import org.apache.hyracks.algebricks.common.utils.Pair;
import org.apache.hyracks.algebricks.common.utils.Triple;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
@@ -63,6 +64,7 @@
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.SwitchOperator;
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;
@@ -410,6 +412,13 @@
}
@Override
+ public Void visitSwitchOperator(SwitchOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+ throws AlgebricksException {
+ // TODO (GLENN): Implement this logic
+ throw new NotImplementedException();
+ }
+
+ @Override
public Void visitMaterializeOperator(MaterializeOperator op, Pair<LogicalVariable, LogicalVariable> pair)
throws AlgebricksException {
// does not produce/use any variables
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
index 174b184..4c994b5 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
@@ -25,6 +25,7 @@
import org.apache.commons.lang3.mutable.Mutable;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
import org.apache.hyracks.algebricks.common.utils.Pair;
import org.apache.hyracks.algebricks.common.utils.Triple;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
@@ -64,6 +65,7 @@
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.SwitchOperator;
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;
@@ -473,6 +475,12 @@
}
@Override
+ public Void visitSwitchOperator(SwitchOperator op, Void arg) throws AlgebricksException {
+ // TODO (GLENN): Implement this logic
+ throw new NotImplementedException();
+ }
+
+ @Override
public Void visitMaterializeOperator(MaterializeOperator op, Void arg) throws AlgebricksException {
return null;
}
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/OneToOneExchangePOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/OneToOneExchangePOperator.java
index 083e4d3..2bd78b1 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/OneToOneExchangePOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/OneToOneExchangePOperator.java
@@ -27,7 +27,6 @@
import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
import org.apache.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
import org.apache.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
-import org.apache.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
import org.apache.hyracks.api.dataflow.IConnectorDescriptor;
import org.apache.hyracks.api.job.IConnectorDescriptorRegistry;
@@ -43,7 +42,9 @@
@Override
public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
AbstractLogicalOperator op2 = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
- deliveredProperties = (StructuralPropertiesVector) op2.getDeliveredPhysicalProperties().clone();
+ if (op2.getDeliveredPhysicalProperties() != null) {
+ deliveredProperties = op2.getDeliveredPhysicalProperties().clone();
+ }
}
@Override
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SinkWritePOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SinkWritePOperator.java
index 3521a27..07c798f 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SinkWritePOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SinkWritePOperator.java
@@ -101,7 +101,7 @@
IMetadataProvider<?, ?> mp = context.getMetadataProvider();
Pair<IPushRuntimeFactory, AlgebricksPartitionConstraint> runtimeAndConstraints =
- mp.getWriteFileRuntime(write.getDataSink(), columns, pf, inputDesc);
+ mp.getWriteFileRuntime(write.getDataSink(), columns, pf, context.getWriterFactory(), inputDesc);
IPushRuntimeFactory runtime = runtimeAndConstraints.first;
runtime.setSourceLocation(write.getSourceLocation());
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SwitchPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SwitchPOperator.java
new file mode 100644
index 0000000..4b58788
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SwitchPOperator.java
@@ -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.
+ */
+package org.apache.hyracks.algebricks.core.algebra.operators.physical;
+
+import java.util.Map;
+
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionRuntimeProvider;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SwitchOperator;
+import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenHelper;
+import org.apache.hyracks.algebricks.data.IBinaryIntegerInspectorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.operators.std.SwitchOperatorDescriptor;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
+
+public class SwitchPOperator extends AbstractReplicatePOperator {
+
+ @Override
+ public PhysicalOperatorTag getOperatorTag() {
+ return PhysicalOperatorTag.SWITCH;
+ }
+
+ @Override
+ public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
+ IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+ throws AlgebricksException {
+ SwitchOperator sop = (SwitchOperator) op;
+ int outputArity = sop.getOutputArity();
+ Map<Integer, int[]> outputMapping = sop.getOutputMapping();
+
+ IOperatorDescriptorRegistry spec = builder.getJobSpec();
+ RecordDescriptor recDescriptor =
+ JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), propagatedSchema, context);
+
+ IExpressionRuntimeProvider expressionRuntimeProvider = context.getExpressionRuntimeProvider();
+ IScalarEvaluatorFactory branchingExprEvalFactory = expressionRuntimeProvider.createEvaluatorFactory(
+ sop.getBranchingExpression().getValue(), context.getTypeEnvironment(op), inputSchemas, context);
+
+ IBinaryIntegerInspectorFactory intInspectorFactory = context.getBinaryIntegerInspectorFactory();
+
+ SwitchOperatorDescriptor sopDesc = new SwitchOperatorDescriptor(spec, recDescriptor, outputArity,
+ branchingExprEvalFactory, intInspectorFactory, outputMapping);
+ sopDesc.setSourceLocation(sop.getSourceLocation());
+
+ contributeOpDesc(builder, sop, sopDesc);
+ ILogicalOperator src = op.getInputs().get(0).getValue();
+ builder.contributeGraphEdge(src, 0, op, 0);
+ }
+}
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..2b53de4 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
@@ -18,6 +18,8 @@
*/
package org.apache.hyracks.algebricks.core.algebra.prettyprint;
+import java.util.Map;
+
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.ILogicalPlan;
@@ -30,17 +32,22 @@
/** 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;
+
+ /** Prints the logical plan, annotated with physical operator and connector ids */
+ IPlanPrettyPrinter printPlan(ILogicalPlan plan, Map<Object, String> log2phys, 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..7aabbef 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
@@ -20,9 +20,11 @@
import java.util.Iterator;
import java.util.List;
+import java.util.Map;
import org.apache.commons.lang3.mutable.Mutable;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
import org.apache.hyracks.algebricks.common.utils.Pair;
import org.apache.hyracks.algebricks.common.utils.Triple;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
@@ -65,6 +67,7 @@
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.SwitchOperator;
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;
@@ -90,27 +93,57 @@
}
@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 printPlan(ILogicalPlan plan, Map<Object, String> log2phys,
+ boolean printOptimizerEstimates) throws AlgebricksException {
+ //TODO(ian): would be nice if the text plan returned real operator ids too
+ printPlanImpl(plan, 0, printOptimizerEstimates);
return this;
}
- private void printPlanImpl(ILogicalPlan plan, int indent) throws AlgebricksException {
+ @Override
+ 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, 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 +156,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);
}
}
}
@@ -430,6 +464,12 @@
}
@Override
+ public Void visitSwitchOperator(SwitchOperator op, Integer indent) throws AlgebricksException {
+ // TODO (GLENN): Implement this logic
+ throw new NotImplementedException();
+ }
+
+ @Override
public Void visitMaterializeOperator(MaterializeOperator op, Integer indent) throws AlgebricksException {
addIndent(indent).append("materialize");
return null;
@@ -608,7 +648,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..e54ef02 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
@@ -19,6 +19,7 @@
package org.apache.hyracks.algebricks.core.algebra.prettyprint;
import java.io.IOException;
+import java.util.Collections;
import java.util.Deque;
import java.util.HashMap;
import java.util.LinkedList;
@@ -27,6 +28,7 @@
import org.apache.commons.lang3.mutable.Mutable;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
import org.apache.hyracks.algebricks.common.utils.Pair;
import org.apache.hyracks.algebricks.common.utils.Triple;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
@@ -34,8 +36,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;
@@ -70,6 +74,7 @@
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.SwitchOperator;
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;
@@ -96,8 +101,9 @@
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 Map<Object, String> log2odid = Collections.emptyMap();
private final IdCounter idCounter = new IdCounter();
private final JsonGenerator jsonGenerator;
@@ -138,8 +144,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 +160,38 @@
}
@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 printPlan(ILogicalPlan plan, Map<Object, String> log2phys,
+ boolean printOptimizerEstimates) throws AlgebricksException {
+ this.log2odid = log2phys;
+ printPlanImpl(plan, printOptimizerEstimates);
flushContentToWriter();
return this;
}
- private void printPlanImpl(ILogicalPlan plan) throws AlgebricksException {
+ @Override
+ public final IPlanPrettyPrinter printOperator(AbstractLogicalOperator op, boolean printInputs,
+ boolean printOptimizerEstimates) throws AlgebricksException {
+ printOperatorImpl(op, printInputs, printOptimizerEstimates);
+ flushContentToWriter();
+ return this;
+ }
+
+ 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,22 +201,27 @@
}
}
- 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);
jsonGenerator.writeStringField("operatorId", idCounter.printOperatorId(op));
+ String od = log2odid.get(op);
+ if (od != null) {
+ jsonGenerator.writeStringField("runtime-id", od);
+ }
IPhysicalOperator pOp = op.getPhysicalOperator();
if (pOp != null) {
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 +229,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 +347,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 +362,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));
@@ -590,6 +647,12 @@
}
@Override
+ public Void visitSwitchOperator(SwitchOperator op, Void indent) throws AlgebricksException {
+ // TODO (GLENN): Implement this logic
+ throw new NotImplementedException();
+ }
+
+ @Override
public Void visitMaterializeOperator(MaterializeOperator op, Void indent) throws AlgebricksException {
try {
jsonGenerator.writeStringField(OPERATOR_FIELD, "materialize");
@@ -767,7 +830,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 +920,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 +963,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 +981,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 +992,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 +1007,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 +1017,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 +1029,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 +1047,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 +1065,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 +1077,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/util/OperatorManipulationUtil.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorManipulationUtil.java
index 26be09c..3457751 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;
@@ -305,10 +308,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 +331,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 {
@@ -402,8 +401,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 +444,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 +509,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/algebra/visitors/ILogicalOperatorVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/visitors/ILogicalOperatorVisitor.java
index d521831..4eb6494 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/visitors/ILogicalOperatorVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/visitors/ILogicalOperatorVisitor.java
@@ -48,6 +48,7 @@
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.SwitchOperator;
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;
@@ -88,6 +89,8 @@
public R visitSplitOperator(SplitOperator op, T arg) throws AlgebricksException;
+ public R visitSwitchOperator(SwitchOperator op, T arg) throws AlgebricksException;
+
public R visitMaterializeOperator(MaterializeOperator op, T arg) throws AlgebricksException;
public R visitScriptOperator(ScriptOperator op, T arg) throws AlgebricksException;
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/visitors/LogicalExpressionReferenceTransformVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/visitors/LogicalExpressionReferenceTransformVisitor.java
index 9350f95..f1613a5 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/visitors/LogicalExpressionReferenceTransformVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/visitors/LogicalExpressionReferenceTransformVisitor.java
@@ -50,6 +50,7 @@
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.SwitchOperator;
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;
@@ -166,6 +167,12 @@
}
@Override
+ public Boolean visitSwitchOperator(SwitchOperator op, ILogicalExpressionReferenceTransform arg)
+ throws AlgebricksException {
+ return visitOperator(op, arg);
+ }
+
+ @Override
public Boolean visitMaterializeOperator(MaterializeOperator op, ILogicalExpressionReferenceTransform arg)
throws AlgebricksException {
return visitOperator(op, arg);
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..c94d72a 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,14 @@
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);
+ public static final boolean BATCH_LOOKUP_DEFAULT = false;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobBuilder.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobBuilder.java
index c8fff99..d8bf190 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobBuilder.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobBuilder.java
@@ -21,6 +21,7 @@
import static org.apache.hyracks.api.exceptions.ErrorCode.DESCRIPTOR_GENERATION_ERROR;
import java.util.ArrayList;
+import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@@ -147,6 +148,15 @@
hyracksOps.put(op, opDesc);
}
+ public Map<Object, String> getLogical2PhysicalMap() {
+ Map<ILogicalOperator, String> mergedOperatorMap = new HashMap<>();
+ hyracksOps.forEach(((k, v) -> mergedOperatorMap.put(k, v.getOperatorId().toString())));
+ algebraicOpBelongingToMetaAsterixOp
+ .forEach((k, v) -> mergedOperatorMap.put(k, metaAsterixOps.get(v).getOperatorId().toString()));
+ connectors.forEach((k, v) -> mergedOperatorMap.put(k, v.getFirst().getConnectorId().toString()));
+ return Collections.unmodifiableMap(mergedOperatorMap);
+ }
+
@Override
public void contributeAlgebricksPartitionConstraint(IOperatorDescriptor opDesc,
AlgebricksPartitionConstraint apcArg) {
@@ -174,6 +184,7 @@
jobSpec.addRoot(opDesc);
}
setAllPartitionConstraints(tgtConstraints);
+ jobSpec.setLogical2PhysicalMap(getLogical2PhysicalMap());
}
public List<IOperatorDescriptor> getGeneratedMetaOps() {
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/AbstractRuleController.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/AbstractRuleController.java
index 0261106..9a47b8a 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/AbstractRuleController.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/AbstractRuleController.java
@@ -67,14 +67,15 @@
*/
protected boolean rewriteOperatorRef(Mutable<ILogicalOperator> opRef, IAlgebraicRewriteRule rule)
throws AlgebricksException {
- return rewriteOperatorRef(opRef, rule, true, false);
+ return rewriteOperatorRef(opRef, rule, true, false, false);
}
protected boolean rewriteOperatorRef(Mutable<ILogicalOperator> opRef, IAlgebraicRewriteRule rule,
- boolean enterNestedPlans, boolean fullDFS) throws AlgebricksException {
+ boolean enterNestedPlans, boolean fullDFS, boolean enteredNestedPlanRoot) throws AlgebricksException {
String preBeforePlan = getPlanString(opRef);
sanityCheckBeforeRewrite(rule, opRef);
+ rule.enteredNestedPlan(enteredNestedPlanRoot);
if (rule.rewritePre(opRef, context)) {
String preAfterPlan = getPlanString(opRef);
printRuleApplication(rule, "fired", preBeforePlan, preAfterPlan);
@@ -88,7 +89,7 @@
AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
for (Mutable<ILogicalOperator> inp : op.getInputs()) {
- if (rewriteOperatorRef(inp, rule, enterNestedPlans, fullDFS)) {
+ if (rewriteOperatorRef(inp, rule, enterNestedPlans, fullDFS, false)) {
rewritten = true;
if (!fullDFS) {
break;
@@ -100,7 +101,7 @@
AbstractOperatorWithNestedPlans o2 = (AbstractOperatorWithNestedPlans) op;
for (ILogicalPlan p : o2.getNestedPlans()) {
for (Mutable<ILogicalOperator> r : p.getRoots()) {
- if (rewriteOperatorRef(r, rule, enterNestedPlans, fullDFS)) {
+ if (rewriteOperatorRef(r, rule, enterNestedPlans, fullDFS, true)) {
rewritten = true;
if (!fullDFS) {
break;
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/IAlgebraicRewriteRule.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/IAlgebraicRewriteRule.java
index 128c372..33bc4a9 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/IAlgebraicRewriteRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/IAlgebraicRewriteRule.java
@@ -54,4 +54,13 @@
throws AlgebricksException {
return false;
}
+
+ /**
+ * Called before calling {@link #rewritePre} to designate if the {@code opRef} is a nested plan root.
+ *
+ * @param enteredNestedPlanRoot whether the operator to be rewritten is a nested plan root.
+ */
+ default void enteredNestedPlan(boolean enteredNestedPlanRoot) {
+ // no op
+ }
}
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..86be6d0 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,11 @@
private static final String MIN_MEMORY_ALLOCATION = "MIN_MEMORY_ALLOCATION";
private static final String ARRAY_INDEX = "ARRAY_INDEX";
private static final String EXTERNAL_SCAN_BUFFER_SIZE = "EXTERNAL_SCAN_BUFFER_SIZE";
+ private static final String BATCH_LOOKUP = "BATCH_LOOKUP";
+ private 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 +244,52 @@
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 boolean isBatchLookupEnabled() {
+ return getBoolean(BATCH_LOOKUP, AlgebricksConfig.BATCH_LOOKUP_DEFAULT);
+ }
+
+ public void setBatchLookup(boolean batchedLookup) {
+ setBoolean(BATCH_LOOKUP, batchedLookup);
+ }
+
public void setExternalScanBufferSize(int bufferSize) {
setInt(EXTERNAL_SCAN_BUFFER_SIZE, bufferSize);
}
@@ -269,4 +320,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/main/java/org/apache/hyracks/algebricks/core/utils/LogicalOperatorDotVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/LogicalOperatorDotVisitor.java
index ee7f7aa..75da5ff 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/LogicalOperatorDotVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/LogicalOperatorDotVisitor.java
@@ -26,6 +26,7 @@
import org.apache.commons.lang3.mutable.Mutable;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
import org.apache.hyracks.algebricks.common.utils.Pair;
import org.apache.hyracks.algebricks.common.utils.Triple;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
@@ -63,6 +64,7 @@
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.SwitchOperator;
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;
@@ -484,6 +486,12 @@
}
@Override
+ public String visitSwitchOperator(SwitchOperator op, Boolean showDetails) throws AlgebricksException {
+ // TODO (GLENN): Implement this logic
+ throw new NotImplementedException();
+ }
+
+ @Override
public String visitMaterializeOperator(MaterializeOperator op, Boolean showDetails) throws AlgebricksException {
stringBuilder.setLength(0);
stringBuilder.append("materialize");
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/src/main/java/org/apache/hyracks/algebricks/data/IResultSerializerFactoryProvider.java b/hyracks-fullstack/algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/IResultSerializerFactoryProvider.java
index 7525180..75fd791 100644
--- a/hyracks-fullstack/algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/IResultSerializerFactoryProvider.java
+++ b/hyracks-fullstack/algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/IResultSerializerFactoryProvider.java
@@ -32,10 +32,8 @@
* - A printer factory array to print the tuple containing different fields.
* @param writerFactory
* - A writer factory to write the serialized data to the print stream.
- * @param inputRecordDesc
- * - The record descriptor describing the input frame to be serialized.
* @return A new instance of result serialized appender.
*/
- public IResultSerializerFactory getAqlResultSerializerFactoryProvider(int[] fields,
- IPrinterFactory[] printerFactories, IAWriterFactory writerFactory);
+ public IResultSerializerFactory getResultSerializerFactoryProvider(int[] fields, IPrinterFactory[] printerFactories,
+ IAWriterFactory writerFactory);
}
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractCommonExpressionsRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractCommonExpressionsRule.java
index 9420498..e2ba557 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractCommonExpressionsRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractCommonExpressionsRule.java
@@ -77,17 +77,17 @@
*/
public class ExtractCommonExpressionsRule implements IAlgebraicRewriteRule {
- private final List<ILogicalExpression> originalAssignExprs = new ArrayList<ILogicalExpression>();
+ private final List<ILogicalExpression> originalAssignExprs = new ArrayList<>();
private final CommonExpressionSubstitutionVisitor substVisitor = new CommonExpressionSubstitutionVisitor();
- private final Map<ILogicalExpression, ExprEquivalenceClass> exprEqClassMap =
- new HashMap<ILogicalExpression, ExprEquivalenceClass>();
+ private final Map<ILogicalExpression, ExprEquivalenceClass> exprEqClassMap = new HashMap<>();
private final List<LogicalVariable> tmpLiveVars = new ArrayList<>();
private final List<LogicalVariable> tmpProducedVars = new ArrayList<>();
+ private boolean enteredNestedPlan = false;
// Set of operators for which common subexpression elimination should not be performed.
- private static final Set<LogicalOperatorTag> ignoreOps = new HashSet<LogicalOperatorTag>(6);
+ private static final Set<LogicalOperatorTag> ignoreOps = new HashSet<>(6);
static {
ignoreOps.add(LogicalOperatorTag.UNNEST);
@@ -100,6 +100,11 @@
}
@Override
+ public void enteredNestedPlan(boolean enteredNestedPlanRoot) {
+ this.enteredNestedPlan = enteredNestedPlanRoot;
+ }
+
+ @Override
public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
throws AlgebricksException {
return false;
@@ -108,6 +113,14 @@
@Override
public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
throws AlgebricksException {
+ ILogicalOperator op = opRef.getValue();
+ if (enteredNestedPlan) {
+ enteredNestedPlan = false;
+ } else if (op.getOperatorTag() != LogicalOperatorTag.DISTRIBUTE_RESULT
+ && op.getOperatorTag() != LogicalOperatorTag.SINK
+ && op.getOperatorTag() != LogicalOperatorTag.DELEGATE_OPERATOR) {
+ return false;
+ }
exprEqClassMap.clear();
substVisitor.setContext(context);
boolean modified = removeCommonExpressions(opRef, context);
@@ -155,9 +168,6 @@
private boolean removeCommonExpressions(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
throws AlgebricksException {
AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
- if (context.checkIfInDontApplySet(this, opRef.getValue())) {
- return false;
- }
boolean modified = false;
// Recurse into children.
@@ -166,7 +176,9 @@
modified = true;
}
}
-
+ if (context.checkIfInDontApplySet(this, opRef.getValue())) {
+ return modified;
+ }
// TODO: Deal with replicate properly. Currently, we just clear the expr equivalence map,
// since we want to avoid incorrect expression replacement
// (the resulting new variables should be assigned live below a replicate/split).
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/IsolateHyracksOperatorsRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/IsolateHyracksOperatorsRule.java
index e13ec30..129aba3 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/IsolateHyracksOperatorsRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/IsolateHyracksOperatorsRule.java
@@ -132,6 +132,7 @@
e.setExecutionMode(inOp.getExecutionMode());
context.computeAndSetTypeEnvironmentForOperator(e);
e.recomputeSchema();
+ e.computeDeliveredPhysicalProperties(context);
inOpRef.setValue(e);
}
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PopulateResultMetadataRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PopulateResultMetadataRule.java
index e1b2e5d..1eda133 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PopulateResultMetadataRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PopulateResultMetadataRule.java
@@ -47,7 +47,7 @@
}
DistributeResultOperator dop = (DistributeResultOperator) op;
IResultMetadata resultMetadata = dop.getResultMetadata();
- if (resultMetadata.getOutputTypes() != null) {
+ if (resultMetadata == null || resultMetadata.getOutputTypes() != null) {
return false;
}
List<Mutable<ILogicalExpression>> exprList = dop.getExpressions();
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/RemoveRedundantVariablesRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/RemoveRedundantVariablesRule.java
index 3760be5..1ed7170 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/RemoveRedundantVariablesRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/RemoveRedundantVariablesRule.java
@@ -138,9 +138,12 @@
if (expr.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
continue;
}
- VariableReferenceExpression rhsVarRefExpr = (VariableReferenceExpression) expr;
- // Update equivalence class map.
LogicalVariable lhs = assignOp.getVariables().get(i);
+ if (context.shouldNotBeInlined(lhs)) {
+ continue;
+ }
+ // Update equivalence class map.
+ VariableReferenceExpression rhsVarRefExpr = (VariableReferenceExpression) expr;
LogicalVariable rhs = rhsVarRefExpr.getVariableReference();
updateEquivalenceClassMap(lhs, rhs);
}
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
index a6fe495..a1aa01a 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
@@ -73,6 +73,7 @@
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.SwitchOperator;
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;
@@ -112,6 +113,7 @@
import org.apache.hyracks.algebricks.core.algebra.operators.physical.StreamSelectPOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.physical.StringStreamingScriptPOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.physical.SubplanPOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.SwitchPOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.physical.TokenizePOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.physical.UnionAllPOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.physical.UnnestPOperator;
@@ -309,6 +311,11 @@
}
@Override
+ public IPhysicalOperator visitSwitchOperator(SwitchOperator op, Boolean topLevelOp) {
+ return new SwitchPOperator();
+ }
+
+ @Override
public IPhysicalOperator visitScriptOperator(ScriptOperator op, Boolean topLevelOp) {
return new StringStreamingScriptPOperator();
}
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetMemoryRequirementsRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetMemoryRequirementsRule.java
index 3b0e9fb..b3748dd 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetMemoryRequirementsRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetMemoryRequirementsRule.java
@@ -59,6 +59,7 @@
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.SwitchOperator;
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;
@@ -272,6 +273,11 @@
}
@Override
+ public Void visitSwitchOperator(SwitchOperator op, Void arg) throws AlgebricksException {
+ return null;
+ }
+
+ @Override
public Void visitMaterializeOperator(MaterializeOperator op, Void arg) throws AlgebricksException {
return null;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/ReplaceNtsWithSubplanInputOperatorVisitor.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/ReplaceNtsWithSubplanInputOperatorVisitor.java
index 1388ccb..f4f9eda 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/ReplaceNtsWithSubplanInputOperatorVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/ReplaceNtsWithSubplanInputOperatorVisitor.java
@@ -56,6 +56,7 @@
import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
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.SwitchOperator;
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;
@@ -186,6 +187,11 @@
}
@Override
+ public ILogicalOperator visitSwitchOperator(SwitchOperator op, Void arg) throws AlgebricksException {
+ return visit(op);
+ }
+
+ @Override
public ILogicalOperator visitMaterializeOperator(MaterializeOperator op, Void arg) throws AlgebricksException {
return visit(op);
}
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..fa0549f 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,6 +32,8 @@
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;
@@ -65,11 +67,35 @@
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);
+ BroadcastSide broadcastSide = getBroadcastJoinSide(conditionExpr, varsLeft, varsRight, context);
+ if (broadcastSide == null) {
+ BuildSide buildSide = getHashJoinBuildSide(conditionExpr, varsLeft, varsRight, 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 +113,7 @@
break;
default:
// This should never happen
- throw new IllegalStateException(side.toString());
+ throw new IllegalStateException(broadcastSide.toString());
}
}
} else {
@@ -188,7 +214,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 +224,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 +233,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/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/NestedPlansRunningAggregatorFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/NestedPlansRunningAggregatorFactory.java
index 07d4e94..c9754f2 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/NestedPlansRunningAggregatorFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/NestedPlansRunningAggregatorFactory.java
@@ -28,7 +28,7 @@
import org.apache.hyracks.api.comm.VSizeFrame;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.EnforceFrameWriter;
-import org.apache.hyracks.api.dataflow.TimedFrameWriter;
+import org.apache.hyracks.api.dataflow.ProfiledFrameWriter;
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.job.JobFlag;
@@ -67,7 +67,7 @@
new RunningAggregatorOutput(ctx, subplans, keyFieldIdx.length + decorFieldIdx.length, writer);
IFrameWriter fw = outputWriter;
if (profile) {
- fw = TimedFrameWriter.time(outputWriter, ctx, "Aggregate Writer");
+ fw = ProfiledFrameWriter.time(outputWriter, ctx, "Aggregate Writer");
} else if (enforce) {
fw = EnforceFrameWriter.enforce(outputWriter);
}
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/StreamLimitRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/StreamLimitRuntimeFactory.java
index 793f095..d9ba859 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/StreamLimitRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/StreamLimitRuntimeFactory.java
@@ -55,9 +55,12 @@
@Override
public String toString() {
- String s = "stream-limit " + maxObjectsEvalFactory.toString();
+ String s = "stream-limit ";
+ if (maxObjectsEvalFactory != null) {
+ s += maxObjectsEvalFactory.toString();
+ }
if (offsetEvalFactory != null) {
- return s + ", " + offsetEvalFactory.toString();
+ return s + maxObjectsEvalFactory == null ? "" : ", " + offsetEvalFactory.toString();
} else {
return s;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/SwitchOperatorDescriptor.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/SwitchOperatorDescriptor.java
new file mode 100644
index 0000000..0fbb1d5
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/SwitchOperatorDescriptor.java
@@ -0,0 +1,228 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.runtime.operators.std;
+
+import java.nio.ByteBuffer;
+import java.util.Map;
+
+import org.apache.commons.lang3.mutable.MutableBoolean;
+import org.apache.hyracks.algebricks.data.IBinaryIntegerInspector;
+import org.apache.hyracks.algebricks.data.IBinaryIntegerInspectorFactory;
+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.evaluators.EvaluatorContext;
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.comm.VSizeFrame;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.ActivityId;
+import org.apache.hyracks.api.dataflow.IActivityGraphBuilder;
+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.util.CleanupUtils;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+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;
+import org.apache.hyracks.dataflow.common.data.accessors.FrameTupleReference;
+import org.apache.hyracks.dataflow.std.base.AbstractActivityNode;
+import org.apache.hyracks.dataflow.std.base.AbstractReplicateOperatorDescriptor;
+import org.apache.hyracks.dataflow.std.base.AbstractUnaryInputOperatorNodePushable;
+
+/**
+ * The switch operator propagates each tuple of the input to one or more output branches based on a given output mapping.
+ * For each tuple, we peek at the value of a given field f. We look up the value of the field in the supplied
+ * output mapping and propagate the tuple to all corresponding output branches.
+ */
+public class SwitchOperatorDescriptor extends AbstractReplicateOperatorDescriptor {
+ private static final long serialVersionUID = 1L;
+ final static int SWITCHER_ACTIVITY_ID = 0;
+
+ private final IScalarEvaluatorFactory branchingExprEvalFactory;
+ private final IBinaryIntegerInspectorFactory intInspectorFactory;
+ private final Map<Integer, int[]> outputMapping;
+
+ /**
+ * @param spec
+ * @param rDesc
+ * @param outputArity equal to the number of non-materialized outputs
+ * @param branchingExprEvalFactory containing the index of the relevant field f
+ * @param intInspectorFactory
+ * @param outputMapping the supplied mapping from field values to arrays of output branch numbers
+ */
+ public SwitchOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor rDesc, int outputArity,
+ IScalarEvaluatorFactory branchingExprEvalFactory, IBinaryIntegerInspectorFactory intInspectorFactory,
+ Map<Integer, int[]> outputMapping) {
+ super(spec, rDesc, outputArity);
+ if (outputArity != numberOfNonMaterializedOutputs) {
+ throw new IllegalArgumentException();
+ }
+ this.branchingExprEvalFactory = branchingExprEvalFactory;
+ this.intInspectorFactory = intInspectorFactory;
+ this.outputMapping = outputMapping;
+ }
+
+ @Override
+ public void contributeActivities(IActivityGraphBuilder builder) {
+ SwitcherActivityNode sma = new SwitcherActivityNode(new ActivityId(odId, SWITCHER_ACTIVITY_ID));
+ builder.addActivity(this, sma);
+ builder.addSourceEdge(0, sma, 0);
+ for (int i = 0; i < outputArity; i++) {
+ builder.addTargetEdge(i, sma, i);
+ }
+ }
+
+ private final class SwitcherActivityNode extends AbstractActivityNode {
+ private static final long serialVersionUID = 1L;
+
+ public SwitcherActivityNode(ActivityId id) {
+ super(id);
+ }
+
+ @Override
+ public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+ IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions)
+ throws HyracksDataException {
+ final IFrameWriter[] writers = new IFrameWriter[outputArity];
+ final boolean[] isOpen = new boolean[outputArity];
+ final IPointable p = VoidPointable.FACTORY.createPointable();
+ // To deal with each tuple in a frame
+ final FrameTupleAccessor accessor = new FrameTupleAccessor(outRecDescs[0]);
+ final FrameTupleAppender[] appenders = new FrameTupleAppender[outputArity];
+ final FrameTupleReference tRef = new FrameTupleReference();
+ final IBinaryIntegerInspector intInspector = intInspectorFactory.createBinaryIntegerInspector(ctx);
+ final IEvaluatorContext evalCtx = new EvaluatorContext(ctx);
+ final IScalarEvaluator eval = branchingExprEvalFactory.createScalarEvaluator(evalCtx);
+ final MutableBoolean hasFailed = new MutableBoolean(false);
+ for (int i = 0; i < outputArity; i++) {
+ appenders[i] = new FrameTupleAppender(new VSizeFrame(ctx), true);
+ }
+
+ return new AbstractUnaryInputOperatorNodePushable() {
+ @Override
+ public void open() throws HyracksDataException {
+ for (int i = 0; i < outputArity; i++) {
+ isOpen[i] = true;
+ writers[i].open();
+ }
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer bufferAccessor) throws HyracksDataException {
+ // Tuple based access
+ accessor.reset(bufferAccessor);
+ int tupleCount = accessor.getTupleCount();
+
+ for (int i = 0; i < tupleCount; i++) {
+ // Get the value of the relevant field in the given tuple.
+ tRef.reset(accessor, i);
+ eval.evaluate(tRef, p);
+ int fieldValue =
+ intInspector.getIntegerValue(p.getByteArray(), p.getStartOffset(), p.getLength());
+
+ // Look up the corresponding output branches based on the given mapping
+ int[] outputBranches = outputMapping.get(fieldValue);
+
+ // Propagate to each output branch
+ for (int j : outputBranches) {
+ FrameUtils.appendToWriter(writers[j], appenders[j], accessor, i);
+ }
+ }
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ Throwable hde = null;
+ // write if hasn't failed
+ if (!hasFailed.booleanValue()) {
+ for (int i = 0; i < outputArity; i++) {
+ if (isOpen[i]) {
+ try {
+ appenders[i].write(writers[i], true);
+ } catch (Throwable th) {
+ hde = th;
+ break;
+ }
+ }
+ }
+ }
+
+ // fail the writers
+ if (hde != null) {
+ for (int i = 0; i < outputArity; i++) {
+ if (isOpen[i]) {
+ CleanupUtils.fail(writers[i], hde);
+ }
+ }
+ }
+
+ // close
+ for (int i = 0; i < outputArity; i++) {
+ if (isOpen[i]) {
+ hde = CleanupUtils.close(writers[i], hde);
+ }
+ }
+ if (hde != null) {
+ throw HyracksDataException.create(hde);
+ }
+ }
+
+ @Override
+ public void fail() throws HyracksDataException {
+ hasFailed.setTrue();
+ HyracksDataException hde = null;
+ for (int i = 0; i < outputArity; i++) {
+ if (isOpen[i]) {
+ try {
+ writers[i].fail();
+ } catch (Throwable th) {
+ if (hde == null) {
+ hde = HyracksDataException.create(th);
+ } else {
+ hde.addSuppressed(th);
+ }
+ }
+ }
+ }
+ if (hde != null) {
+ throw hde;
+ }
+ }
+
+ @Override
+ public void setOutputFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc) {
+ writers[index] = writer;
+ }
+
+ @Override
+ public void flush() throws HyracksDataException {
+ for (int i = 0; i < outputArity; i++) {
+ if (isOpen[i]) {
+ appenders[i].flush(writers[i]);
+ }
+ }
+ }
+ };
+ }
+ }
+}
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/data/device0/data/simple/int-string-part2-switch-0.tbl b/hyracks-fullstack/algebricks/algebricks-tests/data/device0/data/simple/int-string-part2-switch-0.tbl
new file mode 100644
index 0000000..026bbdb
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-tests/data/device0/data/simple/int-string-part2-switch-0.tbl
@@ -0,0 +1,6 @@
+1,second branch1
+2,third branch1
+1,second branch2
+2,third branch2
+1,second branch3
+2,third branch3
diff --git a/hyracks-fullstack/algebricks/algebricks-tests/data/device0/data/simple/int-string-part2-switch-1.tbl b/hyracks-fullstack/algebricks/algebricks-tests/data/device0/data/simple/int-string-part2-switch-1.tbl
new file mode 100644
index 0000000..464a6448
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-tests/data/device0/data/simple/int-string-part2-switch-1.tbl
@@ -0,0 +1,7 @@
+0,first branch1
+2,third branch1
+0,first branch2
+2,third branch2
+0,first branch3
+2,third branch3
+0,first branch4
diff --git a/hyracks-fullstack/algebricks/algebricks-tests/data/device0/data/simple/int-string-part2-switch-2.tbl b/hyracks-fullstack/algebricks/algebricks-tests/data/device0/data/simple/int-string-part2-switch-2.tbl
new file mode 100644
index 0000000..b9eabbc
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-tests/data/device0/data/simple/int-string-part2-switch-2.tbl
@@ -0,0 +1,7 @@
+0,first branch1
+1,second branch1
+0,first branch2
+1,second branch2
+0,first branch3
+1,second branch3
+0,first branch4
diff --git a/hyracks-fullstack/algebricks/algebricks-tests/data/device0/data/simple/int-string-part2.tbl b/hyracks-fullstack/algebricks/algebricks-tests/data/device0/data/simple/int-string-part2.tbl
new file mode 100644
index 0000000..373cd0f
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-tests/data/device0/data/simple/int-string-part2.tbl
@@ -0,0 +1,10 @@
+0|first branch1
+1|second branch1
+2|third branch1
+0|first branch2
+1|second branch2
+2|third branch2
+0|first branch3
+1|second branch3
+2|third branch3
+0|first branch4
diff --git a/hyracks-fullstack/algebricks/algebricks-tests/src/test/java/org/apache/hyracks/algebricks/tests/pushruntime/PushRuntimeTest.java b/hyracks-fullstack/algebricks/algebricks-tests/src/test/java/org/apache/hyracks/algebricks/tests/pushruntime/PushRuntimeTest.java
index aeb22b6..d23f7f9 100644
--- a/hyracks-fullstack/algebricks/algebricks-tests/src/test/java/org/apache/hyracks/algebricks/tests/pushruntime/PushRuntimeTest.java
+++ b/hyracks-fullstack/algebricks/algebricks-tests/src/test/java/org/apache/hyracks/algebricks/tests/pushruntime/PushRuntimeTest.java
@@ -23,6 +23,7 @@
import java.io.FileReader;
import java.io.IOException;
import java.util.Collections;
+import java.util.HashMap;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.commons.io.FileUtils;
@@ -58,6 +59,7 @@
import org.apache.hyracks.algebricks.runtime.operators.std.StreamProjectRuntimeFactory;
import org.apache.hyracks.algebricks.runtime.operators.std.StreamSelectRuntimeFactory;
import org.apache.hyracks.algebricks.runtime.operators.std.StringStreamingRuntimeFactory;
+import org.apache.hyracks.algebricks.runtime.operators.std.SwitchOperatorDescriptor;
import org.apache.hyracks.algebricks.runtime.operators.std.UnnestRuntimeFactory;
import org.apache.hyracks.algebricks.runtime.writers.PrinterBasedWriterFactory;
import org.apache.hyracks.algebricks.tests.util.AlgebricksHyracksIntegrationUtil;
@@ -698,6 +700,76 @@
}
@Test
+ public void scanSwitchWrite() throws Exception {
+ final int outputArity = 3;
+
+ JobSpecification spec = new JobSpecification(FRAME_SIZE);
+
+ String inputFileName[] = { "data" + File.separator + "simple" + File.separator + "int-string-part2.tbl",
+ "data" + File.separator + "simple" + File.separator + "int-string-part2-switch-0.tbl",
+ "data" + File.separator + "simple" + File.separator + "int-string-part2-switch-1.tbl",
+ "data" + File.separator + "simple" + File.separator + "int-string-part2-switch-2.tbl" };
+ File[] inputFiles = new File[inputFileName.length];
+ for (int i = 0; i < inputFileName.length; i++) {
+ inputFiles[i] = new File(inputFileName[i]);
+ }
+ File[] outputFile = new File[outputArity];
+ FileSplit[] outputFileSplit = new FileSplit[outputArity];
+ for (int i = 0; i < outputArity; i++) {
+ outputFileSplit[i] = createFile(AlgebricksHyracksIntegrationUtil.nc1);
+ outputFile[i] = outputFileSplit[i].getFile(AlgebricksHyracksIntegrationUtil.nc1.getIoManager());
+ }
+
+ FileSplit[] inputSplits =
+ new FileSplit[] { new ManagedFileSplit(AlgebricksHyracksIntegrationUtil.NC1_ID, inputFileName[0]) };
+ IFileSplitProvider intSplitProvider = new ConstantFileSplitProvider(inputSplits);
+
+ RecordDescriptor scannerDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ IntegerSerializerDeserializer.INSTANCE, new UTF8StringSerializerDeserializer() });
+
+ IValueParserFactory[] valueParsers =
+ new IValueParserFactory[] { IntegerParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE };
+
+ FileScanOperatorDescriptor intScanner = new FileScanOperatorDescriptor(spec, intSplitProvider,
+ new DelimitedDataTupleParserFactory(valueParsers, '|'), scannerDesc);
+
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, intScanner, DEFAULT_NODES);
+
+ HashMap<Integer, int[]> outputMapping = new HashMap<>();
+ outputMapping.put(0, new int[] { 1, 2 });
+ outputMapping.put(1, new int[] { 0, 2 });
+ outputMapping.put(2, new int[] { 0, 1 });
+
+ SwitchOperatorDescriptor switchOp = new SwitchOperatorDescriptor(spec, scannerDesc, outputArity,
+ new TupleFieldEvaluatorFactory(0), BinaryIntegerInspectorImpl.FACTORY, outputMapping);
+
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, switchOp,
+ new String[] { AlgebricksHyracksIntegrationUtil.NC1_ID });
+
+ IOperatorDescriptor outputOp[] = new IOperatorDescriptor[outputFile.length];
+ for (int i = 0; i < outputArity; i++) {
+ outputOp[i] = new LineFileWriteOperatorDescriptor(spec, new FileSplit[] { outputFileSplit[i] });
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, outputOp[i],
+ new String[] { AlgebricksHyracksIntegrationUtil.NC1_ID });
+ }
+
+ spec.connect(new OneToOneConnectorDescriptor(spec), intScanner, 0, switchOp, 0);
+ for (int i = 0; i < outputArity; i++) {
+ spec.connect(new OneToOneConnectorDescriptor(spec), switchOp, i, outputOp[i], 0);
+ }
+
+ for (int i = 0; i < outputArity; i++) {
+ spec.addRoot(outputOp[i]);
+ }
+ AlgebricksHyracksIntegrationUtil.runJob(spec);
+
+ for (int i = 0; i < outputArity; i++) {
+ compareFiles("data" + File.separator + "device0" + File.separator + inputFileName[i + 1],
+ outputFile[i].getAbsolutePath());
+ }
+ }
+
+ @Test
public void scanMicroSortWrite() throws Exception {
JobSpecification spec = new JobSpecification(FRAME_SIZE);
diff --git a/hyracks-fullstack/hyracks-fullstack-license/src/main/licenses/templates/hyracks-notice.ftl b/hyracks-fullstack/hyracks-fullstack-license/src/main/licenses/templates/hyracks-notice.ftl
index 8a2eb05..0cf9378 100644
--- a/hyracks-fullstack/hyracks-fullstack-license/src/main/licenses/templates/hyracks-notice.ftl
+++ b/hyracks-fullstack/hyracks-fullstack-license/src/main/licenses/templates/hyracks-notice.ftl
@@ -30,6 +30,9 @@
<#assign noticeText = e.getKey()/>
<#assign projects = e.getValue()/>
<#list projects as p>
+ <#if p.shadowed>
+ <#continue/>
+ </#if>
${p.name} (${p.groupId}:${p.artifactId}:${p.version})
<#list p.locations as loc>
- ${loc}${p.jarName}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksClientInterfaceFunctions.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksClientInterfaceFunctions.java
index 0ee3658..7a320b1 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksClientInterfaceFunctions.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksClientInterfaceFunctions.java
@@ -346,12 +346,19 @@
}
public static class WaitForCompletionFunction extends Function {
- private static final long serialVersionUID = 1L;
+ private static final long serialVersionUID = 2L;
private final JobId jobId;
+ private final List<String> statOperatorNames;
+
public WaitForCompletionFunction(JobId jobId) {
+ this(jobId, null);
+ }
+
+ public WaitForCompletionFunction(JobId jobId, List<String> statOperatorNames) {
this.jobId = jobId;
+ this.statOperatorNames = statOperatorNames;
}
@Override
@@ -362,6 +369,10 @@
public JobId getJobId() {
return jobId;
}
+
+ public List<String> getStatOperatorNames() {
+ return statOperatorNames;
+ }
}
public static class GetNodeControllersInfoFunction extends Function {
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientConnection.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientConnection.java
index 1118a68..be965eb 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientConnection.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientConnection.java
@@ -31,6 +31,7 @@
import org.apache.hyracks.api.job.JobInfo;
import org.apache.hyracks.api.job.JobSpecification;
import org.apache.hyracks.api.job.JobStatus;
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
/**
* Interface used by clients to communicate with the Hyracks Cluster Controller.
@@ -157,6 +158,19 @@
void waitForCompletion(JobId jobId) throws Exception;
/**
+ * Waits until the specified job has completed, either successfully or has
+ * encountered a permanent failure.
+ *
+ * @param jobId
+ * JobId of the Job
+ * @param statsOperatorNames
+ * names of the operators which stats should be returned
+ * @return stats for the specified operators
+ * @throws Exception
+ */
+ List<IOperatorStats> waitForCompletion(JobId jobId, List<String> statsOperatorNames) throws Exception;
+
+ /**
* Deploy files to the cluster
*
* @param files
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientInterface.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientInterface.java
index 4cc47d2..be47014 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientInterface.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientInterface.java
@@ -30,6 +30,7 @@
import org.apache.hyracks.api.job.JobId;
import org.apache.hyracks.api.job.JobInfo;
import org.apache.hyracks.api.job.JobStatus;
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
import org.apache.hyracks.api.topology.ClusterTopology;
public interface IHyracksClientInterface {
@@ -53,6 +54,8 @@
public void waitForCompletion(JobId jobId) throws Exception;
+ public List<IOperatorStats> waitForCompletion(JobId jobId, List<String> statOperatorNames) throws Exception;
+
public Map<String, NodeControllerInfo> getNodeControllersInfo() throws Exception;
public ClusterTopology getClusterTopology() throws Exception;
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/context/IHyracksTaskContext.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/context/IHyracksTaskContext.java
index 4561af1..ad5624c 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/context/IHyracksTaskContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/context/IHyracksTaskContext.java
@@ -42,6 +42,8 @@
TaskAttemptId getTaskAttemptId();
+ int getPartitionCount();
+
ICounterContext getCounterContext();
ExecutorService getExecutorService();
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IIntrospectingOperator.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IIntrospectingOperator.java
new file mode 100644
index 0000000..64897ef
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IIntrospectingOperator.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.dataflow;
+
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
+
+public interface IIntrospectingOperator {
+ void setOperatorStats(IOperatorStats stats);
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/ProfiledFrameWriter.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/ProfiledFrameWriter.java
new file mode 100644
index 0000000..5b9495a
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/ProfiledFrameWriter.java
@@ -0,0 +1,182 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.dataflow;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hyracks.api.comm.FrameConstants;
+import org.apache.hyracks.api.comm.FrameHelper;
+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.job.profiling.IOperatorStats;
+import org.apache.hyracks.api.job.profiling.IStatsCollector;
+import org.apache.hyracks.api.job.profiling.OperatorStats;
+import org.apache.hyracks.api.job.profiling.counters.ICounter;
+import org.apache.hyracks.util.IntSerDeUtils;
+
+public class ProfiledFrameWriter implements IFrameWriter, IPassableTimer {
+
+ // The downstream data consumer of this writer.
+ private final IFrameWriter writer;
+ private long frameStart = 0;
+ final ICounter timeCounter;
+ final ICounter tupleCounter;
+ final IStatsCollector collector;
+ final IOperatorStats stats;
+ final IOperatorStats parentStats;
+ private int minSz = Integer.MAX_VALUE;
+ private int maxSz = -1;
+ private long avgSz;
+ final String name;
+
+ public ProfiledFrameWriter(IFrameWriter writer, IStatsCollector collector, String name, IOperatorStats stats,
+ IOperatorStats parentStats) {
+ this.writer = writer;
+ this.collector = collector;
+ this.name = name;
+ this.stats = stats;
+ this.parentStats = parentStats;
+ this.timeCounter = stats.getTimeCounter();
+ this.tupleCounter = parentStats != null ? parentStats.getTupleCounter() : null;
+ }
+
+ @Override
+ public final void open() throws HyracksDataException {
+ try {
+ startClock();
+ writer.open();
+ } finally {
+ stopClock();
+ }
+ }
+
+ @Override
+ public final void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ try {
+ int tupleCountOffset = FrameHelper.getTupleCountOffset(buffer.limit());
+ int tupleCount = IntSerDeUtils.getInt(buffer.array(), tupleCountOffset);
+ if (tupleCounter != null) {
+ long prevCount = tupleCounter.get();
+ for (int i = 0; i < tupleCount; i++) {
+ int tupleLen = getTupleLength(i, tupleCountOffset, buffer);
+ if (maxSz < tupleLen) {
+ maxSz = tupleLen;
+ }
+ if (minSz > tupleLen) {
+ minSz = tupleLen;
+ }
+ long prev = avgSz * prevCount;
+ avgSz = (prev + tupleLen) / (prevCount + 1);
+ prevCount++;
+ }
+ parentStats.getMaxTupleSz().set(maxSz);
+ parentStats.getMinTupleSz().set(minSz);
+ parentStats.getAverageTupleSz().set(avgSz);
+ tupleCounter.update(tupleCount);
+ }
+ startClock();
+ writer.nextFrame(buffer);
+ } finally {
+ stopClock();
+ }
+ }
+
+ @Override
+ public final void flush() throws HyracksDataException {
+ try {
+ startClock();
+ writer.flush();
+ } finally {
+ stopClock();
+ }
+ }
+
+ @Override
+ public final void fail() throws HyracksDataException {
+ writer.fail();
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ try {
+ startClock();
+ writer.close();
+ } finally {
+ stopClock();
+ }
+ }
+
+ private void stopClock() {
+ pause();
+ collector.giveClock(this);
+ }
+
+ private void startClock() {
+ if (frameStart > 0) {
+ return;
+ }
+ frameStart = collector.takeClock(this);
+ }
+
+ @Override
+ public void resume() {
+ if (frameStart > 0) {
+ return;
+ }
+ long nt = System.nanoTime();
+ frameStart = nt;
+ }
+
+ @Override
+ public void pause() {
+ if (frameStart > 1) {
+ long nt = System.nanoTime();
+ long delta = nt - frameStart;
+ timeCounter.update(delta);
+ frameStart = -1;
+ }
+ }
+
+ private int getTupleStartOffset(int tupleIndex, int tupleCountOffset, ByteBuffer buffer) {
+ return tupleIndex == 0 ? FrameConstants.TUPLE_START_OFFSET
+ : IntSerDeUtils.getInt(buffer.array(), tupleCountOffset - FrameConstants.SIZE_LEN * tupleIndex);
+ }
+
+ private int getTupleEndOffset(int tupleIndex, int tupleCountOffset, ByteBuffer buffer) {
+ return IntSerDeUtils.getInt(buffer.array(), tupleCountOffset - FrameConstants.SIZE_LEN * (tupleIndex + 1));
+ }
+
+ public int getTupleLength(int tupleIndex, int tupleCountOffset, ByteBuffer buffer) {
+ return getTupleEndOffset(tupleIndex, tupleCountOffset, buffer)
+ - getTupleStartOffset(tupleIndex, tupleCountOffset, buffer);
+ }
+
+ public static IFrameWriter time(IFrameWriter writer, IHyracksTaskContext ctx, String name)
+ throws HyracksDataException {
+ if (!(writer instanceof ProfiledFrameWriter)) {
+ IStatsCollector statsCollector = ctx.getStatsCollector();
+ IOperatorStats stats = new OperatorStats(name);
+ statsCollector.add(stats);
+ return new ProfiledFrameWriter(writer, ctx.getStatsCollector(), name, stats, null);
+
+ } else
+ return writer;
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/ProfiledOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/ProfiledOperatorNodePushable.java
new file mode 100644
index 0000000..f787a1c
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/ProfiledOperatorNodePushable.java
@@ -0,0 +1,159 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.dataflow;
+
+import java.util.HashMap;
+
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+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 org.apache.hyracks.api.rewriter.runtime.SuperActivityOperatorNodePushable;
+
+public class ProfiledOperatorNodePushable extends ProfiledFrameWriter implements IOperatorNodePushable, IPassableTimer {
+
+ IOperatorNodePushable op;
+ ProfiledOperatorNodePushable parentOp;
+ ActivityId acId;
+ HashMap<Integer, IFrameWriter> inputs;
+ long frameStart;
+
+ ProfiledOperatorNodePushable(IOperatorNodePushable op, ActivityId acId, IStatsCollector collector,
+ IOperatorStats stats, ActivityId parent, ProfiledOperatorNodePushable parentOp)
+ throws HyracksDataException {
+ super(null, collector, acId.toString() + " - " + op.getDisplayName(), stats,
+ parentOp != null ? parentOp.getStats() : null);
+ this.parentOp = parentOp;
+ this.op = op;
+ this.acId = acId;
+ inputs = new HashMap<>();
+ }
+
+ @Override
+ public void initialize() throws HyracksDataException {
+ synchronized (collector) {
+ startClock();
+ op.initialize();
+ stopClock();
+ }
+ }
+
+ @Override
+ public void deinitialize() throws HyracksDataException {
+ synchronized (collector) {
+ startClock();
+ op.deinitialize();
+ stopClock();
+ }
+ }
+
+ @Override
+ public int getInputArity() {
+ return op.getInputArity();
+ }
+
+ @Override
+ public void setOutputFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc)
+ throws HyracksDataException {
+ op.setOutputFrameWriter(index, writer, recordDesc);
+ }
+
+ @Override
+ public IFrameWriter getInputFrameWriter(int index) {
+ IFrameWriter ifw = op.getInputFrameWriter(index);
+ if (!(op instanceof ProfiledFrameWriter) && ifw.equals(op)) {
+ return new ProfiledFrameWriter(op.getInputFrameWriter(index), collector,
+ acId.toString() + "-" + op.getDisplayName(), stats, parentStats);
+ }
+ return op.getInputFrameWriter(index);
+ }
+
+ @Override
+ public String getDisplayName() {
+ return op.getDisplayName();
+ }
+
+ private void stopClock() {
+ pause();
+ collector.giveClock(this);
+ }
+
+ private void startClock() {
+ if (frameStart > 0) {
+ return;
+ }
+ frameStart = collector.takeClock(this);
+ }
+
+ @Override
+ public void resume() {
+ if (frameStart > 0) {
+ return;
+ }
+ long nt = System.nanoTime();
+ frameStart = nt;
+ }
+
+ @Override
+ public void pause() {
+ if (frameStart > 0) {
+ long nt = System.nanoTime();
+ long delta = nt - frameStart;
+ timeCounter.update(delta);
+ frameStart = -1;
+ }
+ }
+
+ public IOperatorStats getStats() {
+ return stats;
+ }
+
+ public IOperatorStats getParentStats() {
+ return parentStats;
+ }
+
+ public static IOperatorNodePushable time(IOperatorNodePushable op, IHyracksTaskContext ctx, ActivityId acId,
+ ProfiledOperatorNodePushable source) throws HyracksDataException {
+ String name = acId.toString() + " - " + op.getDisplayName();
+ IStatsCollector statsCollector = ctx.getStatsCollector();
+ IOperatorStats stats = new OperatorStats(name, acId.getOperatorDescriptorId());
+ statsCollector.add(stats);
+ if (op instanceof IIntrospectingOperator) {
+ ((IIntrospectingOperator) op).setOperatorStats(stats);
+ }
+ if (!(op instanceof ProfiledOperatorNodePushable) && !(op instanceof SuperActivityOperatorNodePushable)) {
+ return new ProfiledOperatorNodePushable(op, acId, ctx.getStatsCollector(), stats, acId, source);
+ }
+ return op;
+ }
+
+ public static void onlyAddStats(IOperatorNodePushable op, IHyracksTaskContext ctx, ActivityId acId)
+ throws HyracksDataException {
+ String name = acId.toString() + " - " + op.getDisplayName();
+ IStatsCollector statsCollector = ctx.getStatsCollector();
+ IOperatorStats stats = new OperatorStats(name, acId.getOperatorDescriptorId());
+ if (op instanceof IIntrospectingOperator) {
+ ((IIntrospectingOperator) op).setOperatorStats(stats);
+ statsCollector.add(stats);
+ }
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/TimedFrameWriter.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/TimedFrameWriter.java
deleted file mode 100644
index 83a4b34..0000000
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/TimedFrameWriter.java
+++ /dev/null
@@ -1,130 +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.api.dataflow;
-
-import java.nio.ByteBuffer;
-
-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.job.profiling.IStatsCollector;
-import org.apache.hyracks.api.job.profiling.counters.ICounter;
-
-public class TimedFrameWriter implements IFrameWriter, IPassableTimer {
-
- // The downstream data consumer of this writer.
- private final IFrameWriter writer;
- private long frameStart = 0;
- final ICounter counter;
- final IStatsCollector collector;
- final String name;
-
- public TimedFrameWriter(IFrameWriter writer, IStatsCollector collector, String name, ICounter counter) {
- this.writer = writer;
- this.collector = collector;
- this.name = name;
- this.counter = counter;
- }
-
- protected TimedFrameWriter(IFrameWriter writer, IStatsCollector collector, String name) {
- this(writer, collector, name, collector.getOrAddOperatorStats(name).getTimeCounter());
- }
-
- @Override
- public final void open() throws HyracksDataException {
- try {
- startClock();
- writer.open();
- } finally {
- stopClock();
- }
- }
-
- @Override
- public final void nextFrame(ByteBuffer buffer) throws HyracksDataException {
- try {
- startClock();
- writer.nextFrame(buffer);
- } finally {
- stopClock();
- }
- }
-
- @Override
- public final void flush() throws HyracksDataException {
- try {
- startClock();
- writer.flush();
- } finally {
- stopClock();
- }
- }
-
- @Override
- public final void fail() throws HyracksDataException {
- writer.fail();
- }
-
- @Override
- public void close() throws HyracksDataException {
- try {
- startClock();
- writer.close();
- } finally {
- stopClock();
- }
- }
-
- private void stopClock() {
- pause();
- collector.giveClock(this);
- }
-
- private void startClock() {
- if (frameStart > 0) {
- return;
- }
- frameStart = collector.takeClock(this);
- }
-
- @Override
- public void resume() {
- if (frameStart > 0) {
- return;
- }
- long nt = System.nanoTime();
- frameStart = nt;
- }
-
- @Override
- public void pause() {
- if (frameStart > 1) {
- long nt = System.nanoTime();
- long delta = nt - frameStart;
- counter.update(delta);
- frameStart = -1;
- }
- }
-
- public static IFrameWriter time(IFrameWriter writer, IHyracksTaskContext ctx, String name)
- throws HyracksDataException {
- return writer instanceof TimedFrameWriter ? writer
- : new TimedFrameWriter(writer, ctx.getStatsCollector(), name);
- }
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/TimedOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/TimedOperatorNodePushable.java
deleted file mode 100644
index 2d46bea..0000000
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/TimedOperatorNodePushable.java
+++ /dev/null
@@ -1,123 +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.api.dataflow;
-
-import java.util.HashMap;
-
-import org.apache.hyracks.api.comm.IFrameWriter;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.job.profiling.IStatsCollector;
-import org.apache.hyracks.api.rewriter.runtime.SuperActivityOperatorNodePushable;
-
-public class TimedOperatorNodePushable extends TimedFrameWriter implements IOperatorNodePushable, IPassableTimer {
-
- IOperatorNodePushable op;
- HashMap<Integer, IFrameWriter> inputs;
- long frameStart;
-
- TimedOperatorNodePushable(IOperatorNodePushable op, IStatsCollector collector) throws HyracksDataException {
- super(null, collector, op.getDisplayName());
- this.op = op;
- inputs = new HashMap<>();
- }
-
- @Override
- public void initialize() throws HyracksDataException {
- synchronized (collector) {
- startClock();
- op.initialize();
- stopClock();
- }
- }
-
- @Override
- public void deinitialize() throws HyracksDataException {
- synchronized (collector) {
- startClock();
- op.deinitialize();
- stopClock();
- }
- }
-
- @Override
- public int getInputArity() {
- return op.getInputArity();
- }
-
- @Override
- public void setOutputFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc)
- throws HyracksDataException {
- op.setOutputFrameWriter(index, writer, recordDesc);
- }
-
- @Override
- public IFrameWriter getInputFrameWriter(int index) {
- IFrameWriter ifw = op.getInputFrameWriter(index);
- if (!(op instanceof TimedFrameWriter) && ifw.equals(op)) {
- return new TimedFrameWriter(op.getInputFrameWriter(index), collector, op.getDisplayName(), counter);
- }
- return op.getInputFrameWriter(index);
- }
-
- @Override
- public String getDisplayName() {
- return op.getDisplayName();
- }
-
- private void stopClock() {
- pause();
- collector.giveClock(this);
- }
-
- private void startClock() {
- if (frameStart > 0) {
- return;
- }
- frameStart = collector.takeClock(this);
- }
-
- @Override
- public void resume() {
- if (frameStart > 0) {
- return;
- }
- long nt = System.nanoTime();
- frameStart = nt;
- }
-
- @Override
- public void pause() {
- if (frameStart > 0) {
- long nt = System.nanoTime();
- long delta = nt - frameStart;
- counter.update(delta);
- frameStart = -1;
- }
- }
-
- public static IOperatorNodePushable time(IOperatorNodePushable op, IHyracksTaskContext ctx)
- throws HyracksDataException {
- if (!(op instanceof TimedOperatorNodePushable) && !(op instanceof SuperActivityOperatorNodePushable)) {
- return new TimedOperatorNodePushable(op, ctx.getStatsCollector());
- }
- return op;
- }
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksDataException.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksDataException.java
index 6deff8d..fdda793 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksDataException.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksDataException.java
@@ -104,7 +104,7 @@
private HyracksDataException(HyracksDataException hde, String nodeId) {
super(hde.getError().orElse(null), hde.getComponent(), hde.getErrorCode(), hde.getMessage(), hde.getCause(),
- hde.getSourceLocation(), nodeId, hde.getStackTrace(), hde.getParams());
+ hde.getSourceLocation(), nodeId, hde.getParams());
}
protected HyracksDataException(IError error, Throwable cause, SourceLocation sourceLoc, Serializable... params) {
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/JobSpecification.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/JobSpecification.java
index 58336a0..2c51d3d 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
@@ -21,6 +21,7 @@
import java.io.IOException;
import java.io.Serializable;
import java.util.ArrayList;
+import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
@@ -49,7 +50,7 @@
import com.fasterxml.jackson.databind.node.ObjectNode;
public class JobSpecification implements Serializable, IOperatorDescriptorRegistry, IConnectorDescriptorRegistry {
- private static final long serialVersionUID = 1L;
+ private static final long serialVersionUID = 2L;
private static final int DEFAULT_FRAME_SIZE = 32768;
@@ -67,6 +68,8 @@
private final Map<ConnectorDescriptorId, Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>>> connectorOpMap;
+ private transient Map<Object, String> logical2PhysicalMap;
+
private final Map<String, Serializable> properties;
private final Set<Constraint> userConstraints;
@@ -108,6 +111,7 @@
opOutputMap = new HashMap<>();
connectorOpMap = new HashMap<>();
properties = new HashMap<>();
+ logical2PhysicalMap = Collections.emptyMap();
userConstraints = new HashSet<>();
operatorIdCounter = 0;
connectorIdCounter = 0;
@@ -326,6 +330,14 @@
return metaOps;
}
+ public void setLogical2PhysicalMap(Map<Object, String> logical2PhysicalMap) {
+ this.logical2PhysicalMap = logical2PhysicalMap;
+ }
+
+ public Map<Object, String> getLogical2PhysicalMap() {
+ return logical2PhysicalMap;
+ }
+
private <K, V> void insertIntoIndexedMap(Map<K, List<V>> map, K key, int index, V value) {
List<V> vList = map.computeIfAbsent(key, k -> new ArrayList<>());
extend(vList, index);
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..26b3a58 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,7 +19,9 @@
package org.apache.hyracks.api.job.profiling;
import java.io.Serializable;
+import java.util.Map;
+import org.apache.hyracks.api.dataflow.OperatorDescriptorId;
import org.apache.hyracks.api.io.IWritable;
import org.apache.hyracks.api.job.profiling.counters.ICounter;
@@ -43,7 +45,52 @@
ICounter getTimeCounter();
/**
- * @return A counter used to track the number of pages pinned by an opeartor
+ * @return A counter used to track the number of pages pinned by an operator
*/
- ICounter getDiskIoCounter();
+ ICounter getPageReads();
+
+ /**
+ * @return A counter used to track the number of pages read from disk by an operator
+ */
+
+ ICounter coldReadCounter();
+
+ /**
+ * @return A counter used to set the average tuple size outputted by an operator
+ */
+
+ ICounter getAverageTupleSz();
+
+ /**
+ * @return A counter used to set the max tuple size outputted by an operator
+ */
+
+ ICounter getMaxTupleSz();
+
+ /**
+ * @return A counter used to set the min tuple size outputted by an operator
+ */
+
+ ICounter getMinTupleSz();
+
+ /**
+ * @return A counter used to track the number of tuples read by operators that originate data,
+ * like index searches or other scan types
+ */
+
+ ICounter getInputTupleCounter();
+
+ ICounter getLevel();
+
+ ICounter getBytesRead();
+
+ ICounter getBytesWritten();
+
+ OperatorDescriptorId getId();
+
+ 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..d2ad20c 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
@@ -38,9 +38,9 @@
/**
* @param operatorName
* @return {@link IOperatorStats} for the operator with name <code>operatorName</code>
- * if it already exists, and adds it if it does not.
+ * if one exists or else null.
*/
- IOperatorStats getOrAddOperatorStats(String operatorName);
+ IOperatorStats getOperatorStats(String operatorName);
/**
* Get every registered operator stats object
@@ -67,5 +67,4 @@
* @param currHolder the timer that needs to be paused
*/
void giveClock(IPassableTimer currHolder);
-
}
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/NoOpOperatorStats.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/profiling/NoOpOperatorStats.java
new file mode 100644
index 0000000..4c86a15
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/profiling/NoOpOperatorStats.java
@@ -0,0 +1,151 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.util.Collections;
+import java.util.Map;
+
+import org.apache.hyracks.api.dataflow.OperatorDescriptorId;
+import org.apache.hyracks.api.job.profiling.counters.ICounter;
+
+public class NoOpOperatorStats implements IOperatorStats {
+
+ private static final long serialVersionUID = 9055940822300360135L;
+
+ public static final NoOpOperatorStats INSTANCE = new NoOpOperatorStats();
+
+ private static final ICounter NOOP_COUNTER = new ICounter() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public long update(long delta) {
+ return 0;
+ }
+
+ @Override
+ public long set(long value) {
+ return 0;
+ }
+
+ @Override
+ public String getName() {
+ return null;
+ }
+
+ @Override
+ public long get() {
+ return 0;
+ }
+ };
+
+ private static final OperatorDescriptorId INVALID_ODID = new OperatorDescriptorId(-1);
+
+ @Override
+ public void writeFields(DataOutput output) throws IOException {
+
+ }
+
+ @Override
+ public void readFields(DataInput input) throws IOException {
+
+ }
+
+ @Override
+ public String getName() {
+ return "";
+ }
+
+ @Override
+ public ICounter getTupleCounter() {
+ return NOOP_COUNTER;
+ }
+
+ @Override
+ public ICounter getTimeCounter() {
+ return NOOP_COUNTER;
+ }
+
+ @Override
+ public ICounter getPageReads() {
+ return NOOP_COUNTER;
+ }
+
+ @Override
+ public ICounter coldReadCounter() {
+ return NOOP_COUNTER;
+ }
+
+ @Override
+ public ICounter getAverageTupleSz() {
+ return NOOP_COUNTER;
+ }
+
+ @Override
+ public ICounter getMaxTupleSz() {
+ return NOOP_COUNTER;
+ }
+
+ @Override
+ public ICounter getMinTupleSz() {
+ return NOOP_COUNTER;
+ }
+
+ @Override
+ public ICounter getInputTupleCounter() {
+ return NOOP_COUNTER;
+ }
+
+ @Override
+ public ICounter getLevel() {
+ return NOOP_COUNTER;
+ }
+
+ @Override
+ public ICounter getBytesRead() {
+ return NOOP_COUNTER;
+ }
+
+ @Override
+ public ICounter getBytesWritten() {
+ return NOOP_COUNTER;
+ }
+
+ @Override
+ public OperatorDescriptorId getId() {
+ return INVALID_ODID;
+ }
+
+ @Override
+ public void updateIndexesStats(Map<String, IndexStats> indexesStats) {
+ // no op
+ }
+
+ @Override
+ public Map<String, IndexStats> getIndexesStats() {
+ return Collections.emptyMap();
+ }
+
+ @Override
+ public void updateFrom(IOperatorStats stats) {
+ // no op
+ }
+}
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..c9f08e0 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,26 +21,56 @@
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.dataflow.OperatorDescriptorId;
import org.apache.hyracks.api.job.profiling.counters.ICounter;
public class OperatorStats implements IOperatorStats {
- private static final long serialVersionUID = 6401830963367567167L;
+ private static final long serialVersionUID = 6401830963367567169L;
public final String operatorName;
+ public OperatorDescriptorId id;
public final ICounter tupleCounter;
public final ICounter timeCounter;
- public final ICounter diskIoCounter;
+ public final ICounter pageReads;
+ public final ICounter coldReadCounter;
+ public final ICounter avgTupleSz;
+ public final ICounter minTupleSz;
+ public final ICounter maxTupleSz;
+ public final ICounter inputTupleCounter;
+ public final ICounter level;
+ public final ICounter bytesRead;
+ public final ICounter bytesWritten;
+ private final Map<String, IndexStats> indexesStats;
- public OperatorStats(String operatorName) {
+ //TODO: this is quickly becoming gross it should just be a map where the value is obliged to be a Counter
+
+ public OperatorStats(String operatorName, OperatorDescriptorId id) {
if (operatorName == null || operatorName.isEmpty()) {
throw new IllegalArgumentException("operatorName must not be null or empty");
}
this.operatorName = operatorName;
+ this.id = id;
tupleCounter = new Counter("tupleCounter");
timeCounter = new Counter("timeCounter");
- diskIoCounter = new Counter("diskIoCounter");
+ pageReads = new Counter("diskIoCounter");
+ coldReadCounter = new Counter("coldReadCounter");
+ avgTupleSz = new Counter("avgTupleSz");
+ minTupleSz = new Counter("minTupleSz");
+ maxTupleSz = new Counter("maxTupleSz");
+ inputTupleCounter = new Counter("inputTupleCounter");
+ level = new Counter("level");
+ bytesRead = new Counter("bytesRead");
+ bytesWritten = new Counter("bytesWritten");
+ level.set(-1);
+ indexesStats = new HashMap<>();
+ }
+
+ public OperatorStats(String operatorName) {
+ this(operatorName, new OperatorDescriptorId(-1));
}
public static IOperatorStats create(DataInput input) throws IOException {
@@ -66,28 +96,146 @@
}
@Override
- public ICounter getDiskIoCounter() {
- return diskIoCounter;
+ public ICounter getPageReads() {
+ return pageReads;
+ }
+
+ @Override
+ public ICounter coldReadCounter() {
+ return coldReadCounter;
+ }
+
+ @Override
+ public ICounter getAverageTupleSz() {
+ return avgTupleSz;
+ }
+
+ @Override
+ public ICounter getMaxTupleSz() {
+ return maxTupleSz;
+ }
+
+ @Override
+ public ICounter getMinTupleSz() {
+ return minTupleSz;
+ }
+
+ @Override
+ public ICounter getInputTupleCounter() {
+ return inputTupleCounter;
+ }
+
+ @Override
+ public ICounter getLevel() {
+ return level;
+ }
+
+ @Override
+ public ICounter getBytesRead() {
+ return bytesRead;
+ }
+
+ @Override
+ public ICounter getBytesWritten() {
+ return bytesWritten;
+ }
+
+ @Override
+ public OperatorDescriptorId getId() {
+ return id;
+ }
+
+ @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());
+ pageReads.update(stats.getPageReads().get());
+ updateIndexesStats(stats.getIndexesStats());
}
@Override
public void writeFields(DataOutput output) throws IOException {
output.writeUTF(operatorName);
+ id.writeFields(output);
output.writeLong(tupleCounter.get());
output.writeLong(timeCounter.get());
- output.writeLong(diskIoCounter.get());
+ output.writeLong(pageReads.get());
+ output.writeLong(coldReadCounter.get());
+ output.writeLong(avgTupleSz.get());
+ output.writeLong(minTupleSz.get());
+ output.writeLong(maxTupleSz.get());
+ output.writeLong(inputTupleCounter.get());
+ output.writeLong(level.get());
+ output.writeLong(bytesRead.get());
+ output.writeLong(bytesWritten.get());
+ writeIndexesStats(output);
}
@Override
public void readFields(DataInput input) throws IOException {
+ id = OperatorDescriptorId.create(input);
tupleCounter.set(input.readLong());
timeCounter.set(input.readLong());
- diskIoCounter.set(input.readLong());
+ pageReads.set(input.readLong());
+ coldReadCounter.set(input.readLong());
+ avgTupleSz.set(input.readLong());
+ minTupleSz.set(input.readLong());
+ maxTupleSz.set(input.readLong());
+ inputTupleCounter.set(input.readLong());
+ level.set(input.readLong());
+ bytesRead.set(input.readLong());
+ bytesWritten.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() + ", \""
+ + coldReadCounter.getName() + "\": " + coldReadCounter.get() + avgTupleSz.getName() + "\": "
+ + avgTupleSz.get() + ", \"" + minTupleSz.getName() + "\": " + minTupleSz.get() + ", \""
+ + minTupleSz.getName() + "\": " + timeCounter.get() + ", \"" + inputTupleCounter.getName() + "\": "
+ + bytesRead.get() + ", \"" + bytesRead.getName() + "\": " + bytesWritten.get() + ", \""
+ + bytesWritten.getName() + "\": " + inputTupleCounter.get() + ", \"" + level.getName() + "\": "
+ + level.get() + ", \"indexStats\": \"" + indexesStats + "\" }";
}
}
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..efd4e07 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
@@ -43,7 +43,7 @@
import org.apache.hyracks.api.dataflow.IActivity;
import org.apache.hyracks.api.dataflow.IConnectorDescriptor;
import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
-import org.apache.hyracks.api.dataflow.TimedOperatorNodePushable;
+import org.apache.hyracks.api.dataflow.ProfiledOperatorNodePushable;
import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider;
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -109,10 +109,12 @@
for (Entry<ActivityId, IActivity> entry : startActivities.entrySet()) {
IOperatorNodePushable opPushable = null;
if (profile) {
- opPushable = TimedOperatorNodePushable
- .time(entry.getValue().createPushRuntime(ctx, recordDescProvider, partition, nPartitions), ctx);
+ IOperatorNodePushable wrapped =
+ entry.getValue().createPushRuntime(ctx, recordDescProvider, partition, nPartitions);
+ opPushable = ProfiledOperatorNodePushable.time(wrapped, ctx, entry.getKey(), null);
} else {
opPushable = entry.getValue().createPushRuntime(ctx, recordDescProvider, partition, nPartitions);
+ ProfiledOperatorNodePushable.onlyAddStats(opPushable, ctx, entry.getKey());
}
operatorNodePushablesBFSOrder.add(opPushable);
operatorNodePushables.put(entry.getKey(), opPushable);
@@ -141,11 +143,18 @@
IOperatorNodePushable destOp = operatorNodePushables.get(destId);
if (destOp == null) {
if (profile) {
- destOp = TimedOperatorNodePushable.time(channel.getRight().getLeft().createPushRuntime(ctx,
- recordDescProvider, partition, nPartitions), ctx);
+ IOperatorNodePushable wrapped = channel.getRight().getLeft().createPushRuntime(ctx,
+ recordDescProvider, partition, nPartitions);
+ if (sourceOp instanceof ProfiledOperatorNodePushable) {
+ destOp = ProfiledOperatorNodePushable.time(wrapped, ctx, destId,
+ (ProfiledOperatorNodePushable) sourceOp);
+ } else {
+ destOp = ProfiledOperatorNodePushable.time(wrapped, ctx, destId, null);
+ }
} else {
destOp = channel.getRight().getLeft().createPushRuntime(ctx, recordDescProvider, partition,
nPartitions);
+ ProfiledOperatorNodePushable.onlyAddStats(destOp, ctx, destId);
}
operatorNodePushablesBFSOrder.add(destOp);
operatorNodePushables.put(destId, destOp);
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/HyracksConstants.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/HyracksConstants.java
index c34a671..3c829e6 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/HyracksConstants.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/HyracksConstants.java
@@ -28,6 +28,8 @@
public static final String INDEX_CURSOR_STATS = "INDEX_CURSOR_STATS";
+ public static final String TUPLE_PROJECTOR = "TUPLE_PROJECTOR";
+
private HyracksConstants() {
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-client/pom.xml b/hyracks-fullstack/hyracks/hyracks-client/pom.xml
index a1d146d..c5fed16 100644
--- a/hyracks-fullstack/hyracks/hyracks-client/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-client/pom.xml
@@ -101,6 +101,11 @@
<version>${project.version}</version>
</dependency>
<dependency>
+ <groupId>org.apache.hyracks</groupId>
+ <artifactId>hyracks-util</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
<groupId>junit</groupId>
<artifactId>junit</artifactId>
<scope>test</scope>
diff --git a/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/result/ResultDirectory.java b/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/result/ResultDirectory.java
index b335a93..5dd5fff 100644
--- a/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/result/ResultDirectory.java
+++ b/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/result/ResultDirectory.java
@@ -18,6 +18,7 @@
*/
package org.apache.hyracks.client.result;
+import java.io.Closeable;
import java.io.IOException;
import java.net.InetSocketAddress;
@@ -35,7 +36,7 @@
import org.apache.hyracks.ipc.impl.JavaSerializationBasedPayloadSerializerDeserializer;
//TODO(madhusudancs): Should this implementation be moved to org.apache.hyracks.client?
-public class ResultDirectory implements IResultDirectory {
+public class ResultDirectory implements IResultDirectory, Closeable {
private final IPCSystem ipc;
private final IResultDirectory remoteResultDirectory;
@@ -64,4 +65,9 @@
public IResultMetadata getResultMetadata(JobId jobId, ResultSetId rsId) throws Exception {
return remoteResultDirectory.getResultMetadata(jobId, rsId);
}
+
+ @Override
+ public void close() throws IOException {
+ ipc.stop();
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/result/ResultSet.java b/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/result/ResultSet.java
index 4d8767f..8a88045 100644
--- a/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/result/ResultSet.java
+++ b/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/result/ResultSet.java
@@ -18,6 +18,9 @@
*/
package org.apache.hyracks.client.result;
+import java.io.Closeable;
+import java.io.IOException;
+
import org.apache.hyracks.api.client.IHyracksClientConnection;
import org.apache.hyracks.api.comm.NetworkAddress;
import org.apache.hyracks.api.context.IHyracksCommonContext;
@@ -25,15 +28,15 @@
import org.apache.hyracks.api.io.IIOManager;
import org.apache.hyracks.api.job.JobId;
import org.apache.hyracks.api.network.ISocketChannelFactory;
-import org.apache.hyracks.api.result.IResultDirectory;
import org.apache.hyracks.api.result.IResultSet;
import org.apache.hyracks.api.result.IResultSetReader;
import org.apache.hyracks.api.result.ResultSetId;
import org.apache.hyracks.client.net.ClientNetworkManager;
import org.apache.hyracks.control.nc.resources.memory.FrameManager;
+import org.apache.hyracks.util.NetworkUtil;
-public class ResultSet implements IResultSet {
- private final IResultDirectory resultDirectory;
+public class ResultSet implements IResultSet, Closeable {
+ private final ResultDirectory resultDirectory;
private final ClientNetworkManager netManager;
@@ -51,6 +54,15 @@
}
@Override
+ public void close() throws IOException {
+ try {
+ netManager.stop();
+ } finally {
+ NetworkUtil.closeQuietly(resultDirectory);
+ }
+ }
+
+ @Override
public IResultSetReader createReader(JobId jobId, ResultSetId resultSetId) throws HyracksDataException {
IResultSetReader reader = null;
try {
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClientInterfaceIPCI.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClientInterfaceIPCI.java
index e6973dd..6da8fdf 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClientInterfaceIPCI.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClientInterfaceIPCI.java
@@ -147,8 +147,8 @@
case WAIT_FOR_COMPLETION:
HyracksClientInterfaceFunctions.WaitForCompletionFunction wfcf =
(HyracksClientInterfaceFunctions.WaitForCompletionFunction) fn;
- ccs.getWorkQueue()
- .schedule(new WaitForJobCompletionWork(ccs, wfcf.getJobId(), new IPCResponder<>(handle, mid)));
+ ccs.getWorkQueue().schedule(new WaitForJobCompletionWork(ccs, wfcf.getJobId(),
+ wfcf.getStatOperatorNames(), new IPCResponder<>(handle, mid)));
break;
case GET_NODE_CONTROLLERS_INFO:
ccs.getWorkQueue().schedule(
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/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..d865b4f 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
@@ -111,10 +111,11 @@
pmm = new PartitionMatchMaker();
participatingNodeIds = new HashSet<>();
cleanupPendingNodeIds = new HashSet<>();
- profile = new JobProfile(jobId);
connectorPolicyMap = new HashMap<>();
operatorLocations = new HashMap<>();
createTime = System.currentTimeMillis();
+ profile = new JobProfile(jobId);
+ profile.setCreateTime(createTime);
}
//Run a deployed job spec
@@ -200,6 +201,7 @@
public void setStartTime(long startTime) {
this.startTime = startTime;
+ this.profile.setStartTime(startTime);
}
public String getStartTimeZoneId() {
@@ -208,6 +210,7 @@
public void setStartTimeZoneId(String startTimeZoneId) {
this.startTimeZoneId = startTimeZoneId;
+ this.profile.setStartTimeZoneId(startTimeZoneId);
}
public long getEndTime() {
@@ -216,6 +219,7 @@
public void setEndTime(long endTime) {
this.endTime = endTime;
+ this.profile.setEndTime(endTime);
}
public void registerOperatorLocation(OperatorDescriptorId op, int partition, String location) {
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/NodeHeartbeatWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/NodeHeartbeatWork.java
index b772ef9..6fa6224 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/NodeHeartbeatWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/NodeHeartbeatWork.java
@@ -28,9 +28,12 @@
import org.apache.hyracks.control.common.heartbeat.HeartbeatData;
import org.apache.hyracks.control.common.ipc.NodeControllerRemoteProxy;
import org.apache.logging.log4j.Level;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
public class NodeHeartbeatWork extends AbstractHeartbeatWork {
+ private static final Logger LOGGER = LogManager.getLogger();
private final InetSocketAddress ncAddress;
public NodeHeartbeatWork(ClusterControllerService ccs, String nodeId, HeartbeatData hbData,
@@ -47,6 +50,8 @@
ncState.getNodeController().heartbeatAck(ccs.getCcId(), null);
} else {
// unregistered nc- let him know
+ LOGGER.info("received a heartbeat from unregistered node {}; sending negative ack to node address {}",
+ nodeId, ncAddress);
NodeControllerRemoteProxy nc =
new NodeControllerRemoteProxy(ccs.getCcId(), ccs.getClusterIPC().getReconnectingHandle(ncAddress));
nc.heartbeatAck(ccs.getCcId(), HyracksDataException.create(ErrorCode.NO_SUCH_NODE, nodeId));
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/WaitForJobCompletionWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/WaitForJobCompletionWork.java
index 911bedf..14d92fb 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/WaitForJobCompletionWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/WaitForJobCompletionWork.java
@@ -33,11 +33,14 @@
public class WaitForJobCompletionWork extends SynchronizableWork {
private final ClusterControllerService ccs;
private final JobId jobId;
+ private final List<String> statOperatorNames;
private final IResultCallback<Object> callback;
- public WaitForJobCompletionWork(ClusterControllerService ccs, JobId jobId, IResultCallback<Object> callback) {
+ public WaitForJobCompletionWork(ClusterControllerService ccs, JobId jobId, List<String> statOperatorNames,
+ IResultCallback<Object> callback) {
this.ccs = ccs;
this.jobId = jobId;
+ this.statOperatorNames = statOperatorNames;
this.callback = callback;
}
@@ -53,7 +56,7 @@
Thread.currentThread()
.setName(Thread.currentThread().getName() + " : WaitForCompletionForJobId: " + jobId);
jobRun.waitForCompletion();
- callback.setValue(null);
+ callback.setValue(jobRun.getJobProfile().getAggregatedStats(statOperatorNames));
} catch (Exception e) {
callback.setException(e);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NCConfig.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NCConfig.java
index 01cb9bf..bb40e2b 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NCConfig.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NCConfig.java
@@ -99,6 +99,7 @@
PYTHON_USE_BUNDLED_MSGPACK(BOOLEAN, true),
PYTHON_ARGS(STRING_ARRAY, (String[]) null),
PYTHON_ENV(STRING_ARRAY, (String[]) null),
+ PYTHON_DS_PATH(STRING, (String) null),
CREDENTIAL_FILE(
OptionTypes.STRING,
(Function<IApplicationConfig, String>) appConfig -> FileUtil
@@ -248,6 +249,8 @@
return "Whether or not to attempt to automatically set PYTHON_CMD to a usable interpreter";
case PYTHON_ENV:
return "List of environment variables to set when invoking the Python interpreter for Python UDFs. E.g. FOO=1";
+ case PYTHON_DS_PATH:
+ return "Path to systemd socket for fenced Python UDFs. Requires JDK17+, *nix operating system, and ";
case CREDENTIAL_FILE:
return "Path to HTTP basic credentials";
default:
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..41beac0 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
@@ -30,10 +30,11 @@
import org.apache.hyracks.api.dataflow.IPassableTimer;
import org.apache.hyracks.api.job.profiling.IOperatorStats;
import org.apache.hyracks.api.job.profiling.IStatsCollector;
+import org.apache.hyracks.api.job.profiling.NoOpOperatorStats;
import org.apache.hyracks.api.job.profiling.OperatorStats;
public class StatsCollector implements IStatsCollector {
- private static final long serialVersionUID = 6858817639895434578L;
+ private static final long serialVersionUID = 6858817639895434572L;
private final Map<String, IOperatorStats> operatorStatsMap = new LinkedHashMap<>();
private transient Deque<IPassableTimer> clockHolder = new ArrayDeque<>();
@@ -47,8 +48,8 @@
}
@Override
- public IOperatorStats getOrAddOperatorStats(String operatorName) {
- return operatorStatsMap.computeIfAbsent(operatorName, OperatorStats::new);
+ public IOperatorStats getOperatorStats(String operatorName) {
+ return operatorStatsMap.getOrDefault(operatorName, NoOpOperatorStats.INSTANCE);
}
@Override
@@ -66,9 +67,9 @@
public IOperatorStats getAggregatedStats() {
IOperatorStats aggregatedStats = new OperatorStats("aggregated");
for (IOperatorStats stats : operatorStatsMap.values()) {
- aggregatedStats.getTupleCounter().update(stats.getTupleCounter().get());
+ aggregatedStats.getInputTupleCounter().update(stats.getInputTupleCounter().get());
aggregatedStats.getTimeCounter().update(stats.getTimeCounter().get());
- aggregatedStats.getDiskIoCounter().update(stats.getDiskIoCounter().get());
+ aggregatedStats.getPageReads().update(stats.getPageReads().get());
}
return aggregatedStats;
}
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..f4cd4d6 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,20 +21,32 @@
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.NoOpOperatorStats;
+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;
+ private long createTime;
+ private long startTime;
+ private String startTimeZoneId;
+ private long endTime;
private Map<String, JobletProfile> jobletProfiles;
@@ -61,12 +73,32 @@
return jobletProfiles;
}
+ public void setCreateTime(long createTime) {
+ this.createTime = createTime;
+ }
+
+ public void setStartTime(long startTime) {
+ this.startTime = startTime;
+ }
+
+ public void setStartTimeZoneId(String startTimeZoneId) {
+ this.startTimeZoneId = startTimeZoneId;
+ }
+
+ public void setEndTime(long endTime) {
+ this.endTime = endTime;
+ }
+
@Override
public ObjectNode toJSON() {
ObjectMapper om = new ObjectMapper();
ObjectNode json = om.createObjectNode();
json.put("job-id", jobId.toString());
+ json.put("create-time", createTime);
+ json.put("start-time", startTime);
+ json.put("queued-time", startTime - createTime);
+ json.put("end-time", endTime);
populateCounters(json);
ArrayNode jobletsArray = om.createArrayNode();
for (JobletProfile p : jobletProfiles.values()) {
@@ -91,6 +123,10 @@
@Override
public void readFields(DataInput input) throws IOException {
jobId = JobId.create(input);
+ createTime = input.readLong();
+ startTime = input.readLong();
+ endTime = input.readLong();
+ startTimeZoneId = input.readUTF();
int size = input.readInt();
jobletProfiles = new HashMap<>();
for (int i = 0; i < size; i++) {
@@ -103,10 +139,53 @@
@Override
public void writeFields(DataOutput output) throws IOException {
jobId.writeFields(output);
+ output.writeLong(createTime);
+ output.writeLong(startTime);
+ output.writeLong(endTime);
+ output.writeUTF(startTimeZoneId);
output.writeInt(jobletProfiles.size());
for (Entry<String, JobletProfile> entry : jobletProfiles.entrySet()) {
output.writeUTF(entry.getKey());
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.equals(NoOpOperatorStats.INSTANCE)) {
+ 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-common/src/main/java/org/apache/hyracks/control/common/job/profiling/om/TaskProfile.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/om/TaskProfile.java
index dab6d26..4036f00 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/om/TaskProfile.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/om/TaskProfile.java
@@ -130,7 +130,28 @@
jpe.put("name", key);
jpe.put("time", Double
.parseDouble(new DecimalFormat("#.####").format((double) value.getTimeCounter().get() / 1000000)));
- jpe.put("disk-io", value.getDiskIoCounter().get());
+ if (value.getId().getId() >= 0) {
+ jpe.put("runtime-id", value.getId().toString());
+ }
+ if (value.getPageReads().get() > 0) {
+ jpe.put("pages-read", value.getPageReads().get());
+ jpe.put("pages-read-cold", value.coldReadCounter().get());
+ }
+ if (value.getTupleCounter().get() > 0) {
+ jpe.put("cardinality-out", value.getTupleCounter().get());
+ jpe.put("avg-tuple-size", value.getAverageTupleSz().get());
+ jpe.put("min-tuple-size", value.getMinTupleSz().get());
+ jpe.put("max-tuple-size", value.getMaxTupleSz().get());
+ }
+ if (value.getLevel().get() > -1) {
+ jpe.put("level", value.getLevel().get());
+ }
+ if (value.getBytesRead().get() > 0) {
+ jpe.put("bytes-read", value.getBytesRead().get());
+ }
+ if (value.getBytesWritten().get() > 0) {
+ jpe.put("bytes-written", value.getBytesWritten().get());
+ }
countersObj.add(jpe);
});
json.set("counters", countersObj);
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/Task.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/Task.java
index 158e24e..900ac7e 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/Task.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/Task.java
@@ -87,6 +87,8 @@
private final TaskAttemptId taskAttemptId;
+ private final int partitionCount;
+
private final String displayName;
private final ExecutorService executorService;
@@ -131,12 +133,13 @@
private final Map<Long, IThreadStats> perThreadStats = new HashMap<>();
- public Task(Joblet joblet, Set<JobFlag> jobFlags, TaskAttemptId taskId, String displayName,
+ public Task(Joblet joblet, Set<JobFlag> jobFlags, TaskAttemptId taskId, int partitionCount, String displayName,
ExecutorService executor, NodeControllerService ncs,
List<List<PartitionChannel>> inputChannelsFromConnectors) {
this.joblet = joblet;
this.jobFlags = jobFlags;
this.taskAttemptId = taskId;
+ this.partitionCount = partitionCount;
this.displayName = displayName;
this.executorService = executor;
fileFactory = new WorkspaceFileFactory(this, joblet.getIoManager());
@@ -225,6 +228,11 @@
}
@Override
+ public int getPartitionCount() {
+ return partitionCount;
+ }
+
+ @Override
public ICounter getCounter(String name, boolean create) {
Counter counter = counterMap.get(name);
if (counter == null && create) {
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/StartTasksWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/StartTasksWork.java
index 7d84fe9..f6c144d 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/StartTasksWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/StartTasksWork.java
@@ -152,11 +152,12 @@
ActivityCluster ac = acg.getActivityMap().get(aid);
IActivity han = ac.getActivityMap().get(aid);
LOGGER.trace("Initializing {} -> {} for {}", taId, han, jobId);
+ int partitionCount = td.getPartitionCount();
final int partition = tid.getPartition();
List<IConnectorDescriptor> inputs = ac.getActivityInputMap().get(aid);
- task = new Task(joblet, flags, taId, han.getDisplayName(), ncs.getExecutor(), ncs,
+ task = new Task(joblet, flags, taId, partitionCount, han.getDisplayName(), ncs.getExecutor(), ncs,
createInputChannels(td, inputs));
- IOperatorNodePushable operator = han.createPushRuntime(task, rdp, partition, td.getPartitionCount());
+ IOperatorNodePushable operator = han.createPushRuntime(task, rdp, partition, partitionCount);
List<IPartitionCollector> collectors = new ArrayList<>();
if (inputs != null) {
for (int i = 0; i < inputs.size(); ++i) {
@@ -182,7 +183,7 @@
createPartitionWriterFactory(task, cPolicy, jobId, conn, partition, taId, flags);
LOGGER.trace("input: {}: {}", i, conn.getConnectorId());
IFrameWriter writer = conn.createPartitioner(task, recordDesc, pwFactory, partition,
- td.getPartitionCount(), td.getOutputPartitionCounts()[i]);
+ partitionCount, td.getOutputPartitionCounts()[i]);
writer = (enforce && !profile) ? EnforceFrameWriter.enforce(writer) : writer;
operator.setOutputFrameWriter(i, writer, recordDesc);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-nc-service/pom.xml b/hyracks-fullstack/hyracks/hyracks-control/hyracks-nc-service/pom.xml
index 0d411cf..9852592 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-nc-service/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-nc-service/pom.xml
@@ -57,7 +57,6 @@
<dependency>
<groupId>com.e-movimento.tinytools</groupId>
<artifactId>privilegedaccessor</artifactId>
- <version>1.2.2</version>
<scope>test</scope>
</dependency>
<dependency>
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..8013e05 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
@@ -33,6 +33,7 @@
import org.apache.hyracks.data.std.api.IHashable;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.api.IPointableFactory;
+import org.apache.hyracks.data.std.api.IValueReference;
import org.apache.hyracks.data.std.util.GrowableArray;
import org.apache.hyracks.data.std.util.UTF8StringBuilder;
import org.apache.hyracks.util.string.UTF8StringUtil;
@@ -108,8 +109,7 @@
* Returns the character at the given byte offset. The caller is responsible for making sure that
* the provided offset is within bounds and points to the beginning of a valid UTF8 character.
*
- * @param offset
- * - Byte offset
+ * @param offset - Byte offset
* @return Character at the given offset.
*/
public char charAt(int offset) {
@@ -218,13 +218,15 @@
pointable2.utf8Length);
}
+ public static int compare(IValueReference valueA, IValueReference valueB) {
+ return UTF8StringUtil.compareTo(valueA.getByteArray(), valueA.getStartOffset(), valueA.getLength(),
+ valueB.getByteArray(), valueB.getStartOffset(), valueB.getLength());
+ }
+
/**
- * @param src,
- * the source string.
- * @param pattern,
- * the pattern string.
- * @param ignoreCase,
- * to ignore case or not.
+ * @param src, the source string.
+ * @param pattern, the pattern string.
+ * @param ignoreCase, 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) {
@@ -232,12 +234,9 @@
}
/**
- * @param src,
- * the source string.
- * @param pattern,
- * the pattern string.
- * @param ignoreCase,
- * to ignore case or not.
+ * @param src, the source string.
+ * @param pattern, the pattern string.
+ * @param ignoreCase, 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) {
@@ -245,30 +244,22 @@
}
/**
- * @param src,
- * the source string.
- * @param pattern,
- * the pattern string.
- * @param ignoreCase,
- * to ignore case or not.
- * @param startMatch,
- * the start offset.
+ * @param src, the source string.
+ * @param pattern, the pattern string.
+ * @param ignoreCase, to ignore case or not.
+ * @param startMatch, the start offset.
* @return the byte offset of the first character of the matching string after <code>startMatchPos}</code>.
- * Not including the MetaLength.
+ * Not including the MetaLength.
*/
public static int find(UTF8StringPointable src, UTF8StringPointable pattern, boolean ignoreCase, int startMatch) {
return findInByteOrCodePoint(src, pattern, ignoreCase, startMatch, true);
}
/**
- * @param src,
- * the source string.
- * @param pattern,
- * the pattern string.
- * @param ignoreCase,
- * to ignore case or not.
- * @param startMatch,
- * the start offset.
+ * @param src, the source string.
+ * @param pattern, the pattern string.
+ * @param ignoreCase, to ignore case or not.
+ * @param startMatch, the start offset.
* @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,
@@ -324,7 +315,7 @@
}
// The result is counted in code point instead of bytes
- if (resultInByte == false) {
+ if (!resultInByte) {
char ch = src.charAt(srcStart + startMatchPos);
if (Character.isHighSurrogate(ch)) {
prevHighSurrogate = true;
@@ -431,9 +422,10 @@
/**
* Return the substring. Note that the offset and length are in the unit of code point.
+ *
* @return {@code true} if substring was successfully written into given {@code out}, or
- * {@code false} if substring could not be obtained ({@code codePointOffset} or {@code codePointLength}
- * are less than 0 or starting position is greater than the input length)
+ * {@code false} if substring could not be obtained ({@code codePointOffset} or {@code codePointLength}
+ * are less than 0 or starting position is greater than the input length)
*/
public boolean substr(int codePointOffset, int codePointLength, UTF8StringBuilder builder, GrowableArray out)
throws IOException {
@@ -442,9 +434,10 @@
/**
* Return the substring. Note that the offset and length are in the unit of code point.
+ *
* @return {@code true} if substring was successfully written into given {@code out}, or
- * {@code false} if substring could not be obtained ({@code codePointOffset} or {@code codePointLength}
- * are less than 0 or starting position is greater than the input length)
+ * {@code false} if substring could not be obtained ({@code codePointOffset} or {@code codePointLength}
+ * are less than 0 or starting position is greater than the input length)
*/
public static boolean substr(UTF8StringPointable src, int codePointOffset, int codePointLength,
UTF8StringBuilder builder, GrowableArray out) throws IOException {
@@ -548,12 +541,9 @@
/**
* Generates a lower case string of an input string.
*
- * @param src
- * , the input source string.
- * @param builder
- * , a builder for the resulting string.
- * @param out
- * , the storage for a result string.
+ * @param src , the input source string.
+ * @param builder , a builder for the resulting string.
+ * @param out , the storage for a result string.
* @throws IOException
*/
public static void lowercase(UTF8StringPointable src, UTF8StringBuilder builder, GrowableArray out)
@@ -577,12 +567,9 @@
/**
* Generates an upper case string of an input string.
*
- * @param src
- * , the input source string.
- * @param builder
- * , a builder for the resulting string.
- * @param out
- * , the storage for a result string.
+ * @param src , the input source string.
+ * @param builder , a builder for the resulting string.
+ * @param out , the storage for a result string.
* @throws IOException
*/
public static void uppercase(UTF8StringPointable src, UTF8StringBuilder builder, GrowableArray out)
@@ -607,12 +594,9 @@
* Generates a "title" format string from an input source string, i.e., the first letter of each word
* is in the upper case while the other letter is in the lower case.
*
- * @param src
- * , the input source string.
- * @param builder
- * , a builder for the resulting string.
- * @param out
- * , the storage for a result string.
+ * @param src , the input source string.
+ * @param builder , a builder for the resulting string.
+ * @param out , the storage for a result string.
* @throws IOException
*/
public static void initCap(UTF8StringPointable src, UTF8StringBuilder builder, GrowableArray out)
@@ -642,18 +626,12 @@
/**
* Generates a trimmed string of an input source string.
*
- * @param srcPtr
- * , the input source string
- * @param builder
- * , the result string builder.
- * @param out
- * , the storage for the output string.
- * @param left
- * , whether to trim the left side.
- * @param right
- * , whether to trim the right side.
- * @param codePointSet
- * , the set of code points that should be trimmed.
+ * @param srcPtr , the input source string
+ * @param builder , the result string builder.
+ * @param out , the storage for the output string.
+ * @param left , whether to trim the left side.
+ * @param right , whether to trim the right side.
+ * @param codePointSet , the set of code points that should be trimmed.
* @throws IOException
*/
public static void trim(UTF8StringPointable srcPtr, UTF8StringBuilder builder, GrowableArray out, boolean left,
@@ -696,16 +674,11 @@
/**
* Generates a trimmed string from the original string.
*
- * @param builder
- * , the result string builder.
- * @param out
- * , the storage for the output string.
- * @param left
- * , whether to trim the left side.
- * @param right
- * , whether to trim the right side.
- * @param codePointSet
- * , the set of code points that should be trimmed.
+ * @param builder , the result string builder.
+ * @param out , the storage for the output string.
+ * @param left , whether to trim the left side.
+ * @param right , whether to trim the right side.
+ * @param codePointSet , the set of code points that should be trimmed.
* @throws IOException
*/
public void trim(UTF8StringBuilder builder, GrowableArray out, boolean left, boolean right,
@@ -716,12 +689,9 @@
/**
* Generates a reversed string from an input source string
*
- * @param srcPtr
- * , the input source string.
- * @param builder
- * , a builder for the resulting string.
- * @param out
- * , the storage for a result string.
+ * @param srcPtr , the input source string.
+ * @param builder , a builder for the resulting string.
+ * @param out , the storage for a result string.
* @throws IOException
*/
public static void reverse(UTF8StringPointable srcPtr, UTF8StringBuilder builder, GrowableArray out)
@@ -739,7 +709,7 @@
cursorIndex--;
if (UTF8StringUtil.isCharStart(srcPtr.bytes, cursorIndex)) {
ch = UTF8StringUtil.charAt(srcPtr.bytes, cursorIndex);
- if (Character.isHighSurrogate(ch) == false) {
+ if (!Character.isHighSurrogate(ch)) {
throw new IllegalArgumentException(
"Decoding Error: no corresponding high surrogate found for the following low surrogate");
}
diff --git a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/util/ArrayBackedValueStorage.java b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/util/ArrayBackedValueStorage.java
index d5a4481..d4feff6 100644
--- a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/util/ArrayBackedValueStorage.java
+++ b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/util/ArrayBackedValueStorage.java
@@ -65,8 +65,12 @@
}
public void append(IValueReference value) throws HyracksDataException {
+ append(value.getByteArray(), value.getStartOffset(), value.getLength());
+ }
+
+ public void append(byte[] bytes, int start, int length) throws HyracksDataException {
try {
- data.append(value);
+ data.append(bytes, start, length);
} catch (IOException e) {
throw HyracksDataException.create(e);
}
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-std/pom.xml b/hyracks-fullstack/hyracks/hyracks-dataflow-std/pom.xml
index d2eb362..6789025 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/pom.xml
@@ -83,7 +83,6 @@
<dependency>
<groupId>com.e-movimento.tinytools</groupId>
<artifactId>privilegedaccessor</artifactId>
- <version>1.2.2</version>
<scope>test</scope>
</dependency>
<dependency>
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractOperatorNodePushable.java
index 3fa88bb..5559663 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractOperatorNodePushable.java
@@ -18,11 +18,18 @@
*/
package org.apache.hyracks.dataflow.std.base;
+import org.apache.hyracks.api.dataflow.IIntrospectingOperator;
import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
-public abstract class AbstractOperatorNodePushable implements IOperatorNodePushable {
+public abstract class AbstractOperatorNodePushable implements IOperatorNodePushable, IIntrospectingOperator {
@Override
public String getDisplayName() {
return toString();
}
+
+ @Override
+ public void setOperatorStats(IOperatorStats stats) {
+ }
+
}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractReplicateOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractReplicateOperatorDescriptor.java
index 4c728ce..666c250 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractReplicateOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractReplicateOperatorDescriptor.java
@@ -35,10 +35,11 @@
import org.apache.hyracks.dataflow.std.misc.MaterializerTaskState;
/**
- * Abstract class for two replication related operator descriptor - replicate and split
+ * Abstract class for three replication related operator descriptors - replicate, split, and switch.
* Replicate operator propagates all frames to all output branches.
* That is, each tuple will be propagated to all output branches.
* Split operator propagates each tuple in a frame to one output branch only.
+ * Switch is a generalization of split that propagates tuples based on a given output mapping.
*/
public abstract class AbstractReplicateOperatorDescriptor extends AbstractOperatorDescriptor {
protected static final long serialVersionUID = 1L;
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IPartitionedTupleBufferManager.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IPartitionedTupleBufferManager.java
index 8051305..190baa2 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IPartitionedTupleBufferManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IPartitionedTupleBufferManager.java
@@ -75,6 +75,15 @@
throws HyracksDataException;
/**
+ * Returns the number of frames needed to accommodate the tuple.
+ *
+ * @param tupleSize tuple size
+ * @param fieldCount field count. 0 if the tuple size already accounts for fields offsets size.
+ * @return the number of frames needed to accommodate the tuple.
+ */
+ int framesNeeded(int tupleSize, int fieldCount);
+
+ /**
* Cancels the effect of last insertTuple() operation. i.e. undoes the last insertTuple() operation.
*/
void cancelInsertTuple(int partition) throws HyracksDataException;
@@ -108,9 +117,10 @@
*
* @param pid
* @param writer
+ * @return number of bytes spilled
* @throws HyracksDataException
*/
- void flushPartition(int pid, IFrameWriter writer) throws HyracksDataException;
+ int flushPartition(int pid, IFrameWriter writer) throws HyracksDataException;
/**
* Clear the memory occupation of the particular partition.
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/PreferToSpillFullyOccupiedFramePolicy.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/PreferToSpillFullyOccupiedFramePolicy.java
index 12985c0..613a396 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/PreferToSpillFullyOccupiedFramePolicy.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/PreferToSpillFullyOccupiedFramePolicy.java
@@ -98,4 +98,14 @@
}
};
}
+
+ public String partitionsStatus() {
+ StringBuilder sb = new StringBuilder();
+ int numPartitions = bufferManager.getNumPartitions();
+ for (int p = 0; p < numPartitions; p++) {
+ sb.append("p:").append(p).append(",#t:").append(bufferManager.getNumTuples(p)).append(",s:")
+ .append(spilledStatus.get(p)).append(",s:").append(bufferManager.getPhysicalSize(p)).append('\n');
+ }
+ return sb.toString();
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VPartitionTupleBufferManager.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VPartitionTupleBufferManager.java
index d3d06cb..9cabb8a 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VPartitionTupleBufferManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VPartitionTupleBufferManager.java
@@ -132,15 +132,15 @@
@Override
public boolean insertTuple(int partition, byte[] byteArray, int[] fieldEndOffsets, int start, int size,
TuplePointer pointer) throws HyracksDataException {
- int actualSize = calculateActualSize(fieldEndOffsets, size);
- int fid = getLastBufferOrCreateNewIfNotExist(partition, actualSize);
+ int fieldCount = fieldEndOffsets == null ? 0 : fieldEndOffsets.length;
+ int fid = getLastBufferOrCreateNewIfNotExist(partition, size, fieldCount);
if (fid < 0) {
return false;
}
partitionArray[partition].getFrame(fid, tempInfo);
int tid = appendTupleToBuffer(tempInfo, fieldEndOffsets, byteArray, start, size);
if (tid < 0) {
- fid = createNewBuffer(partition, actualSize);
+ fid = createNewBuffer(partition, size, fieldCount);
if (fid < 0) {
return false;
}
@@ -170,6 +170,12 @@
numTuples[partition]--;
}
+ @Override
+ public int framesNeeded(int tupleSize, int fieldCount) {
+ int minFrameSize = framePool.getMinFrameSize();
+ return FrameHelper.calcAlignedFrameSizeToStore(fieldCount, tupleSize, minFrameSize) / minFrameSize;
+ }
+
public static int calculateActualSize(int[] fieldEndOffsets, int size) {
if (fieldEndOffsets != null) {
return FrameHelper.calcRequiredSpace(fieldEndOffsets.length, size);
@@ -189,8 +195,8 @@
return externalFrameId / getNumPartitions();
}
- private int createNewBuffer(int partition, int size) throws HyracksDataException {
- ByteBuffer newBuffer = requestNewBufferFromPool(size, partition);
+ private int createNewBuffer(int partition, int tupleSize, int fieldCount) throws HyracksDataException {
+ ByteBuffer newBuffer = requestNewBufferFromPool(tupleSize, partition, fieldCount);
if (newBuffer == null) {
return -1;
}
@@ -199,9 +205,11 @@
return partitionArray[partition].insertFrame(newBuffer);
}
- private ByteBuffer requestNewBufferFromPool(int recordSize, int partition) throws HyracksDataException {
- int frameSize = FrameHelper.calcAlignedFrameSizeToStore(0, recordSize, framePool.getMinFrameSize());
- if ((double) frameSize / (double) framePool.getMinFrameSize() + getPhysicalSize(partition) > constrain
+ private ByteBuffer requestNewBufferFromPool(int recordSize, int partition, int fieldCount)
+ throws HyracksDataException {
+ int minFrameSize = framePool.getMinFrameSize();
+ int frameSize = FrameHelper.calcAlignedFrameSizeToStore(fieldCount, recordSize, minFrameSize);
+ if ((double) frameSize / (double) minFrameSize + getPhysicalSize(partition) / (double) minFrameSize > constrain
.frameLimit(partition)) {
return null;
}
@@ -238,10 +246,11 @@
}
}
- private int getLastBufferOrCreateNewIfNotExist(int partition, int actualSize) throws HyracksDataException {
+ private int getLastBufferOrCreateNewIfNotExist(int partition, int tupleSize, int fieldCount)
+ throws HyracksDataException {
if (partitionArray[partition] == null || partitionArray[partition].getNumFrames() == 0) {
partitionArray[partition] = new FrameBufferManager();
- return createNewBuffer(partition, actualSize);
+ return createNewBuffer(partition, tupleSize, fieldCount);
}
return getLastBuffer(partition);
}
@@ -281,17 +290,20 @@
}
@Override
- public void flushPartition(int pid, IFrameWriter writer) throws HyracksDataException {
+ public int flushPartition(int pid, IFrameWriter writer) throws HyracksDataException {
IFrameBufferManager partition = partitionArray[pid];
+ int written = 0;
if (partition != null && getNumTuples(pid) > 0) {
for (int i = 0; i < partition.getNumFrames(); ++i) {
partition.getFrame(i, tempInfo);
tempInfo.getBuffer().position(tempInfo.getStartOffset());
tempInfo.getBuffer().limit(tempInfo.getStartOffset() + tempInfo.getLength());
+ int sz = tempInfo.getLength();
writer.nextFrame(tempInfo.getBuffer());
+ written += sz;
}
}
-
+ return written;
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/sort/SortGroupByOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/sort/SortGroupByOperatorDescriptor.java
index c545e7d..815536b 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/sort/SortGroupByOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/sort/SortGroupByOperatorDescriptor.java
@@ -37,7 +37,7 @@
import org.apache.hyracks.dataflow.std.sort.AbstractSorterOperatorDescriptor;
import org.apache.hyracks.dataflow.std.sort.Algorithm;
import org.apache.hyracks.dataflow.std.sort.IRunGenerator;
-import org.apache.hyracks.dataflow.std.sort.TimedRunGenerator;
+import org.apache.hyracks.dataflow.std.sort.ProfiledRunGenerator;
/**
* This Operator pushes group-by aggregation into the external sort.
@@ -148,7 +148,8 @@
recordDescriptorProvider.getInputRecordDescriptor(this.getActivityId(), 0), framesLimit,
groupFields, keyNormalizerFactories, comparatorFactories, partialAggregatorFactory,
partialAggRecordDesc, ALG);
- return profile ? TimedRunGenerator.time(runGen, ctx, "GroupBy (Sort Runs)") : runGen;
+ return profile ? ProfiledRunGenerator.time(runGen, ctx, "GroupBy (Sort Runs)", this.getActivityId())
+ : runGen;
}
};
}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
index 7a9bb25..04f5fe8 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
@@ -34,6 +34,7 @@
import org.apache.hyracks.api.exceptions.ErrorCode;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
import org.apache.hyracks.api.util.CleanupUtils;
import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
@@ -49,6 +50,8 @@
import org.apache.hyracks.dataflow.std.structures.ISerializableTable;
import org.apache.hyracks.dataflow.std.structures.SerializableHashTable;
import org.apache.hyracks.dataflow.std.structures.TuplePointer;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
/**
* This class mainly applies one level of HHJ on a pair of
@@ -56,6 +59,7 @@
*/
public class OptimizedHybridHashJoin {
+ private static final Logger LOGGER = LogManager.getLogger();
// Used for special probe BigObject which can not be held into the Join memory
private FrameTupleAppender bigFrameAppender;
@@ -91,6 +95,7 @@
// corresponding function signature.
private final TuplePointer tempPtr = new TuplePointer();
private int[] probePSizeInTups;
+ private IOperatorStats stats = null;
public OptimizedHybridHashJoin(IHyracksJobletContext jobletCtx, int memSizeInFrames, int numOfPartitions,
String probeRelName, String buildRelName, RecordDescriptor probeRd, RecordDescriptor buildRd,
@@ -152,19 +157,23 @@
private void processTupleBuildPhase(int tid, int pid) throws HyracksDataException {
// insertTuple prevents the tuple to acquire a number of frames that is > the frame limit
while (!bufferManager.insertTuple(pid, accessorBuild, tid, tempPtr)) {
- int recordSize = VPartitionTupleBufferManager.calculateActualSize(null, accessorBuild.getTupleLength(tid));
- double numFrames = (double) recordSize / (double) jobletCtx.getInitialFrameSize();
+ int numFrames = bufferManager.framesNeeded(accessorBuild.getTupleLength(tid), 0);
int victimPartition;
- if (numFrames > bufferManager.getConstrain().frameLimit(pid)
- || (victimPartition = spillPolicy.selectVictimPartition(pid)) < 0) {
+ int partitionFrameLimit = bufferManager.getConstrain().frameLimit(pid);
+ if (numFrames > partitionFrameLimit || (victimPartition = spillPolicy.selectVictimPartition(pid)) < 0) {
// insert request can never be satisfied
- if (numFrames > memSizeInFrames || recordSize < jobletCtx.getInitialFrameSize()) {
- // the tuple is greater than the memory budget or although the record is small we could not find
- // a frame for it (possibly due to a bug)
+ if (numFrames > memSizeInFrames) {
+ // the tuple is greater than the memory budget
+ logTupleInsertionFailure(tid, pid, numFrames, partitionFrameLimit);
throw HyracksDataException.create(ErrorCode.INSUFFICIENT_MEMORY);
}
+ if (numFrames <= 1) {
+ // this shouldn't happen. whether the partition is spilled or not, it should be able to get 1 frame
+ logTupleInsertionFailure(tid, pid, numFrames, partitionFrameLimit);
+ throw new IllegalStateException("can't insert tuple in join memory");
+ }
// Record is large but insertion failed either 1) we could not satisfy the request because of the
- // frame limit or 2) we could not find a victim anymore (exhaused all victims) and the partition is
+ // frame limit or 2) we could not find a victim anymore (exhausted all victims) and the partition is
// memory-resident with no frame.
flushBigObjectToDisk(pid, accessorBuild, tid, buildRFWriters, buildRelName);
spilledStatus.set(pid);
@@ -183,7 +192,10 @@
private void spillPartition(int pid) throws HyracksDataException {
RunFileWriter writer = getSpillWriterOrCreateNewOneIfNotExist(buildRFWriters, buildRelName, pid);
- bufferManager.flushPartition(pid, writer);
+ int spilt = bufferManager.flushPartition(pid, writer);
+ if (stats != null) {
+ stats.getBytesWritten().update(spilt);
+ }
bufferManager.clearPartition(pid);
spilledStatus.set(pid);
}
@@ -261,8 +273,12 @@
for (int pid = spilledStatus.nextSetBit(0); pid >= 0 && pid < numOfPartitions; pid =
spilledStatus.nextSetBit(pid + 1)) {
if (bufferManager.getNumTuples(pid) > 0) {
- bufferManager.flushPartition(pid,
+ int spilt = bufferManager.flushPartition(pid,
getSpillWriterOrCreateNewOneIfNotExist(runFileWriters, refName, pid));
+ if (stats != null) {
+ stats.getBytesWritten().update(spilt);
+
+ }
bufferManager.clearPartition(pid);
}
}
@@ -417,6 +433,10 @@
reloadBuffer = new VSizeFrame(jobletCtx);
}
while (r.nextFrame(reloadBuffer)) {
+ if (stats != null) {
+ //TODO: be certain it is the case this is actually eagerly read
+ stats.getBytesRead().update(reloadBuffer.getBuffer().limit());
+ }
accessorBuild.reset(reloadBuffer.getBuffer());
for (int tid = 0; tid < accessorBuild.getTupleCount(); tid++) {
if (!bufferManager.insertTuple(pid, accessorBuild, tid, tempPtr)) {
@@ -520,7 +540,10 @@
if (victim >= 0 && bufferManager.getPhysicalSize(victim) >= recordSize) {
RunFileWriter runFileWriter =
getSpillWriterOrCreateNewOneIfNotExist(probeRFWriters, probeRelName, victim);
- bufferManager.flushPartition(victim, runFileWriter);
+ int spilt = bufferManager.flushPartition(victim, runFileWriter);
+ if (stats != null) {
+ stats.getBytesWritten().update(spilt);
+ }
bufferManager.clearPartition(victim);
if (!bufferManager.insertTuple(pid, accessorProbe, tupleId, tempPtr)) {
// This should not happen if the size calculations are correct, just not to let the query fail.
@@ -543,6 +566,9 @@
throw new HyracksDataException("The given tuple is too big");
}
+ if (stats != null) {
+ stats.getBytesWritten().update(bigFrameAppender.getBuffer().limit());
+ }
bigFrameAppender.write(runFileWriter, true);
}
@@ -613,4 +639,18 @@
}
this.isReversed = reversed;
}
+
+ private void logTupleInsertionFailure(int tid, int pid, int numFrames, int partitionFrameLimit) {
+ int recordSize = VPartitionTupleBufferManager.calculateActualSize(null, accessorBuild.getTupleLength(tid));
+ String details = String.format(
+ "partition %s, tuple size %s, needed # frames %s, partition frame limit %s, join "
+ + "memory in frames %s, initial frame size %s",
+ pid, recordSize, numFrames, partitionFrameLimit, memSizeInFrames, jobletCtx.getInitialFrameSize());
+ LOGGER.debug("can't insert tuple in join memory. {}", details);
+ LOGGER.debug("partitions status:\n{}", spillPolicy.partitionsStatus());
+ }
+
+ public void setOperatorStats(IOperatorStats stats) {
+ this.stats = stats;
+ }
}
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..e04eebe 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
@@ -45,6 +45,8 @@
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
+import org.apache.hyracks.api.job.profiling.NoOpOperatorStats;
import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
@@ -288,6 +290,7 @@
new FieldHashPartitionComputerFamily(buildKeys, buildHashFunctionFactories)
.createPartitioner(INIT_SEED);
boolean failed = false;
+ IOperatorStats stats = new NoOpOperatorStats();
@Override
public void open() throws HyracksDataException {
@@ -300,6 +303,7 @@
state.hybridHJ = new OptimizedHybridHashJoin(ctx.getJobletContext(), state.memForJoin,
state.numOfPartitions, PROBE_REL, BUILD_REL, probeRd, buildRd, probeHpc, buildHpc,
probePredEval, buildPredEval, isLeftOuter, nonMatchWriterFactories);
+ state.hybridHJ.setOperatorStats(stats);
state.hybridHJ.initBuild();
if (LOGGER.isTraceEnabled()) {
@@ -341,6 +345,11 @@
return "Hybrid Hash Join: Build";
}
+ @Override
+ public void setOperatorStats(IOperatorStats stats) {
+ this.stats = stats;
+ }
+
};
}
}
@@ -394,6 +403,7 @@
private FrameTupleAppender nullResultAppender = null;
private FrameTupleAccessor probeTupleAccessor;
private boolean failed = false;
+ IOperatorStats stats = null;
@Override
public void open() throws HyracksDataException {
@@ -402,6 +412,7 @@
writer.open();
state.hybridHJ.initProbe(probComp);
+ state.hybridHJ.setOperatorStats(stats);
if (LOGGER.isDebugEnabled()) {
LOGGER.debug("OptimizedHybridHashJoin is starting the probe phase.");
@@ -485,6 +496,11 @@
}
}
+ @Override
+ public void setOperatorStats(IOperatorStats stats) {
+ this.stats = stats;
+ }
+
private void logProbeComplete() {
if (LOGGER.isDebugEnabled()) {
LOGGER.debug("OptimizedHybridHashJoin closed its probe phase");
@@ -505,6 +521,9 @@
long buildPartSize = (long) Math.ceil((double) buildSideReader.getFileSize() / (double) frameSize);
long probePartSize = (long) Math.ceil((double) probeSideReader.getFileSize() / (double) frameSize);
int beforeMax = Math.max(buildSizeInTuple, probeSizeInTuple);
+ if (stats.getLevel().get() < level) {
+ stats.getLevel().set(level);
+ }
if (LOGGER.isDebugEnabled()) {
LOGGER.debug("\n>>>Joining Partition Pairs (thread_id " + Thread.currentThread().getId()
@@ -758,6 +777,7 @@
bReader.open();
rPartbuff.reset();
while (bReader.nextFrame(rPartbuff)) {
+ stats.getBytesRead().update(rPartbuff.getBuffer().limit());
// We need to allocate a copyBuffer, because this buffer gets added to the buffers list
// in the InMemoryHashJoin.
ByteBuffer copyBuffer = bufferManager.acquireFrame(rPartbuff.getFrameSize());
@@ -786,6 +806,9 @@
rPartbuff.reset();
try {
while (pReader.nextFrame(rPartbuff)) {
+ if (stats != null) {
+ stats.getBytesRead().update(rPartbuff.getBuffer().limit());
+ }
joiner.join(rPartbuff.getBuffer(), writer);
rPartbuff.reset();
}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/ExternalSortOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/ExternalSortOperatorDescriptor.java
index 654f3a3..ebe871b 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/ExternalSortOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/ExternalSortOperatorDescriptor.java
@@ -82,7 +82,8 @@
final boolean profile = ctx.getJobFlags().contains(JobFlag.PROFILE_RUNTIME);
IRunGenerator runGen = new ExternalSortRunGenerator(ctx, sortFields, keyNormalizerFactories,
comparatorFactories, outRecDescs[0], alg, policy, framesLimit, outputLimit);
- return profile ? TimedRunGenerator.time(runGen, ctx, "ExternalSort(Sort)") : runGen;
+ return profile ? ProfiledRunGenerator.time(runGen, ctx, "ExternalSort(Sort)", this.getActivityId())
+ : runGen;
}
};
}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/ProfiledRunGenerator.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/ProfiledRunGenerator.java
new file mode 100644
index 0000000..5cc1882
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/ProfiledRunGenerator.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.hyracks.dataflow.std.sort;
+
+import java.util.List;
+
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.ActivityId;
+import org.apache.hyracks.api.dataflow.ProfiledFrameWriter;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+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 org.apache.hyracks.dataflow.common.io.GeneratedRunFileReader;
+
+public class ProfiledRunGenerator extends ProfiledFrameWriter implements IRunGenerator {
+
+ private final IRunGenerator runGenerator;
+
+ private ProfiledRunGenerator(IRunGenerator runGenerator, IStatsCollector collector, String name,
+ IOperatorStats stats, ActivityId root) {
+ super(runGenerator, collector, name, stats, null);
+ this.runGenerator = runGenerator;
+ }
+
+ @Override
+ public List<GeneratedRunFileReader> getRuns() {
+ return runGenerator.getRuns();
+ }
+
+ @Override
+ public ISorter getSorter() {
+ return runGenerator.getSorter();
+ }
+
+ public static IRunGenerator time(IRunGenerator runGenerator, IHyracksTaskContext ctx, String name, ActivityId root)
+ throws HyracksDataException {
+ if (!(runGenerator instanceof ProfiledRunGenerator)) {
+ String statName = root.toString() + " - " + name;
+ IStatsCollector statsCollector = ctx.getStatsCollector();
+ IOperatorStats stats = new OperatorStats(statName);
+ statsCollector.add(stats);
+ return new ProfiledRunGenerator(runGenerator, ctx.getStatsCollector(), name, stats, root);
+ }
+ return runGenerator;
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/TimedRunGenerator.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/TimedRunGenerator.java
deleted file mode 100644
index b3a4aee..0000000
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/TimedRunGenerator.java
+++ /dev/null
@@ -1,51 +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.dataflow.std.sort;
-
-import java.util.List;
-
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.TimedFrameWriter;
-import org.apache.hyracks.api.job.profiling.IStatsCollector;
-import org.apache.hyracks.dataflow.common.io.GeneratedRunFileReader;
-
-public class TimedRunGenerator extends TimedFrameWriter implements IRunGenerator {
-
- private final IRunGenerator runGenerator;
-
- private TimedRunGenerator(IRunGenerator runGenerator, IStatsCollector collector, String name) {
- super(runGenerator, collector, name);
- this.runGenerator = runGenerator;
- }
-
- @Override
- public List<GeneratedRunFileReader> getRuns() {
- return runGenerator.getRuns();
- }
-
- @Override
- public ISorter getSorter() {
- return runGenerator.getSorter();
- }
-
- public static IRunGenerator time(IRunGenerator runGenerator, IHyracksTaskContext ctx, String name) {
- return runGenerator instanceof TimedRunGenerator ? runGenerator
- : new TimedRunGenerator(runGenerator, ctx.getStatsCollector(), name);
- }
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/TopKSorterOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/TopKSorterOperatorDescriptor.java
index b7ff530..2322910 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/TopKSorterOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/TopKSorterOperatorDescriptor.java
@@ -29,6 +29,7 @@
import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
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.JobFlag;
import org.apache.hyracks.dataflow.common.io.GeneratedRunFileReader;
@@ -65,8 +66,13 @@
final boolean profile = ctx.getJobFlags().contains(JobFlag.PROFILE_RUNTIME);
IRunGenerator runGen = new HybridTopKSortRunGenerator(ctx, framesLimit, topK, sortFields,
keyNormalizerFactories, comparatorFactories, outRecDescs[0]);
- return profile ? TimedRunGenerator.time(runGen, ctx, "TopKSort (Sort)") : runGen;
-
+ try {
+ return profile ? ProfiledRunGenerator.time(runGen, ctx, "TopKSort (Sort)", this.getActivityId())
+ : runGen;
+ } catch (HyracksDataException e) {
+ e.printStackTrace();
+ }
+ return null;
}
};
}
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 40802f8..470f413 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/pom.xml
@@ -111,7 +111,6 @@
<dependency>
<groupId>com.e-movimento.tinytools</groupId>
<artifactId>privilegedaccessor</artifactId>
- <version>1.2.2</version>
<scope>test</scope>
</dependency>
<dependency>
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-hdfs/pom.xml b/hyracks-fullstack/hyracks/hyracks-hdfs/pom.xml
index 678e94e..e5c1d06 100644
--- a/hyracks-fullstack/hyracks/hyracks-hdfs/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-hdfs/pom.xml
@@ -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,21 @@
<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>
+ <groupId>ch.qos.reload4j</groupId>
+ <artifactId>reload4j</artifactId>
+ </exclusion>
+ <!-- TODO(htowaileb): Remove after updating to hadoop 3.3.4 -->
+ <exclusion>
+ <groupId>org.eclipse.jetty</groupId>
+ <artifactId>jetty-util</artifactId>
</exclusion>
<exclusion>
- <groupId>org.slf4j</groupId>
- <artifactId>slf4j-log4j12</artifactId>
+ <groupId>org.eclipse.jetty</groupId>
+ <artifactId>jetty-util-ajax</artifactId>
</exclusion>
</exclusions>
</dependency>
@@ -174,16 +179,17 @@
<scope>test</scope>
<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>
+ <groupId>ch.qos.reload4j</groupId>
+ <artifactId>reload4j</artifactId>
</exclusion>
+ <!-- TODO(htowaileb): Remove after updating to hadoop 3.3.4 -->
<exclusion>
- <groupId>org.slf4j</groupId>
- <artifactId>slf4j-log4j12</artifactId>
+ <groupId>org.eclipse.jetty</groupId>
+ <artifactId>jetty-util</artifactId>
</exclusion>
</exclusions>
</dependency>
diff --git a/hyracks-fullstack/hyracks/hyracks-hdfs/src/main/java/org/apache/hyracks/hdfs/scheduler/Scheduler.java b/hyracks-fullstack/hyracks/hyracks-hdfs/src/main/java/org/apache/hyracks/hdfs/scheduler/Scheduler.java
index b9d68f7..f5bf07b 100644
--- a/hyracks-fullstack/hyracks/hyracks-hdfs/src/main/java/org/apache/hyracks/hdfs/scheduler/Scheduler.java
+++ b/hyracks-fullstack/hyracks/hyracks-hdfs/src/main/java/org/apache/hyracks/hdfs/scheduler/Scheduler.java
@@ -37,6 +37,7 @@
import org.apache.hyracks.api.client.NodeControllerInfo;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.exceptions.HyracksException;
+import org.apache.hyracks.api.network.ISocketChannelFactory;
import org.apache.hyracks.api.topology.ClusterTopology;
import org.apache.hyracks.hdfs.api.INcCollection;
import org.apache.hyracks.hdfs.api.INcCollectionBuilder;
@@ -56,16 +57,24 @@
private static final Logger LOGGER = LogManager.getLogger();
- /** a list of NCs */
+ /**
+ * a list of NCs
+ */
private String[] NCs;
- /** a map from ip to NCs */
+ /**
+ * a map from ip to NCs
+ */
private Map<String, List<String>> ipToNcMapping = new HashMap<>();
- /** a map from the NC name to the index */
+ /**
+ * a map from the NC name to the index
+ */
private Map<String, Integer> ncNameToIndex = new HashMap<>();
- /** a map from NC name to the NodeControllerInfo */
+ /**
+ * a map from NC name to the NodeControllerInfo
+ */
private Map<String, NodeControllerInfo> ncNameToNcInfos;
/**
@@ -76,13 +85,15 @@
/**
* The constructor of the scheduler.
*
- * @param ncNameToNcInfos
+ * @param ipAddress IP address
+ * @param port Port
+ * @param channelFactory Channel Factory
* @throws HyracksException
*/
- public Scheduler(String ipAddress, int port) throws HyracksException {
+ public Scheduler(String ipAddress, int port, ISocketChannelFactory channelFactory) throws HyracksException {
try {
- IHyracksClientConnection hcc = new HyracksConnection(ipAddress, port);
+ IHyracksClientConnection hcc = new HyracksConnection(ipAddress, port, channelFactory);
this.ncNameToNcInfos = hcc.getNodeControllerInfos();
ClusterTopology topology = hcc.getClusterTopology();
this.ncCollectionBuilder = topology == null ? new IPProximityNcCollectionBuilder()
@@ -96,23 +107,6 @@
/**
* The constructor of the scheduler.
*
- * @param ncNameToNcInfos
- * @throws HyracksException
- */
- public Scheduler(String ipAddress, int port, INcCollectionBuilder ncCollectionBuilder) throws HyracksException {
- try {
- IHyracksClientConnection hcc = new HyracksConnection(ipAddress, port);
- this.ncNameToNcInfos = hcc.getNodeControllerInfos();
- this.ncCollectionBuilder = ncCollectionBuilder;
- loadIPAddressToNCMap(ncNameToNcInfos);
- } catch (Exception e) {
- throw HyracksException.create(e);
- }
- }
-
- /**
- * The constructor of the scheduler.
- *
* @param ncNameToNcInfos the mapping from nc names to nc infos
* @throws HyracksException
*/
diff --git a/hyracks-fullstack/hyracks/hyracks-hdfs/src/main/java/org/apache/hyracks/hdfs2/scheduler/Scheduler.java b/hyracks-fullstack/hyracks/hyracks-hdfs/src/main/java/org/apache/hyracks/hdfs2/scheduler/Scheduler.java
index ddf140f..a26a5f7 100644
--- a/hyracks-fullstack/hyracks/hyracks-hdfs/src/main/java/org/apache/hyracks/hdfs2/scheduler/Scheduler.java
+++ b/hyracks-fullstack/hyracks/hyracks-hdfs/src/main/java/org/apache/hyracks/hdfs2/scheduler/Scheduler.java
@@ -26,6 +26,7 @@
import org.apache.hyracks.api.client.NodeControllerInfo;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.exceptions.HyracksException;
+import org.apache.hyracks.api.network.ISocketChannelFactory;
import org.apache.hyracks.api.topology.ClusterTopology;
import org.apache.hyracks.hdfs.api.INcCollectionBuilder;
@@ -44,15 +45,14 @@
* @param ncNameToNcInfos
* @throws HyracksException
*/
- public Scheduler(String ipAddress, int port) throws HyracksException {
- scheduler = new org.apache.hyracks.hdfs.scheduler.Scheduler(ipAddress, port);
+ public Scheduler(String ipAddress, int port, ISocketChannelFactory channelFactory) throws HyracksException {
+ scheduler = new org.apache.hyracks.hdfs.scheduler.Scheduler(ipAddress, port, channelFactory);
}
/**
* The constructor of the scheduler.
*
- * @param ncNameToNcInfos
- * the mapping from nc names to nc infos
+ * @param ncNameToNcInfos the mapping from nc names to nc infos
* @throws HyracksException
*/
public Scheduler(Map<String, NodeControllerInfo> ncNameToNcInfos) throws HyracksException {
@@ -62,10 +62,8 @@
/**
* The constructor of the scheduler.
*
- * @param ncNameToNcInfos
- * the mapping from nc names to nc infos
- * @param topology
- * the hyracks cluster toplogy
+ * @param ncNameToNcInfos the mapping from nc names to nc infos
+ * @param topology the hyracks cluster toplogy
* @throws HyracksException
*/
public Scheduler(Map<String, NodeControllerInfo> ncNameToNcInfos, ClusterTopology topology)
@@ -76,8 +74,7 @@
/**
* The constructor of the scheduler.
*
- * @param ncNameToNcInfos
- * the mapping from nc names to nc infos
+ * @param ncNameToNcInfos the mapping from nc names to nc infos
* @throws HyracksException
*/
public Scheduler(Map<String, NodeControllerInfo> ncNameToNcInfos, INcCollectionBuilder builder)
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/CLFLogger.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/CLFLogger.java
index 4732d71..1f5b441 100644
--- a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/CLFLogger.java
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/CLFLogger.java
@@ -43,7 +43,7 @@
//Based in part on LoggingHandler from Netty
public class CLFLogger extends ChannelDuplexHandler {
- private static final Logger accessLogger = LogManager.getLogger();
+ private static final Logger LOGGER = LogManager.getLogger();
private static final Level ACCESS_LOG_LEVEL = Level.forName("ACCESS", 550);
private static final DateTimeFormatter DATE_TIME_FORMATTER =
DateTimeFormatter.ofPattern("dd/MMM/yyyy:HH:mm:ss Z").withZone(ZoneId.systemDefault());
@@ -66,9 +66,14 @@
public void channelRead(ChannelHandlerContext ctx, Object msg) {
if (msg instanceof HttpRequest) {
HttpRequest req = (HttpRequest) msg;
- clientIp = ((NioSocketChannel) ctx.channel()).remoteAddress().getAddress().toString().substring(1);
+ try {
+ clientIp = ((NioSocketChannel) ctx.channel()).remoteAddress().getAddress().toString().substring(1);
+ } catch (Exception e) {
+ LOGGER.debug("ignoring {} obtaining client ip for {}", e, ctx.channel());
+ clientIp = "-";
+ }
requestTime = Instant.now();
- reqLine = req.method().toString() + " " + req.getUri() + " " + req.getProtocolVersion().toString();
+ reqLine = req.method().toString() + " " + req.uri() + " " + req.protocolVersion();
userAgentRef = headerValueOrDash("Referer", req) + " " + headerValueOrDash("User-Agent", req);
lastChunk = false;
}
@@ -116,7 +121,7 @@
}
private void printAndPrepare() {
- if (!accessLogger.isEnabled(ACCESS_LOG_LEVEL)) {
+ if (!LOGGER.isEnabled(ACCESS_LOG_LEVEL)) {
return;
}
logLineBuilder.append(clientIp);
@@ -131,7 +136,7 @@
logLineBuilder.append(" ").append(statusCode);
logLineBuilder.append(" ").append(respSize);
logLineBuilder.append(" ").append(userAgentRef);
- accessLogger.log(ACCESS_LOG_LEVEL, logLineBuilder);
+ LOGGER.log(ACCESS_LOG_LEVEL, logLineBuilder);
respSize = 0;
logLineBuilder.setLength(0);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/HyracksClientInterfaceRemoteProxy.java b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/HyracksClientInterfaceRemoteProxy.java
index 82d7a3b..3b59a70 100644
--- a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/HyracksClientInterfaceRemoteProxy.java
+++ b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/HyracksClientInterfaceRemoteProxy.java
@@ -35,6 +35,7 @@
import org.apache.hyracks.api.job.JobId;
import org.apache.hyracks.api.job.JobInfo;
import org.apache.hyracks.api.job.JobStatus;
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
import org.apache.hyracks.api.topology.ClusterTopology;
import org.apache.hyracks.ipc.api.IIPCHandle;
import org.apache.hyracks.ipc.api.RPCInterface;
@@ -130,6 +131,13 @@
}
@Override
+ public List<IOperatorStats> waitForCompletion(JobId jobId, List<String> statOperatorNames) throws Exception {
+ HyracksClientInterfaceFunctions.WaitForCompletionFunction wfcf =
+ new HyracksClientInterfaceFunctions.WaitForCompletionFunction(jobId, statOperatorNames);
+ return (List<IOperatorStats>) rpci.call(ipcHandle, wfcf);
+ }
+
+ @Override
public Map<String, NodeControllerInfo> getNodeControllersInfo() throws Exception {
HyracksClientInterfaceFunctions.GetNodeControllersInfoFunction gncif =
new HyracksClientInterfaceFunctions.GetNodeControllersInfoFunction();
diff --git a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/HyracksConnection.java b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/HyracksConnection.java
index 9351348..1b6d8af 100644
--- a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/HyracksConnection.java
+++ b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/HyracksConnection.java
@@ -18,6 +18,7 @@
*/
package org.apache.hyracks.ipc.impl;
+import java.io.Closeable;
import java.io.File;
import java.net.InetSocketAddress;
import java.net.URL;
@@ -52,6 +53,7 @@
import org.apache.hyracks.api.job.JobInfo;
import org.apache.hyracks.api.job.JobSpecification;
import org.apache.hyracks.api.job.JobStatus;
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
import org.apache.hyracks.api.network.ISocketChannelFactory;
import org.apache.hyracks.api.topology.ClusterTopology;
import org.apache.hyracks.api.util.InvokeUtil;
@@ -70,7 +72,7 @@
*
* @author vinayakb
*/
-public final class HyracksConnection implements IHyracksClientConnection {
+public final class HyracksConnection implements Closeable, IHyracksClientConnection {
private static final Logger LOGGER = LogManager.getLogger();
@@ -123,6 +125,11 @@
}
@Override
+ public void close() {
+ ipc.stop();
+ }
+
+ @Override
public JobStatus getJobStatus(JobId jobId) throws Exception {
return hci.getJobStatus(jobId);
}
@@ -196,6 +203,17 @@
}
@Override
+ public List<IOperatorStats> waitForCompletion(JobId jobId, List<String> statOperatorNames) throws Exception {
+ try {
+ return hci.waitForCompletion(jobId, statOperatorNames);
+ } catch (InterruptedException e) {
+ // Cancels an on-going job if the current thread gets interrupted.
+ cancelJob(jobId);
+ throw e;
+ }
+ }
+
+ @Override
public Map<String, NodeControllerInfo> getNodeControllerInfos() throws HyracksException {
try {
return hci.getNodeControllersInfo();
diff --git a/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/DownloadLicensesMojo.java b/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/DownloadLicensesMojo.java
index 293c08c..c8980cd 100644
--- a/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/DownloadLicensesMojo.java
+++ b/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/DownloadLicensesMojo.java
@@ -63,7 +63,7 @@
String fileName = entry.getLicense().getContentFile(false);
doDownload(timeoutMillis, i, url, fileName);
});
- } catch (ProjectBuildingException e) {
+ } catch (ProjectBuildingException | IOException e) {
throw new MojoExecutionException("Unexpected exception: " + e, e);
}
}
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 347c19a..c548179 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
@@ -20,11 +20,13 @@
import static org.apache.hyracks.maven.license.GenerateFileMojo.EmbeddedArtifact.LICENSE;
import static org.apache.hyracks.maven.license.GenerateFileMojo.EmbeddedArtifact.NOTICE;
+import static org.apache.hyracks.maven.license.LicenseUtil.toGav;
import static org.apache.hyracks.maven.license.ProjectFlag.ALTERNATE_LICENSE_FILE;
import static org.apache.hyracks.maven.license.ProjectFlag.ALTERNATE_NOTICE_FILE;
import static org.apache.hyracks.maven.license.ProjectFlag.IGNORE_MISSING_EMBEDDED_LICENSE;
import static org.apache.hyracks.maven.license.ProjectFlag.IGNORE_MISSING_EMBEDDED_NOTICE;
import static org.apache.hyracks.maven.license.ProjectFlag.IGNORE_NOTICE_OVERRIDE;
+import static org.apache.hyracks.maven.license.ProjectFlag.IGNORE_SHADOWED_DEPENDENCIES;
import static org.apache.hyracks.maven.license.ProjectFlag.ON_MULTIPLE_EMBEDDED_LICENSE;
import static org.apache.hyracks.maven.license.ProjectFlag.ON_MULTIPLE_EMBEDDED_NOTICE;
@@ -36,6 +38,7 @@
import java.io.Writer;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
+import java.util.Collections;
import java.util.Comparator;
import java.util.Enumeration;
import java.util.HashMap;
@@ -43,6 +46,7 @@
import java.util.Iterator;
import java.util.List;
import java.util.Map;
+import java.util.Properties;
import java.util.Set;
import java.util.SortedMap;
import java.util.SortedSet;
@@ -54,17 +58,26 @@
import java.util.jar.JarEntry;
import java.util.jar.JarFile;
import java.util.regex.Pattern;
+import java.util.stream.Collectors;
import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.tuple.Pair;
import org.apache.hyracks.maven.license.freemarker.IndentDirective;
import org.apache.hyracks.maven.license.freemarker.LoadFileDirective;
import org.apache.hyracks.maven.license.project.LicensedProjects;
import org.apache.hyracks.maven.license.project.Project;
+import org.apache.maven.artifact.Artifact;
+import org.apache.maven.artifact.DefaultArtifact;
+import org.apache.maven.artifact.handler.ArtifactHandler;
+import org.apache.maven.artifact.handler.DefaultArtifactHandler;
import org.apache.maven.plugin.MojoExecutionException;
import org.apache.maven.plugin.MojoFailureException;
import org.apache.maven.plugins.annotations.Mojo;
import org.apache.maven.plugins.annotations.Parameter;
import org.apache.maven.plugins.annotations.ResolutionScope;
+import org.apache.maven.project.MavenProject;
import org.apache.maven.project.ProjectBuildingException;
import com.fasterxml.jackson.databind.ObjectMapper;
@@ -110,6 +123,12 @@
@Parameter
private boolean stripFoundationAssertionFromNotices = false;
+ @Parameter
+ private boolean includeShadowedDependencies = true;
+
+ @Parameter
+ private boolean validateShadowLicenses = false;
+
private SortedMap<String, SortedSet<Project>> noticeMap;
@java.lang.Override
@@ -143,11 +162,12 @@
}
licenseSpecs.addAll(urlToLicenseMap.values());
for (LicenseSpec license : licenseSpecs) {
- resolveArtifactContent(license, true);
+ resolveArtifactContent(license, true, false);
}
}
- private String resolveArtifactContent(ArtifactSpec artifact, boolean bestEffort) throws IOException {
+ private String resolveArtifactContent(ArtifactSpec artifact, boolean bestEffort, boolean suppressWarning)
+ throws IOException {
if (artifact.getContent() == null) {
getLog().debug("Resolving content for " + artifact.getUrl() + " (" + artifact.getContentFile() + ")");
File cFile = new File(artifact.getContentFile());
@@ -161,7 +181,9 @@
}
if (!cFile.exists()) {
if (!bestEffort) {
- getLog().warn("MISSING: content file (" + cFile + ") for url: " + artifact.getUrl());
+ if (!suppressWarning) {
+ getLog().warn("MISSING: content file (" + cFile + ") for url: " + artifact.getUrl());
+ }
artifact.setContent("MISSING: " + artifact.getContentFile() + " (" + artifact.getUrl() + ")");
}
} else {
@@ -268,10 +290,12 @@
for (Project p : lps.getProjects()) {
String licenseText = p.getLicenseText();
if (licenseText == null) {
- warnUnlessFlag(p.gav(), IGNORE_MISSING_EMBEDDED_LICENSE,
- "Using license other than from within artifact: " + p.gav() + " (" + lps.getLicense()
- + ")");
- licenseText = resolveArtifactContent(lps.getLicense(), false);
+ if (validateProjectLicense(p)) {
+ warnUnlessFlag(p.gav(), IGNORE_MISSING_EMBEDDED_LICENSE,
+ "Using license other than from within artifact: " + p.gav() + " (" + lps.getLicense()
+ + ")");
+ }
+ licenseText = resolveArtifactContent(lps.getLicense(), false, !validateProjectLicense(p));
}
LicenseSpec spec = lps.getLicense();
if (spec.getDisplayName() == null) {
@@ -297,6 +321,10 @@
licenseMap = licenseMap2;
}
+ private boolean validateProjectLicense(Project p) {
+ return !p.isShadowed() || validateShadowLicenses;
+ }
+
private Set<Project> getProjects() {
Set<Project> projects = new HashSet<>();
licenseMap.values().forEach(p -> projects.addAll(p.getProjects()));
@@ -309,9 +337,11 @@
String noticeText = p.getNoticeText();
if (noticeText == null && noticeOverrides.containsKey(p.gav())) {
String noticeUrl = noticeOverrides.get(p.gav());
- warnUnlessFlag(p.gav(), IGNORE_NOTICE_OVERRIDE,
- "Using notice other than from within artifact: " + p.gav() + " (" + noticeUrl + ")");
- p.setNoticeText(resolveArtifactContent(new NoticeSpec(noticeUrl), false));
+ if (validateProjectLicense(p)) {
+ warnUnlessFlag(p.gav(), IGNORE_NOTICE_OVERRIDE,
+ "Using notice other than from within artifact: " + p.gav() + " (" + noticeUrl + ")");
+ }
+ p.setNoticeText(resolveArtifactContent(new NoticeSpec(noticeUrl), false, p.isShadowed()));
} else if (noticeText == null && !noticeOverrides.containsKey(p.gav())
&& Boolean.TRUE.equals(getProjectFlag(p.gav(), IGNORE_NOTICE_OVERRIDE))) {
getLog().warn(p + " has IGNORE_NOTICE_OVERRIDE flag set, but no override defined...");
@@ -370,7 +400,7 @@
// this seems way too liberal
filter = entry -> entry.getName().matches("(.*/|^)" + "NOTICE" + "(.(txt|md))?");
consumer = Project::setNoticeText;
- contentTransformer = UnaryOperator.identity();
+ contentTransformer = getNoticeFileContentTransformer();
break;
case LICENSE:
name = "LICENSE";
@@ -381,8 +411,7 @@
// this seems way too liberal
filter = entry -> entry.getName().matches("(.*/|^)" + "LICENSE" + "(.(txt|md))?");
consumer = Project::setLicenseText;
- contentTransformer = stripFoundationAssertionFromNotices
- ? text -> FOUNDATION_PATTERN.matcher(text).replaceAll("") : UnaryOperator.identity();
+ contentTransformer = UnaryOperator.identity();
break;
default:
throw new IllegalStateException("NYI: " + artifact);
@@ -394,6 +423,9 @@
} else if (!artifactFile.getName().endsWith(".jar")) {
getLog().info("Skipping unknown artifact file type: " + artifactFile);
continue;
+ } else if (!validateShadowLicenses && p.isShadowed()) {
+ getLog().info("Skipping shadowed project: " + p.gav());
+ continue;
}
String alternateFilename = (String) getProjectFlag(p.gav(), alternateFilenameFlag);
Predicate<JarEntry> finalFilter =
@@ -472,4 +504,92 @@
}
return matches;
}
+
+ private UnaryOperator<String> getNoticeFileContentTransformer() {
+ UnaryOperator<String> transformer;
+ if (stripFoundationAssertionFromNotices) {
+ transformer = text -> FOUNDATION_PATTERN.matcher(text).replaceAll("");
+ } else {
+ transformer = UnaryOperator.identity();
+ }
+ return transformer;
+ }
+
+ @java.lang.Override
+ protected void gatherProjectDependencies(MavenProject project,
+ Map<MavenProject, List<Pair<String, String>>> dependencyLicenseMap,
+ Map<String, MavenProject> dependencyGavMap) throws ProjectBuildingException, MojoExecutionException {
+ super.gatherProjectDependencies(project, dependencyLicenseMap, dependencyGavMap);
+ gatherShadowedDependencies(dependencyLicenseMap, dependencyGavMap);
+ }
+
+ @java.lang.Override
+ protected void processExtraDependencies(Map<MavenProject, List<Pair<String, String>>> dependencyLicenseMap,
+ Map<String, MavenProject> dependencyGavMap) throws ProjectBuildingException, MojoExecutionException {
+ super.processExtraDependencies(dependencyLicenseMap, dependencyGavMap);
+ gatherShadowedDependencies(dependencyLicenseMap, dependencyGavMap);
+ }
+
+ 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));
+ if (!includeShadowedDependencies) {
+ getLog().info("Not gathering shadowed dependencies as 'includeShadowedDependencies' is set to "
+ + includeShadowedDependencies);
+ return;
+ }
+ projects.addAll(dependencyLicenseMap.keySet());
+ for (MavenProject p : projects) {
+ File artifactFile = p.getArtifact().getFile();
+ if (!artifactFile.exists()) {
+ throw new MojoExecutionException("Artifact file " + artifactFile + " does not exist!");
+ } else if (!artifactFile.getName().endsWith(".jar")) {
+ getLog().info("Skipping unknown artifact file type: " + artifactFile);
+ continue;
+ }
+ @SuppressWarnings("unchecked")
+ List<String[]> specs = (List<String[]>) getProjectFlags()
+ .getOrDefault(Pair.of(toGav(p), IGNORE_SHADOWED_DEPENDENCIES), Collections.emptyList());
+ getLog().debug(p + " has " + IGNORE_SHADOWED_DEPENDENCIES.propName() + " set to "
+ + specs.stream().map(ArrayUtils::toString).collect(Collectors.joining(",")));
+ try (JarFile jarFile = new JarFile(artifactFile)) {
+ SortedMap<String, JarEntry> matches = gatherMatchingEntries(jarFile,
+ entry -> entry.getName().matches("(.*/|^)" + "pom\\.properties"));
+ if (!matches.isEmpty()) {
+ jarEntryLoop: for (JarEntry entry : matches.values()) {
+ Properties props = new Properties();
+ props.load(jarFile.getInputStream(entry));
+ String groupId = props.getProperty("groupId");
+ String artifactId = props.getProperty("artifactId");
+ String version = props.getProperty("version");
+ String gav = groupId + ":" + artifactId + ":" + version;
+ if (!dependencyGavMap.containsKey(gav)) {
+ for (String[] ignoreSpec : specs) {
+ if ((ignoreSpec[0].equals(groupId) || ignoreSpec[0].equals("*"))
+ && (ignoreSpec[1].equals(artifactId) || ignoreSpec[1].equals("*"))
+ && (ignoreSpec[2].equals(version) || ignoreSpec[2].equals("*"))) {
+ getLog().info("skipping " + gav + " (shadowed from " + p.getId()
+ + "), as it matches " + IGNORE_SHADOWED_DEPENDENCIES.propName());
+ continue jarEntryLoop;
+ }
+ }
+ getLog().info("adding " + gav + " (shadowed from " + p.getId() + ")");
+ ArtifactHandler handler = new DefaultArtifactHandler("jar");
+ String[] gavParts = StringUtils.split(gav, ':');
+ Artifact manualDep = new DefaultArtifact(gavParts[0], gavParts[1], gavParts[2],
+ Artifact.SCOPE_COMPILE, "jar", null, handler);
+ processArtifact(manualDep, dependencyLicenseMap, dependencyGavMap, true);
+ }
+ }
+ }
+ } catch (IOException e) {
+ throw new MojoExecutionException(e);
+ }
+ }
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/LicenseMojo.java b/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/LicenseMojo.java
index 2206621..3b5fde6 100644
--- a/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/LicenseMojo.java
+++ b/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/LicenseMojo.java
@@ -70,6 +70,7 @@
public abstract class LicenseMojo extends AbstractMojo {
+ private static final String SHADOWED_KEY = LicenseMojo.class.getName() + "_shadowed";
@Parameter
protected List<Override> overrides = new ArrayList<>();
@@ -249,7 +250,7 @@
});
}
- protected void addDependenciesToLicenseMap() throws ProjectBuildingException {
+ protected void addDependenciesToLicenseMap() throws ProjectBuildingException, MojoExecutionException, IOException {
Map<MavenProject, List<Pair<String, String>>> dependencyLicenseMap = gatherDependencies();
dependencyLicenseMap.forEach((depProject, value) -> {
Set<String> locations = dependencySets.isEmpty() ? Collections.singleton(location)
@@ -310,7 +311,9 @@
licenseUrl = fakeLicenseUrl;
}
}
- addProject(new Project(depProject, depLocation, depProject.getArtifact().getFile()),
+ addProject(
+ new Project(depProject, depLocation, depProject.getArtifact().getFile(),
+ Boolean.parseBoolean(String.valueOf(depProject.getContextValue(SHADOWED_KEY)))),
new LicenseSpec(licenseUrl, displayName), true);
}
@@ -363,11 +366,13 @@
}
}
- protected Map<MavenProject, List<Pair<String, String>>> gatherDependencies() throws ProjectBuildingException {
+ protected Map<MavenProject, List<Pair<String, String>>> gatherDependencies()
+ throws ProjectBuildingException, MojoExecutionException, IOException {
Map<MavenProject, List<Pair<String, String>>> dependencyLicenseMap = new HashMap<>();
Map<String, MavenProject> dependencyGavMap = new HashMap<>();
gatherProjectDependencies(project, dependencyLicenseMap, dependencyGavMap);
+ processExtraDependencies(dependencyLicenseMap, dependencyGavMap);
for (Override override : overrides) {
// Collect both <gav></gav> and <gavs><gav></gav><gav></gav>...</gavs>
@@ -399,29 +404,44 @@
return dependencyLicenseMap;
}
- private void gatherProjectDependencies(MavenProject project,
+ protected void gatherProjectDependencies(MavenProject project,
Map<MavenProject, List<Pair<String, String>>> dependencyLicenseMap,
- Map<String, MavenProject> dependencyGavMap) throws ProjectBuildingException {
+ Map<String, MavenProject> dependencyGavMap) throws ProjectBuildingException, MojoExecutionException {
+ getLog().debug("+gatherProjectDependencies " + toGav(project));
final Set<Artifact> dependencyArtifacts = project.getArtifacts();
if (dependencyArtifacts != null) {
for (Artifact depArtifact : dependencyArtifacts) {
- processArtifact(depArtifact, dependencyLicenseMap, dependencyGavMap);
+ processArtifact(depArtifact, dependencyLicenseMap, dependencyGavMap, false);
}
}
- for (String gav : extraDependencies) {
- ArtifactHandler handler = new DefaultArtifactHandler("jar");
- String[] gavParts = StringUtils.split(gav, ':');
- Artifact manualDep = new DefaultArtifact(gavParts[0], gavParts[1], gavParts[2], Artifact.SCOPE_COMPILE,
- "jar", null, handler);
- processArtifact(manualDep, dependencyLicenseMap, dependencyGavMap);
- }
}
- private void processArtifact(Artifact depArtifact,
+ protected void processExtraDependencies(Map<MavenProject, List<Pair<String, String>>> dependencyLicenseMap,
+ Map<String, MavenProject> dependencyGavMap) throws ProjectBuildingException, MojoExecutionException {
+ for (String extraDependency : extraDependencies) {
+ String[] gavParts = StringUtils.split(extraDependency, ':');
+ String gav = gavParts[0] + ":" + gavParts[1] + ":" + gavParts[2];
+ if (!dependencyGavMap.containsKey(gav)) {
+ ArtifactHandler handler = new DefaultArtifactHandler("jar");
+ Artifact manualDep = new DefaultArtifact(gavParts[0], gavParts[1], gavParts[2], Artifact.SCOPE_COMPILE,
+ "jar", null, handler);
+ boolean shadowed = false;
+ if (gavParts.length > 3) {
+ shadowed = "shadowed".equals(gavParts[3]);
+ }
+ processArtifact(manualDep, dependencyLicenseMap, dependencyGavMap, shadowed);
+ } else {
+ getLog().warn("not adding extra dependency " + gav + " as it is already added");
+ }
+ }
+ }
+
+ protected void processArtifact(Artifact depArtifact,
Map<MavenProject, List<Pair<String, String>>> dependencyLicenseMap,
- Map<String, MavenProject> dependencyGavMap) throws ProjectBuildingException {
+ Map<String, MavenProject> dependencyGavMap, boolean shadowed) throws ProjectBuildingException {
if (!excludedScopes.contains(depArtifact.getScope())) {
MavenProject dep = resolveDependency(depArtifact);
+ getLog().debug("+processArtifact " + toGav(dep));
if (!depArtifact.isResolved()) {
ArtifactResolutionRequest arr = new ArtifactResolutionRequest();
arr.setLocalRepository(localRepository);
@@ -442,6 +462,7 @@
: (license1.getName() != null ? license1.getName() : "LICENSE_EMPTY_NAME_URL");
licenseUrls.add(new ImmutablePair<>(url, license1.getName()));
}
+ dep.setContextValue(SHADOWED_KEY, shadowed);
dependencyLicenseMap.put(dep, licenseUrls);
}
}
@@ -457,6 +478,10 @@
} catch (ProjectBuildingException e) {
throw new ProjectBuildingException(key, "Error creating dependent artifacts", e);
}
+ // override the gav in the built dependency with the gavs in depObj
+ depProj.setGroupId(depObj.getGroupId());
+ depProj.setArtifactId(depObj.getArtifactId());
+ depProj.setVersion(depObj.getVersion());
Model supplement = supplementModels
.get(SupplementalModelHelper.generateSupplementMapKey(depObj.getGroupId(), depObj.getArtifactId()));
diff --git a/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/ProjectFlag.java b/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/ProjectFlag.java
index d61dde1..eb46041 100644
--- a/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/ProjectFlag.java
+++ b/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/ProjectFlag.java
@@ -20,9 +20,12 @@
import static org.apache.hyracks.maven.license.LicenseUtil.toGav;
+import java.util.ArrayList;
import java.util.Arrays;
+import java.util.List;
import java.util.Properties;
+import org.apache.commons.lang3.ArrayUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.commons.lang3.tuple.Pair;
import org.apache.hyracks.util.StringUtil;
@@ -36,13 +39,15 @@
ON_MULTIPLE_EMBEDDED_LICENSE,
ON_MULTIPLE_EMBEDDED_NOTICE,
ALTERNATE_LICENSE_FILE,
- ALTERNATE_NOTICE_FILE;
+ ALTERNATE_NOTICE_FILE,
+ IGNORE_SHADOWED_DEPENDENCIES;
String propName() {
return "license." + StringUtil.toCamelCase(name());
}
void visit(MavenProject depObj, Properties properties, LicenseMojo licenseMojo) {
+ licenseMojo.getLog().debug("+" + propName() + ".visit: " + toGav(depObj));
String value = properties.getProperty(propName());
if (value == null) {
return;
@@ -59,12 +64,34 @@
+ " for " + toGav(depObj));
}
break;
+ case IGNORE_SHADOWED_DEPENDENCIES:
+ // <license.ignoreShadowedDependencies>*:com.couchbase.client:core-io:*</license.ignoreShadowedDependencies>
+ List<String[]> specsList = new ArrayList<>();
+ for (String spec : StringUtils.split(value, ",")) {
+ boolean found = false;
+ String[] specSplit = StringUtils.split(spec, ":");
+ if (specSplit.length != 4) {
+ throw new IllegalArgumentException(spec);
+ }
+ if (specSplit[0].equals(depObj.getVersion()) || specSplit[0].equals("*")) {
+ specsList.add(ArrayUtils.subarray(specSplit, 1, specSplit.length));
+ found = true;
+ }
+ if (!found) {
+ licenseMojo.getLog().info(propName() + " defined on versions that *do not* match: "
+ + specSplit[0] + " for " + toGav(depObj));
+ }
+ }
+ if (!specsList.isEmpty()) {
+ licenseMojo.getProjectFlags().put(Pair.of(toGav(depObj), this), specsList);
+ }
+ break;
case ALTERNATE_LICENSE_FILE:
case ALTERNATE_NOTICE_FILE:
case ON_MULTIPLE_EMBEDDED_NOTICE:
case ON_MULTIPLE_EMBEDDED_LICENSE:
- boolean found = false;
for (String spec : StringUtils.split(value, ",")) {
+ boolean found = false;
String[] specSplit = StringUtils.split(spec, ":");
if (specSplit.length != 2) {
throw new IllegalArgumentException(spec);
@@ -73,10 +100,10 @@
licenseMojo.getProjectFlags().put(Pair.of(toGav(depObj), this), specSplit[1]);
found = true;
}
- }
- if (!found) {
- licenseMojo.getLog().info(propName() + " defined on versions that *do not* match: " + value
- + " for " + toGav(depObj));
+ if (!found) {
+ licenseMojo.getLog().info(propName() + " defined on versions that *do not* match: " + value
+ + " for " + toGav(depObj));
+ }
}
break;
default:
diff --git a/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/project/LicensedProjects.java b/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/project/LicensedProjects.java
index f6f9f32..31b4a86 100644
--- a/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/project/LicensedProjects.java
+++ b/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/project/LicensedProjects.java
@@ -21,6 +21,7 @@
import java.util.Set;
import java.util.SortedSet;
import java.util.TreeSet;
+import java.util.function.Predicate;
import org.apache.hyracks.maven.license.LicenseSpec;
@@ -51,6 +52,11 @@
return projects;
}
+ public SortedSet<Project> getNonShadowedProjects() {
+ return projects.stream().filter(Predicate.not(Project::isShadowed))
+ .collect(() -> new TreeSet<>(Project.PROJECT_COMPARATOR), TreeSet::add, TreeSet::addAll);
+ }
+
public void addProject(Project project) {
projects.add(project);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/project/Project.java b/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/project/Project.java
index e44914d..a755c05 100644
--- a/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/project/Project.java
+++ b/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/project/Project.java
@@ -41,6 +41,7 @@
private String licenseText;
private String sourcePointer;
private String classifier;
+ private final boolean shadowed;
@JsonIgnore
private MavenProject mavenProject;
@@ -48,7 +49,7 @@
public static final Comparator<Project> PROJECT_COMPARATOR =
(o1, o2) -> o1.compareToken().compareTo(o2.compareToken());
- public Project(MavenProject project, String location, File artifactPath) {
+ public Project(MavenProject project, String location, File artifactPath, boolean shadowed) {
mavenProject = project;
name = project.getName();
groupId = project.getGroupId();
@@ -57,6 +58,7 @@
url = project.getUrl();
classifier = project.getArtifact().getClassifier();
this.artifactPath = artifactPath.getPath();
+ this.shadowed = shadowed;
setLocation(location);
}
@@ -65,7 +67,8 @@
@JsonProperty("artifactId") String artifactId, @JsonProperty("url") String url,
@JsonProperty("version") String version, @JsonProperty("location") String location,
@JsonProperty("artifactPath") String artifactPath, @JsonProperty("noticeText") String noticeText,
- @JsonProperty("licenseText") String licenseText, @JsonProperty("classifier") String classifier) {
+ @JsonProperty("licenseText") String licenseText, @JsonProperty("classifier") String classifier,
+ @JsonProperty("shadowed") boolean shadowed) {
this.name = name;
this.groupId = groupId;
this.artifactId = artifactId;
@@ -76,6 +79,7 @@
this.noticeText = noticeText;
this.licenseText = licenseText;
this.classifier = classifier;
+ this.shadowed = shadowed;
}
public String getName() {
@@ -185,6 +189,10 @@
this.sourcePointer = sourcePointer;
}
+ public boolean isShadowed() {
+ return shadowed;
+ }
+
@Override
public String toString() {
return "Project [" + gav() + "]";
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/api/IDiskBTreeStatefulPointSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/api/IDiskBTreeStatefulPointSearchCursor.java
new file mode 100644
index 0000000..f436636
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/api/IDiskBTreeStatefulPointSearchCursor.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.btree.api;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.btree.impls.DiskBTreePointSearchCursor;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
+import org.apache.hyracks.storage.common.ISearchPredicate;
+
+/**
+ * Allows stateful {@link DiskBTreePointSearchCursor} to resume the search without closing and reopening the cursor
+ * Note: upon closing a cursor, ensure the search state is cleared
+ */
+public interface IDiskBTreeStatefulPointSearchCursor {
+ int getLastPageId();
+
+ void setCursorToNextKey(ISearchPredicate searchPred) throws HyracksDataException;
+
+ ITreeIndexFrame getFrame();
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
index 2455625..0ab88a5 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
@@ -29,10 +29,12 @@
import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import org.apache.hyracks.storage.am.common.impls.DefaultTupleProjectorFactory;
+import org.apache.hyracks.storage.common.projection.ITupleProjectorFactory;
public class BTreeSearchOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
- private static final long serialVersionUID = 1L;
+ private static final long serialVersionUID = 2L;
protected final int[] lowKeyFields;
protected final int[] highKeyFields;
@@ -52,6 +54,7 @@
protected byte[] searchCallbackProceedResultTrueValue;
protected final ITupleFilterFactory tupleFilterFactory;
protected final long outputLimit;
+ protected final ITupleProjectorFactory tupleProjectorFactory;
public BTreeSearchOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor outRecDesc,
int[] lowKeyFields, int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
@@ -61,7 +64,8 @@
IMissingWriterFactory nonFilterWriterFactory) {
this(spec, outRecDesc, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory,
retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, minFilterFieldIndexes,
- maxFilterFieldIndexes, appendIndexFilter, nonFilterWriterFactory, null, -1, false, null, null);
+ maxFilterFieldIndexes, appendIndexFilter, nonFilterWriterFactory, null, -1, false, null, null,
+ DefaultTupleProjectorFactory.INSTANCE);
}
public BTreeSearchOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor outRecDesc,
@@ -71,7 +75,7 @@
int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, boolean appendIndexFilter,
IMissingWriterFactory nonFilterWriterFactory, ITupleFilterFactory tupleFilterFactory, long outputLimit,
boolean appendOpCallbackProceedResult, byte[] searchCallbackProceedResultFalseValue,
- byte[] searchCallbackProceedResultTrueValue) {
+ byte[] searchCallbackProceedResultTrueValue, ITupleProjectorFactory tupleProjectorFactory) {
super(spec, 1, 1);
this.indexHelperFactory = indexHelperFactory;
this.retainInput = retainInput;
@@ -92,6 +96,7 @@
this.appendOpCallbackProceedResult = appendOpCallbackProceedResult;
this.searchCallbackProceedResultFalseValue = searchCallbackProceedResultFalseValue;
this.searchCallbackProceedResultTrueValue = searchCallbackProceedResultTrueValue;
+ this.tupleProjectorFactory = tupleProjectorFactory;
}
@Override
@@ -102,7 +107,7 @@
lowKeyInclusive, highKeyInclusive, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter,
nonFilterWriterFactory, tupleFilterFactory, outputLimit, appendOpCallbackProceedResult,
- searchCallbackProceedResultFalseValue, searchCallbackProceedResultTrueValue);
+ searchCallbackProceedResultFalseValue, searchCallbackProceedResultTrueValue, tupleProjectorFactory);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
index fb331bf..a56e305 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
@@ -22,6 +22,7 @@
import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.util.HyracksConstants;
import org.apache.hyracks.dataflow.common.data.accessors.PermutingFrameTupleReference;
import org.apache.hyracks.storage.am.btree.impls.RangePredicate;
import org.apache.hyracks.storage.am.btree.util.BTreeUtils;
@@ -30,9 +31,11 @@
import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
import org.apache.hyracks.storage.am.common.dataflow.IndexSearchOperatorNodePushable;
+import org.apache.hyracks.storage.am.common.impls.DefaultTupleProjectorFactory;
import org.apache.hyracks.storage.common.IIndexAccessParameters;
import org.apache.hyracks.storage.common.ISearchPredicate;
import org.apache.hyracks.storage.common.MultiComparator;
+import org.apache.hyracks.storage.common.projection.ITupleProjectorFactory;
public class BTreeSearchOperatorNodePushable extends IndexSearchOperatorNodePushable {
protected final boolean lowKeyInclusive;
@@ -52,7 +55,7 @@
this(ctx, partition, inputRecDesc, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive,
minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory, retainInput, retainMissing,
nonMatchWriterFactory, searchCallbackFactory, appendIndexFilter, nonFilterWriterFactory, null, -1,
- false, null, null);
+ false, null, null, DefaultTupleProjectorFactory.INSTANCE);
}
public BTreeSearchOperatorNodePushable(IHyracksTaskContext ctx, int partition, RecordDescriptor inputRecDesc,
@@ -62,11 +65,12 @@
ISearchOperationCallbackFactory searchCallbackFactory, boolean appendIndexFilter,
IMissingWriterFactory nonFilterWriterFactory, ITupleFilterFactory tupleFilterFactory, long outputLimit,
boolean appendOpCallbackProceedResult, byte[] searchCallbackProceedResultFalseValue,
- byte[] searchCallbackProceedResultTrueValue) throws HyracksDataException {
+ byte[] searchCallbackProceedResultTrueValue, ITupleProjectorFactory projectorFactory)
+ throws HyracksDataException {
super(ctx, inputRecDesc, partition, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
retainInput, retainMissing, nonMatchWriterFactory, searchCallbackFactory, appendIndexFilter,
nonFilterWriterFactory, tupleFilterFactory, outputLimit, appendOpCallbackProceedResult,
- searchCallbackProceedResultFalseValue, searchCallbackProceedResultTrueValue);
+ searchCallbackProceedResultFalseValue, searchCallbackProceedResultTrueValue, projectorFactory);
this.lowKeyInclusive = lowKeyInclusive;
this.highKeyInclusive = highKeyInclusive;
if (lowKeyFields != null && lowKeyFields.length > 0) {
@@ -111,7 +115,8 @@
@Override
protected void addAdditionalIndexAccessorParams(IIndexAccessParameters iap) throws HyracksDataException {
- // No additional parameters are required for the B+Tree search case
+ //Set tuple projector to get the information about the pushed down value accesses (if supported by the index)
+ iap.getParameters().put(HyracksConstants.TUPLE_PROJECTOR, tupleProjector);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java
index 78faaff..40b2f5c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java
@@ -42,7 +42,6 @@
import org.apache.hyracks.storage.am.btree.impls.BTreeOpContext.PageValidationInfo;
import org.apache.hyracks.storage.am.common.api.IBTreeIndexTupleReference;
import org.apache.hyracks.storage.am.common.api.IPageManager;
-import org.apache.hyracks.storage.am.common.api.ISplitKey;
import org.apache.hyracks.storage.am.common.api.ITreeIndexAccessor;
import org.apache.hyracks.storage.am.common.api.ITreeIndexCursor;
import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
@@ -51,7 +50,6 @@
import org.apache.hyracks.storage.am.common.frames.FrameOpSpaceStatus;
import org.apache.hyracks.storage.am.common.impls.AbstractTreeIndex;
import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
-import org.apache.hyracks.storage.am.common.impls.NodeFrontier;
import org.apache.hyracks.storage.am.common.impls.TreeIndexDiskOrderScanCursor;
import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
import org.apache.hyracks.storage.common.IIndexAccessParameters;
@@ -64,17 +62,13 @@
import org.apache.hyracks.storage.common.ISearchPredicate;
import org.apache.hyracks.storage.common.MultiComparator;
import org.apache.hyracks.storage.common.NoOpIndexCursorStats;
-import org.apache.hyracks.storage.common.buffercache.BufferCache;
import org.apache.hyracks.storage.common.buffercache.IBufferCache;
import org.apache.hyracks.storage.common.buffercache.ICachedPage;
import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
import org.apache.hyracks.storage.common.file.BufferedFileHandle;
-import org.apache.hyracks.util.JSONUtil;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
-import com.fasterxml.jackson.databind.node.ObjectNode;
-
public class BTree extends AbstractTreeIndex {
public static final float DEFAULT_FILL_FACTOR = 0.7f;
@@ -85,7 +79,7 @@
private final AtomicInteger smoCounter;
private final ReadWriteLock treeLatch;
- private final int maxTupleSize;
+ protected final int maxTupleSize;
public BTree(IBufferCache bufferCache, IPageManager freePageManager, ITreeIndexFrameFactory interiorFrameFactory,
ITreeIndexFrameFactory leafFrameFactory, IBinaryComparatorFactory[] cmpFactories, int fieldCount,
@@ -886,13 +880,13 @@
}
@Override
- public BTreeRangeSearchCursor createSearchCursor(boolean exclusive) {
+ public ITreeIndexCursor createSearchCursor(boolean exclusive) {
IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) btree.getLeafFrameFactory().createFrame();
return new BTreeRangeSearchCursor(leafFrame, exclusive, (IIndexCursorStats) iap.getParameters()
.getOrDefault(HyracksConstants.INDEX_CURSOR_STATS, NoOpIndexCursorStats.INSTANCE));
}
- public BTreeRangeSearchCursor createPointCursor(boolean exclusive, boolean stateful) {
+ public ITreeIndexCursor createPointCursor(boolean exclusive, boolean stateful) {
return createSearchCursor(exclusive);
}
@@ -1004,220 +998,7 @@
@Override
public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
boolean checkIfEmptyIndex, IPageWriteCallback callback) throws HyracksDataException {
- return new BTreeBulkLoader(fillFactor, verifyInput, callback);
- }
-
- public class BTreeBulkLoader extends AbstractTreeIndex.AbstractTreeIndexBulkLoader {
- protected final ISplitKey splitKey;
- protected final boolean verifyInput;
-
- public BTreeBulkLoader(float fillFactor, boolean verifyInput, IPageWriteCallback callback)
- throws HyracksDataException {
- super(fillFactor, callback);
- this.verifyInput = verifyInput;
- splitKey = new BTreeSplitKey(leafFrame.getTupleWriter().createTupleReference());
- splitKey.getTuple().setFieldCount(cmp.getKeyFieldCount());
- }
-
- @Override
- public void add(ITupleReference tuple) throws HyracksDataException {
- try {
- int tupleSize = Math.max(leafFrame.getBytesRequiredToWriteTuple(tuple),
- interiorFrame.getBytesRequiredToWriteTuple(tuple));
- NodeFrontier leafFrontier = nodeFrontiers.get(0);
- int spaceNeeded = tupleWriter.bytesRequired(tuple) + slotSize;
- int spaceUsed = leafFrame.getBuffer().capacity() - leafFrame.getTotalFreeSpace();
-
- // try to free space by compression
- if (spaceUsed + spaceNeeded > leafMaxBytes) {
- leafFrame.compress();
- spaceUsed = leafFrame.getBuffer().capacity() - leafFrame.getTotalFreeSpace();
- }
- //full, allocate new page
- if (spaceUsed + spaceNeeded > leafMaxBytes) {
- if (leafFrame.getTupleCount() == 0) {
- bufferCache.returnPage(leafFrontier.page, false);
- } else {
- leafFrontier.lastTuple.resetByTupleIndex(leafFrame, leafFrame.getTupleCount() - 1);
- if (verifyInput) {
- verifyInputTuple(tuple, leafFrontier.lastTuple);
- }
- int splitKeySize = tupleWriter.bytesRequired(leafFrontier.lastTuple, 0, cmp.getKeyFieldCount());
- splitKey.initData(splitKeySize);
- tupleWriter.writeTupleFields(leafFrontier.lastTuple, 0, cmp.getKeyFieldCount(),
- splitKey.getBuffer().array(), 0);
- splitKey.getTuple().resetByTupleOffset(splitKey.getBuffer().array(), 0);
- splitKey.setLeftPage(leafFrontier.pageId);
-
- propagateBulk(1, pagesToWrite);
-
- leafFrontier.pageId = freePageManager.takePage(metaFrame);
-
- ((IBTreeLeafFrame) leafFrame).setNextLeaf(leafFrontier.pageId);
-
- write(leafFrontier.page);
- for (ICachedPage c : pagesToWrite) {
- write(c);
- }
- pagesToWrite.clear();
- splitKey.setRightPage(leafFrontier.pageId);
- }
- if (tupleSize > maxTupleSize) {
- final long dpid = BufferedFileHandle.getDiskPageId(getFileId(), leafFrontier.pageId);
- // calculate required number of pages.
- int headerSize = Math.max(leafFrame.getPageHeaderSize(), interiorFrame.getPageHeaderSize());
- final int multiplier =
- (int) Math.ceil((double) tupleSize / (bufferCache.getPageSize() - headerSize));
- if (multiplier > 1) {
- leafFrontier.page = bufferCache.confiscateLargePage(dpid, multiplier,
- freePageManager.takeBlock(metaFrame, multiplier - 1));
- } else {
- leafFrontier.page = bufferCache.confiscatePage(dpid);
- }
- leafFrame.setPage(leafFrontier.page);
- leafFrame.initBuffer((byte) 0);
- ((IBTreeLeafFrame) leafFrame).setLargeFlag(true);
- } else {
- final long dpid = BufferedFileHandle.getDiskPageId(getFileId(), leafFrontier.pageId);
- leafFrontier.page = bufferCache.confiscatePage(dpid);
- leafFrame.setPage(leafFrontier.page);
- leafFrame.initBuffer((byte) 0);
- }
- } else {
- if (verifyInput && leafFrame.getTupleCount() > 0) {
- leafFrontier.lastTuple.resetByTupleIndex(leafFrame, leafFrame.getTupleCount() - 1);
- verifyInputTuple(tuple, leafFrontier.lastTuple);
- }
- }
- ((IBTreeLeafFrame) leafFrame).insertSorted(tuple);
- } catch (HyracksDataException | RuntimeException e) {
- logState(tuple, e);
- handleException();
- throw e;
- }
- }
-
- protected void verifyInputTuple(ITupleReference tuple, ITupleReference prevTuple) throws HyracksDataException {
- // New tuple should be strictly greater than last tuple.
- int cmpResult = cmp.compare(tuple, prevTuple);
- if (cmpResult < 0) {
- throw HyracksDataException.create(ErrorCode.UNSORTED_LOAD_INPUT);
- }
- if (cmpResult == 0) {
- throw HyracksDataException.create(ErrorCode.DUPLICATE_LOAD_INPUT);
- }
- }
-
- protected void propagateBulk(int level, List<ICachedPage> pagesToWrite) throws HyracksDataException {
- if (splitKey.getBuffer() == null) {
- return;
- }
-
- if (level >= nodeFrontiers.size()) {
- addLevel();
- }
-
- NodeFrontier frontier = nodeFrontiers.get(level);
- interiorFrame.setPage(frontier.page);
-
- ITupleReference tuple = splitKey.getTuple();
- int tupleBytes = tupleWriter.bytesRequired(tuple, 0, cmp.getKeyFieldCount());
- int spaceNeeded = tupleBytes + slotSize + 4;
- if (tupleBytes > interiorFrame.getMaxTupleSize(BTree.this.bufferCache.getPageSize())) {
- throw HyracksDataException.create(ErrorCode.RECORD_IS_TOO_LARGE, tupleBytes,
- interiorFrame.getMaxTupleSize(BTree.this.bufferCache.getPageSize()));
- }
-
- int spaceUsed = interiorFrame.getBuffer().capacity() - interiorFrame.getTotalFreeSpace();
- if (spaceUsed + spaceNeeded > interiorMaxBytes) {
-
- ISplitKey copyKey = splitKey.duplicate(leafFrame.getTupleWriter().createTupleReference());
- tuple = copyKey.getTuple();
-
- frontier.lastTuple.resetByTupleIndex(interiorFrame, interiorFrame.getTupleCount() - 1);
- int splitKeySize = tupleWriter.bytesRequired(frontier.lastTuple, 0, cmp.getKeyFieldCount());
- splitKey.initData(splitKeySize);
- tupleWriter.writeTupleFields(frontier.lastTuple, 0, cmp.getKeyFieldCount(),
- splitKey.getBuffer().array(), 0);
- splitKey.getTuple().resetByTupleOffset(splitKey.getBuffer().array(), 0);
-
- ((IBTreeInteriorFrame) interiorFrame).deleteGreatest();
- int finalPageId = freePageManager.takePage(metaFrame);
- frontier.page.setDiskPageId(BufferedFileHandle.getDiskPageId(getFileId(), finalPageId));
- pagesToWrite.add(frontier.page);
- splitKey.setLeftPage(finalPageId);
-
- propagateBulk(level + 1, pagesToWrite);
- frontier.page = bufferCache.confiscatePage(BufferCache.INVALID_DPID);
- interiorFrame.setPage(frontier.page);
- interiorFrame.initBuffer((byte) level);
- }
- ((IBTreeInteriorFrame) interiorFrame).insertSorted(tuple);
- }
-
- private void persistFrontiers(int level, int rightPage) throws HyracksDataException {
- if (level >= nodeFrontiers.size()) {
- rootPage = nodeFrontiers.get(level - 1).pageId;
- releasedLatches = true;
- return;
- }
- if (level < 1) {
- ICachedPage lastLeaf = nodeFrontiers.get(level).page;
- int lastLeafPage = nodeFrontiers.get(level).pageId;
- lastLeaf.setDiskPageId(BufferedFileHandle.getDiskPageId(getFileId(), nodeFrontiers.get(level).pageId));
- write(lastLeaf);
- nodeFrontiers.get(level).page = null;
- persistFrontiers(level + 1, lastLeafPage);
- return;
- }
- NodeFrontier frontier = nodeFrontiers.get(level);
- interiorFrame.setPage(frontier.page);
- //just finalize = the layer right above the leaves has correct righthand pointers already
- if (rightPage < 0) {
- throw new HyracksDataException(
- "Error in index creation. Internal node appears to have no rightmost guide");
- }
- ((IBTreeInteriorFrame) interiorFrame).setRightmostChildPageId(rightPage);
- int finalPageId = freePageManager.takePage(metaFrame);
- frontier.page.setDiskPageId(BufferedFileHandle.getDiskPageId(getFileId(), finalPageId));
- write(frontier.page);
- frontier.pageId = finalPageId;
- persistFrontiers(level + 1, finalPageId);
- }
-
- @Override
- public void end() throws HyracksDataException {
- try {
- persistFrontiers(0, -1);
- super.end();
- } catch (HyracksDataException | RuntimeException e) {
- handleException();
- throw e;
- }
- }
-
- @Override
- public void abort() throws HyracksDataException {
- super.handleException();
- }
-
- private void logState(ITupleReference tuple, Exception e) {
- try {
- ObjectNode state = JSONUtil.createObject();
- state.set("leafFrame", leafFrame.getState());
- state.set("interiorFrame", interiorFrame.getState());
- int tupleSize = Math.max(leafFrame.getBytesRequiredToWriteTuple(tuple),
- interiorFrame.getBytesRequiredToWriteTuple(tuple));
- state.put("tupleSize", tupleSize);
- state.put("spaceNeeded", tupleWriter.bytesRequired(tuple) + slotSize);
- state.put("spaceUsed", leafFrame.getBuffer().capacity() - leafFrame.getTotalFreeSpace());
- state.put("leafMaxBytes", leafMaxBytes);
- state.put("maxTupleSize", maxTupleSize);
- LOGGER.error("failed to add tuple {}", state, e);
- } catch (Throwable t) {
- e.addSuppressed(t);
- }
- }
+ return new BTreeNSMBulkLoader(fillFactor, verifyInput, callback, this);
}
@SuppressWarnings("rawtypes")
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeNSMBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeNSMBulkLoader.java
new file mode 100644
index 0000000..04c84e1
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeNSMBulkLoader.java
@@ -0,0 +1,279 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.btree.impls;
+
+import java.util.List;
+
+import org.apache.hyracks.api.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.btree.api.IBTreeInteriorFrame;
+import org.apache.hyracks.storage.am.btree.api.IBTreeLeafFrame;
+import org.apache.hyracks.storage.am.common.api.ISplitKey;
+import org.apache.hyracks.storage.am.common.api.ITreeIndex;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
+import org.apache.hyracks.storage.am.common.impls.AbstractTreeIndexBulkLoader;
+import org.apache.hyracks.storage.am.common.impls.NodeFrontier;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.storage.common.buffercache.ICachedPage;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
+import org.apache.hyracks.storage.common.file.BufferedFileHandle;
+import org.apache.hyracks.util.JSONUtil;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
+public class BTreeNSMBulkLoader extends AbstractTreeIndexBulkLoader {
+ private static final Logger LOGGER = LogManager.getLogger();
+ protected final ISplitKey splitKey;
+ protected final boolean verifyInput;
+ private final int maxTupleSize;
+
+ public BTreeNSMBulkLoader(float fillFactor, boolean verifyInput, IPageWriteCallback callback, ITreeIndex index)
+ throws HyracksDataException {
+ this(fillFactor, verifyInput, callback, index, index.getLeafFrameFactory().createFrame());
+ }
+
+ protected BTreeNSMBulkLoader(float fillFactor, boolean verifyInput, IPageWriteCallback callback, ITreeIndex index,
+ ITreeIndexFrame leafFrame) throws HyracksDataException {
+ super(fillFactor, callback, index, leafFrame);
+ this.verifyInput = verifyInput;
+ splitKey = new BTreeSplitKey(tupleWriter.createTupleReference());
+ splitKey.getTuple().setFieldCount(cmp.getKeyFieldCount());
+ maxTupleSize = ((BTree) index).maxTupleSize;
+ }
+
+ @Override
+ public void add(ITupleReference tuple) throws HyracksDataException {
+ try {
+ int tupleSize = Math.max(leafFrame.getBytesRequiredToWriteTuple(tuple),
+ interiorFrame.getBytesRequiredToWriteTuple(tuple));
+ NodeFrontier leafFrontier = nodeFrontiers.get(0);
+ int spaceNeeded = tupleWriter.bytesRequired(tuple) + slotSize;
+ int spaceUsed = leafFrame.getBuffer().capacity() - leafFrame.getTotalFreeSpace();
+
+ // try to free space by compression
+ if (spaceUsed + spaceNeeded > leafMaxBytes) {
+ leafFrame.compress();
+ spaceUsed = leafFrame.getBuffer().capacity() - leafFrame.getTotalFreeSpace();
+ }
+ //full, allocate new page
+ if (spaceUsed + spaceNeeded > leafMaxBytes) {
+ if (leafFrame.getTupleCount() == 0) {
+ //The current page is empty. Return it.
+ bufferCache.returnPage(leafFrontier.page, false);
+ } else {
+ leafFrontier.lastTuple.resetByTupleIndex(leafFrame, leafFrame.getTupleCount() - 1);
+ if (verifyInput) {
+ verifyInputTuple(tuple, leafFrontier.lastTuple);
+ }
+ //The current page is not empty. Write it.
+ writeFullLeafPage();
+ }
+ if (tupleSize > maxTupleSize) {
+ //We need a large page
+ final long dpid = BufferedFileHandle.getDiskPageId(fileId, leafFrontier.pageId);
+ // calculate required number of pages.
+ int headerSize = Math.max(leafFrame.getPageHeaderSize(), interiorFrame.getPageHeaderSize());
+ final int multiplier =
+ (int) Math.ceil((double) tupleSize / (bufferCache.getPageSize() - headerSize));
+ if (multiplier > 1) {
+ leafFrontier.page = bufferCache.confiscateLargePage(dpid, multiplier,
+ freePageManager.takeBlock(metaFrame, multiplier - 1));
+ } else {
+ leafFrontier.page = bufferCache.confiscatePage(dpid);
+ }
+ leafFrame.setPage(leafFrontier.page);
+ leafFrame.initBuffer((byte) 0);
+ ((IBTreeLeafFrame) leafFrame).setLargeFlag(true);
+ } else {
+ //allocate a new page
+ confiscateNewLeafPage();
+ }
+ } else {
+ if (verifyInput && leafFrame.getTupleCount() > 0) {
+ leafFrontier.lastTuple.resetByTupleIndex(leafFrame, leafFrame.getTupleCount() - 1);
+ verifyInputTuple(tuple, leafFrontier.lastTuple);
+ }
+ }
+ ((IBTreeLeafFrame) leafFrame).insertSorted(tuple);
+ } catch (HyracksDataException | RuntimeException e) {
+ logState(tuple, e);
+ handleException();
+ throw e;
+ }
+ }
+
+ protected void verifyInputTuple(ITupleReference tuple, ITupleReference prevTuple) throws HyracksDataException {
+ // New tuple should be strictly greater than last tuple.
+ int cmpResult = cmp.compare(tuple, prevTuple);
+ if (cmpResult < 0) {
+ throw HyracksDataException.create(ErrorCode.UNSORTED_LOAD_INPUT);
+ }
+ if (cmpResult == 0) {
+ throw HyracksDataException.create(ErrorCode.DUPLICATE_LOAD_INPUT);
+ }
+ }
+
+ protected void propagateBulk(int level, List<ICachedPage> pagesToWrite) throws HyracksDataException {
+ if (splitKey.getBuffer() == null) {
+ return;
+ }
+
+ if (level >= nodeFrontiers.size()) {
+ addLevel();
+ }
+
+ NodeFrontier frontier = nodeFrontiers.get(level);
+ interiorFrame.setPage(frontier.page);
+
+ ITupleReference tuple = splitKey.getTuple();
+ int tupleBytes = tupleWriter.bytesRequired(tuple, 0, cmp.getKeyFieldCount());
+ int spaceNeeded = tupleBytes + slotSize + 4;
+ if (tupleBytes > interiorFrame.getMaxTupleSize(bufferCache.getPageSize())) {
+ throw HyracksDataException.create(ErrorCode.RECORD_IS_TOO_LARGE, tupleBytes,
+ interiorFrame.getMaxTupleSize(bufferCache.getPageSize()));
+ }
+
+ int spaceUsed = interiorFrame.getBuffer().capacity() - interiorFrame.getTotalFreeSpace();
+ if (spaceUsed + spaceNeeded > interiorMaxBytes) {
+ ISplitKey copyKey = splitKey.duplicate(tupleWriter.createTupleReference());
+ tuple = copyKey.getTuple();
+
+ frontier.lastTuple.resetByTupleIndex(interiorFrame, interiorFrame.getTupleCount() - 1);
+ int splitKeySize = tupleWriter.bytesRequired(frontier.lastTuple, 0, cmp.getKeyFieldCount());
+ splitKey.initData(splitKeySize);
+ tupleWriter.writeTupleFields(frontier.lastTuple, 0, cmp.getKeyFieldCount(), splitKey.getBuffer().array(),
+ 0);
+ splitKey.getTuple().resetByTupleOffset(splitKey.getBuffer().array(), 0);
+
+ ((IBTreeInteriorFrame) interiorFrame).deleteGreatest();
+ int finalPageId = freePageManager.takePage(metaFrame);
+ frontier.page.setDiskPageId(BufferedFileHandle.getDiskPageId(fileId, finalPageId));
+ pagesToWrite.add(frontier.page);
+ splitKey.setLeftPage(finalPageId);
+
+ propagateBulk(level + 1, pagesToWrite);
+ frontier.page = bufferCache.confiscatePage(IBufferCache.INVALID_DPID);
+ interiorFrame.setPage(frontier.page);
+ interiorFrame.initBuffer((byte) level);
+ }
+ ((IBTreeInteriorFrame) interiorFrame).insertSorted(tuple);
+ }
+
+ private void persistFrontiers(int level, int rightPage) throws HyracksDataException {
+ if (level >= nodeFrontiers.size()) {
+ setRootPageId(nodeFrontiers.get(level - 1).pageId);
+ releasedLatches = true;
+ return;
+ }
+ if (level < 1) {
+ ICachedPage lastLeaf = nodeFrontiers.get(level).page;
+ int lastLeafPage = nodeFrontiers.get(level).pageId;
+ lastLeaf.setDiskPageId(BufferedFileHandle.getDiskPageId(fileId, nodeFrontiers.get(level).pageId));
+ writeLastLeaf(lastLeaf);
+ nodeFrontiers.get(level).page = null;
+ persistFrontiers(level + 1, lastLeafPage);
+ return;
+ }
+ NodeFrontier frontier = nodeFrontiers.get(level);
+ interiorFrame.setPage(frontier.page);
+ //just finalize = the layer right above the leaves has correct righthand pointers already
+ if (rightPage < 0) {
+ throw new HyracksDataException("Error in index creation. Internal node appears to have no rightmost guide");
+ }
+ ((IBTreeInteriorFrame) interiorFrame).setRightmostChildPageId(rightPage);
+ int finalPageId = freePageManager.takePage(metaFrame);
+ frontier.page.setDiskPageId(BufferedFileHandle.getDiskPageId(fileId, finalPageId));
+ write(frontier.page);
+ frontier.pageId = finalPageId;
+ persistFrontiers(level + 1, finalPageId);
+ }
+
+ @Override
+ public void end() throws HyracksDataException {
+ try {
+ persistFrontiers(0, -1);
+ super.end();
+ } catch (HyracksDataException | RuntimeException e) {
+ handleException();
+ throw e;
+ }
+ }
+
+ @Override
+ public void abort() throws HyracksDataException {
+ super.handleException();
+ }
+
+ protected void writeFullLeafPage() throws HyracksDataException {
+ final NodeFrontier leafFrontier = nodeFrontiers.get(0);
+ leafFrontier.lastTuple.resetByTupleIndex(leafFrame, leafFrame.getTupleCount() - 1);
+ final int splitKeySize = tupleWriter.bytesRequired(leafFrontier.lastTuple, 0, cmp.getKeyFieldCount());
+ splitKey.initData(splitKeySize);
+ tupleWriter.writeTupleFields(leafFrontier.lastTuple, 0, cmp.getKeyFieldCount(), splitKey.getBuffer().array(),
+ 0);
+ splitKey.getTuple().resetByTupleOffset(splitKey.getBuffer().array(), 0);
+ splitKey.setLeftPage(leafFrontier.pageId);
+
+ propagateBulk(1, pagesToWrite);
+
+ leafFrontier.pageId = freePageManager.takePage(metaFrame);
+
+ ((IBTreeLeafFrame) leafFrame).setNextLeaf(leafFrontier.pageId);
+
+ write(leafFrontier.page);
+ for (ICachedPage c : pagesToWrite) {
+ write(c);
+ }
+ pagesToWrite.clear();
+ splitKey.setRightPage(leafFrontier.pageId);
+ }
+
+ protected void writeLastLeaf(ICachedPage page) throws HyracksDataException {
+ write(page);
+ }
+
+ protected final void confiscateNewLeafPage() throws HyracksDataException {
+ final NodeFrontier leafFrontier = nodeFrontiers.get(0);
+ final long dpid = BufferedFileHandle.getDiskPageId(fileId, leafFrontier.pageId);
+ leafFrontier.page = bufferCache.confiscatePage(dpid);
+ leafFrame.setPage(leafFrontier.page);
+ leafFrame.initBuffer((byte) 0);
+ }
+
+ private void logState(ITupleReference tuple, Exception e) {
+ try {
+ ObjectNode state = JSONUtil.createObject();
+ state.set("leafFrame", leafFrame.getState());
+ state.set("interiorFrame", interiorFrame.getState());
+ int tupleSize = Math.max(leafFrame.getBytesRequiredToWriteTuple(tuple),
+ interiorFrame.getBytesRequiredToWriteTuple(tuple));
+ state.put("tupleSize", tupleSize);
+ state.put("spaceNeeded", tupleWriter.bytesRequired(tuple) + slotSize);
+ state.put("spaceUsed", leafFrame.getBuffer().capacity() - leafFrame.getTotalFreeSpace());
+ state.put("leafMaxBytes", leafMaxBytes);
+ state.put("maxTupleSize", maxTupleSize);
+ LOGGER.error("failed to add tuple {}", state, e);
+ } catch (Throwable t) {
+ e.addSuppressed(t);
+ }
+ }
+}
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTree.java
index ae6bbaa..179f1da 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTree.java
@@ -24,8 +24,8 @@
import org.apache.hyracks.api.io.FileReference;
import org.apache.hyracks.api.util.HyracksConstants;
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
-import org.apache.hyracks.storage.am.btree.api.IBTreeFrame;
import org.apache.hyracks.storage.am.btree.api.IBTreeLeafFrame;
+import org.apache.hyracks.storage.am.btree.api.IDiskBTreeStatefulPointSearchCursor;
import org.apache.hyracks.storage.am.common.api.IPageManager;
import org.apache.hyracks.storage.am.common.api.ITreeIndexCursor;
import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
@@ -38,7 +38,6 @@
import org.apache.hyracks.storage.common.ISearchPredicate;
import org.apache.hyracks.storage.common.MultiComparator;
import org.apache.hyracks.storage.common.NoOpIndexCursorStats;
-import org.apache.hyracks.storage.common.buffercache.BufferCache;
import org.apache.hyracks.storage.common.buffercache.IBufferCache;
import org.apache.hyracks.storage.common.buffercache.ICachedPage;
import org.apache.hyracks.storage.common.file.BufferedFileHandle;
@@ -76,7 +75,8 @@
private void search(ITreeIndexCursor cursor, ISearchPredicate searchPred, BTreeOpContext ctx)
throws HyracksDataException {
ctx.reset();
- ctx.setPred((RangePredicate) searchPred);
+ RangePredicate rangePredicate = (RangePredicate) searchPred;
+ ctx.setPred(rangePredicate);
ctx.setCursor(cursor);
if (ctx.getPred().getLowKeyComparator() == null) {
ctx.getPred().setLowKeyComparator(ctx.getCmp());
@@ -87,25 +87,17 @@
cursor.setBufferCache(bufferCache);
cursor.setFileId(getFileId());
- if (cursor instanceof DiskBTreePointSearchCursor) {
- DiskBTreePointSearchCursor pointCursor = (DiskBTreePointSearchCursor) cursor;
+ if (cursor instanceof IDiskBTreeStatefulPointSearchCursor) {
+ IDiskBTreeStatefulPointSearchCursor pointCursor = (IDiskBTreeStatefulPointSearchCursor) cursor;
int lastPageId = pointCursor.getLastPageId();
- if (lastPageId != BufferCache.INVALID_PAGEID) {
- // check whether the last leaf page contains this key
- ICachedPage lastPage =
- bufferCache.pin(BufferedFileHandle.getDiskPageId(getFileId(), lastPageId), false);
- ctx.getLeafFrame().setPage(lastPage);
- if (fitInPage(ctx.getPred().getLowKey(), ctx.getPred().getLowKeyComparator(), ctx.getLeafFrame())) {
- // use this page
- ctx.getCursorInitialState().setPage(lastPage);
- ctx.getCursorInitialState().setPageId(lastPageId);
- pointCursor.open(ctx.getCursorInitialState(), searchPred);
+ if (lastPageId != IBufferCache.INVALID_PAGEID) {
+ if (fitInPage(ctx.getPred().getLowKey(), ctx.getPred().getLowKeyComparator(), pointCursor.getFrame())) {
+ pointCursor.setCursorToNextKey(searchPred);
return;
} else {
- // release the last page and clear the states of this cursor
+ // release the last page, clear the states of this cursor, and close the cursor
// then retry the search from root to leaf
- bufferCache.unpin(lastPage);
- pointCursor.clearSearchState();
+ cursor.close();
}
}
}
@@ -113,7 +105,7 @@
searchDown(rootNode, rootPage, ctx, cursor);
}
- private boolean fitInPage(ITupleReference key, MultiComparator comparator, IBTreeFrame frame)
+ private boolean fitInPage(ITupleReference key, MultiComparator comparator, ITreeIndexFrame frame)
throws HyracksDataException {
// assume that search keys are sorted (non-decreasing)
ITupleReference rightmostTuple = frame.getRightmostTuple();
@@ -191,14 +183,14 @@
}
@Override
- public DiskBTreeRangeSearchCursor createSearchCursor(boolean exclusive) {
+ public ITreeIndexCursor createSearchCursor(boolean exclusive) {
IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) btree.getLeafFrameFactory().createFrame();
return new DiskBTreeRangeSearchCursor(leafFrame, exclusive, (IIndexCursorStats) iap.getParameters()
.getOrDefault(HyracksConstants.INDEX_CURSOR_STATS, NoOpIndexCursorStats.INSTANCE));
}
@Override
- public BTreeRangeSearchCursor createPointCursor(boolean exclusive, boolean stateful) {
+ public ITreeIndexCursor createPointCursor(boolean exclusive, boolean stateful) {
IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) btree.getLeafFrameFactory().createFrame();
return new DiskBTreePointSearchCursor(leafFrame, exclusive, stateful);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTreePointSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTreePointSearchCursor.java
index 1bf3ecf..8fd9a96 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTreePointSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTreePointSearchCursor.java
@@ -21,13 +21,16 @@
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.storage.am.btree.api.IBTreeLeafFrame;
+import org.apache.hyracks.storage.am.btree.api.IDiskBTreeStatefulPointSearchCursor;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
import org.apache.hyracks.storage.am.common.ophelpers.FindTupleMode;
import org.apache.hyracks.storage.am.common.ophelpers.FindTupleNoExactMatchPolicy;
import org.apache.hyracks.storage.common.ICursorInitialState;
import org.apache.hyracks.storage.common.ISearchPredicate;
-import org.apache.hyracks.storage.common.buffercache.BufferCache;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
-public class DiskBTreePointSearchCursor extends DiskBTreeRangeSearchCursor {
+public class DiskBTreePointSearchCursor extends DiskBTreeRangeSearchCursor
+ implements IDiskBTreeStatefulPointSearchCursor {
/**
* A stateful cursor keeps the search state (last search page Id + index) across multiple searches
* until {@link #clearSearchState()} is called explicity
@@ -36,16 +39,14 @@
private boolean nextHasBeenCalled;
- private int lastPageId = BufferCache.INVALID_PAGEID;
+ private int lastPageId;
private int lastTupleIndex = 0;
public DiskBTreePointSearchCursor(IBTreeLeafFrame frame, boolean exclusiveLatchNodes, boolean stateful) {
super(frame, exclusiveLatchNodes);
this.stateful = stateful;
- }
-
- public DiskBTreePointSearchCursor(IBTreeLeafFrame frame, boolean exclusiveLatchNodes) {
- this(frame, exclusiveLatchNodes, false);
+ lastPageId = IBufferCache.INVALID_PAGEID;
+ lastTupleIndex = 0;
}
@Override
@@ -59,6 +60,12 @@
}
@Override
+ public void doClose() throws HyracksDataException {
+ clearSearchState();
+ super.doClose();
+ }
+
+ @Override
public void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
// in case open is called multiple times without closing
if (page != null) {
@@ -71,30 +78,10 @@
page = initialState.getPage();
isPageDirty = false;
frame.setPage(page);
-
- pred = (RangePredicate) searchPred;
- lowKeyCmp = pred.getLowKeyComparator();
- lowKey = pred.getLowKey();
-
- reusablePredicate.setLowKeyComparator(originalKeyCmp);
-
- lowKeyFtm = FindTupleMode.EXACT;
- lowKeyFtp = FindTupleNoExactMatchPolicy.NONE;
-
- nextHasBeenCalled = false;
-
- // only get the low key position
- tupleIndex = getLowKeyIndex();
- if (stateful) {
- lastPageId = pageId;
- if (tupleIndex >= 0) {
- lastTupleIndex = tupleIndex;
- } else {
- lastTupleIndex = -tupleIndex - 1;
- }
- }
+ setCursorToNextKey(searchPred);
}
+ @Override
public int getLastPageId() {
return lastPageId;
}
@@ -108,9 +95,38 @@
}
}
- public void clearSearchState() {
- this.lastPageId = BufferCache.INVALID_PAGEID;
+ @Override
+ public void setCursorToNextKey(ISearchPredicate searchPred) throws HyracksDataException {
+ pred = (RangePredicate) searchPred;
+ lowKeyCmp = pred.getLowKeyComparator();
+ lowKey = pred.getLowKey();
+
+ reusablePredicate.setLowKeyComparator(originalKeyCmp);
+
+ lowKeyFtm = FindTupleMode.EXACT;
+ lowKeyFtp = FindTupleNoExactMatchPolicy.NONE;
+
+ nextHasBeenCalled = false;
+
+ // only get the lowKey position
+ tupleIndex = getLowKeyIndex();
+ if (stateful) {
+ lastPageId = pageId;
+ if (tupleIndex >= 0) {
+ lastTupleIndex = tupleIndex;
+ } else {
+ lastTupleIndex = -tupleIndex - 1;
+ }
+ }
+ }
+
+ private void clearSearchState() {
+ this.lastPageId = IBufferCache.INVALID_PAGEID;
this.lastTupleIndex = 0;
}
+ @Override
+ public ITreeIndexFrame getFrame() {
+ return frame;
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
index eab8c96..d555b31 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
@@ -24,11 +24,13 @@
import org.apache.hyracks.api.comm.VSizeFrame;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.IIntrospectingOperator;
import org.apache.hyracks.api.dataflow.value.IMissingWriter;
import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.job.profiling.IOperatorStats;
+import org.apache.hyracks.api.job.profiling.NoOpOperatorStats;
import org.apache.hyracks.api.util.CleanupUtils;
import org.apache.hyracks.api.util.ExceptionUtils;
import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
@@ -44,6 +46,7 @@
import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
import org.apache.hyracks.storage.am.common.api.ITupleFilter;
import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
+import org.apache.hyracks.storage.am.common.impls.DefaultTupleProjectorFactory;
import org.apache.hyracks.storage.am.common.impls.IndexAccessParameters;
import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
import org.apache.hyracks.storage.am.common.tuples.ReferenceFrameTupleReference;
@@ -54,12 +57,15 @@
import org.apache.hyracks.storage.common.IIndexCursor;
import org.apache.hyracks.storage.common.ISearchOperationCallback;
import org.apache.hyracks.storage.common.ISearchPredicate;
+import org.apache.hyracks.storage.common.projection.ITupleProjector;
+import org.apache.hyracks.storage.common.projection.ITupleProjectorFactory;
import org.apache.hyracks.util.IThreadStatsCollector;
import org.apache.logging.log4j.Level;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
-public abstract class IndexSearchOperatorNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
+public abstract class IndexSearchOperatorNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable
+ implements IIntrospectingOperator {
static final Logger LOGGER = LogManager.getLogger();
protected final IHyracksTaskContext ctx;
@@ -107,6 +113,7 @@
protected final long outputLimit;
protected long outputCount = 0;
protected boolean finished;
+ protected final ITupleProjector tupleProjector;
// no filter and limit pushdown
public IndexSearchOperatorNodePushable(IHyracksTaskContext ctx, RecordDescriptor inputRecDesc, int partition,
@@ -116,7 +123,7 @@
IMissingWriterFactory nonFilterWriterFactory) throws HyracksDataException {
this(ctx, inputRecDesc, partition, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
retainInput, retainMissing, nonMatchWriterFactory, searchCallbackFactory, appendIndexFilter,
- nonFilterWriterFactory, null, -1, false, null, null);
+ nonFilterWriterFactory, null, -1, false, null, null, DefaultTupleProjectorFactory.INSTANCE);
}
public IndexSearchOperatorNodePushable(IHyracksTaskContext ctx, RecordDescriptor inputRecDesc, int partition,
@@ -125,7 +132,8 @@
ISearchOperationCallbackFactory searchCallbackFactory, boolean appendIndexFilter,
IMissingWriterFactory nonFilterWriterFactory, ITupleFilterFactory tupleFactoryFactory, long outputLimit,
boolean appendSearchCallbackProceedResult, byte[] searchCallbackProceedResultFalseValue,
- byte[] searchCallbackProceedResultTrueValue) throws HyracksDataException {
+ byte[] searchCallbackProceedResultTrueValue, ITupleProjectorFactory projectorFactory)
+ throws HyracksDataException {
this.ctx = ctx;
this.indexHelper = indexHelperFactory.create(ctx.getJobletContext().getServiceContext(), partition);
this.retainInput = retainInput;
@@ -154,14 +162,13 @@
this.searchCallbackProceedResultTrueValue = searchCallbackProceedResultTrueValue;
this.tupleFilterFactory = tupleFactoryFactory;
this.outputLimit = outputLimit;
-
- if (ctx != null && ctx.getStatsCollector() != null) {
- stats = ctx.getStatsCollector().getOrAddOperatorStats(getDisplayName());
- }
+ this.stats = new NoOpOperatorStats();
if (this.tupleFilterFactory != null && this.retainMissing) {
throw new IllegalStateException("RetainMissing with tuple filter is not supported");
}
+
+ tupleProjector = projectorFactory.createTupleProjector(ctx);
}
protected abstract ISearchPredicate createSearchPredicate();
@@ -266,7 +273,7 @@
break;
}
}
- stats.getTupleCounter().update(matchingTupleCount);
+ stats.getInputTupleCounter().update(matchingTupleCount);
if (matchingTupleCount == 0 && retainInput && retainMissing) {
FrameUtils.appendConcatToWriter(writer, appender, accessor, tupleIndex,
@@ -314,7 +321,8 @@
if (appender.getTupleCount() > 0) {
appender.write(writer, true);
}
- stats.getDiskIoCounter().update(ctx.getThreadStats().getPinnedPagesCount());
+ stats.getPageReads().update(ctx.getThreadStats().getPinnedPagesCount());
+ stats.coldReadCounter().update(ctx.getThreadStats().getColdReadCount());
} catch (Throwable th) { // NOSONAR Must ensure writer.fail is called.
// subsequently, the failure will be thrown
failure = th;
@@ -349,10 +357,7 @@
protected void writeTupleToOutput(ITupleReference tuple) throws IOException {
try {
- for (int i = 0; i < tuple.getFieldCount(); i++) {
- dos.write(tuple.getFieldData(i), tuple.getFieldStart(i), tuple.getFieldLength(i));
- tb.addFieldEndOffset();
- }
+ tupleProjector.project(tuple, dos, tb);
} catch (Exception e) {
throw e;
}
@@ -399,4 +404,9 @@
return "Index Search";
}
+ @Override
+ public void setOperatorStats(IOperatorStats stats) {
+ this.stats = stats;
+ }
+
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndex.java
index 81e528b..11368bf 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndex.java
@@ -19,9 +19,6 @@
package org.apache.hyracks.storage.am.common.impls;
-import java.util.ArrayList;
-import java.util.List;
-
import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import org.apache.hyracks.api.exceptions.ErrorCode;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -30,17 +27,9 @@
import org.apache.hyracks.storage.am.common.api.ITreeIndex;
import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
-import org.apache.hyracks.storage.am.common.api.ITreeIndexMetadataFrame;
-import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
-import org.apache.hyracks.storage.common.IIndexBulkLoader;
-import org.apache.hyracks.storage.common.MultiComparator;
import org.apache.hyracks.storage.common.buffercache.HaltOnFailureCallback;
import org.apache.hyracks.storage.common.buffercache.IBufferCache;
import org.apache.hyracks.storage.common.buffercache.ICachedPage;
-import org.apache.hyracks.storage.common.buffercache.IFIFOPageWriter;
-import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
-import org.apache.hyracks.storage.common.buffercache.PageWriteFailureCallback;
-import org.apache.hyracks.storage.common.compression.file.ICompressedPageWriter;
import org.apache.hyracks.storage.common.file.BufferedFileHandle;
public abstract class AbstractTreeIndex implements ITreeIndex {
@@ -229,118 +218,6 @@
return fieldCount;
}
- public abstract class AbstractTreeIndexBulkLoader extends PageWriteFailureCallback implements IIndexBulkLoader {
- protected final MultiComparator cmp;
- protected final int slotSize;
- protected final int leafMaxBytes;
- protected final int interiorMaxBytes;
- protected final ArrayList<NodeFrontier> nodeFrontiers = new ArrayList<>();
- protected final ITreeIndexMetadataFrame metaFrame;
- protected final ITreeIndexTupleWriter tupleWriter;
- protected ITreeIndexFrame leafFrame;
- protected ITreeIndexFrame interiorFrame;
- // Immutable bulk loaders write their root page at page -2, as needed e.g. by append-only file systems such as
- // HDFS. Since loading this tree relies on the root page actually being at that point, no further inserts into
- // that tree are allowed. Currently, this is not enforced.
- protected boolean releasedLatches;
- private final IFIFOPageWriter pageWriter;
- protected List<ICachedPage> pagesToWrite;
- private final ICompressedPageWriter compressedPageWriter;
-
- public AbstractTreeIndexBulkLoader(float fillFactor, IPageWriteCallback callback) throws HyracksDataException {
- leafFrame = leafFrameFactory.createFrame();
- interiorFrame = interiorFrameFactory.createFrame();
- metaFrame = freePageManager.createMetadataFrame();
-
- pageWriter = bufferCache.createFIFOWriter(callback, this);
-
- if (!isEmptyTree(leafFrame)) {
- throw HyracksDataException.create(ErrorCode.CANNOT_BULK_LOAD_NON_EMPTY_TREE);
- }
-
- this.cmp = MultiComparator.create(cmpFactories);
-
- leafFrame.setMultiComparator(cmp);
- interiorFrame.setMultiComparator(cmp);
-
- tupleWriter = leafFrame.getTupleWriter();
-
- NodeFrontier leafFrontier = new NodeFrontier(leafFrame.createTupleReference());
- leafFrontier.pageId = freePageManager.takePage(metaFrame);
- leafFrontier.page =
- bufferCache.confiscatePage(BufferedFileHandle.getDiskPageId(fileId, leafFrontier.pageId));
-
- interiorFrame.setPage(leafFrontier.page);
- interiorFrame.initBuffer((byte) 0);
- interiorMaxBytes = (int) (interiorFrame.getBuffer().capacity() * fillFactor);
-
- leafFrame.setPage(leafFrontier.page);
- leafFrame.initBuffer((byte) 0);
- leafMaxBytes = (int) (leafFrame.getBuffer().capacity() * fillFactor);
- slotSize = leafFrame.getSlotSize();
-
- nodeFrontiers.add(leafFrontier);
- pagesToWrite = new ArrayList<>();
- compressedPageWriter = bufferCache.getCompressedPageWriter(fileId);
- }
-
- protected void handleException() {
- // Unlatch and unpin pages that weren't in the queue to avoid leaking memory.
- compressedPageWriter.abort();
- for (NodeFrontier nodeFrontier : nodeFrontiers) {
- if (nodeFrontier != null && nodeFrontier.page != null) {
- ICachedPage frontierPage = nodeFrontier.page;
- if (frontierPage.confiscated()) {
- bufferCache.returnPage(frontierPage, false);
- }
- }
- }
- for (ICachedPage pageToDiscard : pagesToWrite) {
- if (pageToDiscard != null) {
- bufferCache.returnPage(pageToDiscard, false);
- }
- }
- releasedLatches = true;
- }
-
- @Override
- public void end() throws HyracksDataException {
- if (hasFailed()) {
- throw HyracksDataException.create(getFailure());
- }
- freePageManager.setRootPageId(rootPage);
- }
-
- protected void addLevel() throws HyracksDataException {
- NodeFrontier frontier = new NodeFrontier(tupleWriter.createTupleReference());
- frontier.page = bufferCache.confiscatePage(IBufferCache.INVALID_DPID);
- frontier.pageId = -1;
- frontier.lastTuple.setFieldCount(cmp.getKeyFieldCount());
- interiorFrame.setPage(frontier.page);
- interiorFrame.initBuffer((byte) nodeFrontiers.size());
- nodeFrontiers.add(frontier);
- }
-
- public ITreeIndexFrame getLeafFrame() {
- return leafFrame;
- }
-
- public void setLeafFrame(ITreeIndexFrame leafFrame) {
- this.leafFrame = leafFrame;
- }
-
- public void write(ICachedPage cPage) throws HyracksDataException {
- compressedPageWriter.prepareWrite(cPage);
- pageWriter.write(cPage);
- }
-
- @Override
- public void force() throws HyracksDataException {
- bufferCache.force(fileId, false);
- }
-
- }
-
public IBinaryComparatorFactory[] getCmpFactories() {
return cmpFactories;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndexBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndexBulkLoader.java
new file mode 100644
index 0000000..45a88a7
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndexBulkLoader.java
@@ -0,0 +1,171 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.common.impls;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hyracks.api.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.common.api.IPageManager;
+import org.apache.hyracks.storage.am.common.api.ITreeIndex;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexMetadataFrame;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleReference;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
+import org.apache.hyracks.storage.common.IIndexBulkLoader;
+import org.apache.hyracks.storage.common.MultiComparator;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.storage.common.buffercache.ICachedPage;
+import org.apache.hyracks.storage.common.buffercache.IFIFOPageWriter;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
+import org.apache.hyracks.storage.common.buffercache.PageWriteFailureCallback;
+import org.apache.hyracks.storage.common.compression.file.ICompressedPageWriter;
+import org.apache.hyracks.storage.common.file.BufferedFileHandle;
+
+public abstract class AbstractTreeIndexBulkLoader extends PageWriteFailureCallback implements IIndexBulkLoader {
+ protected final IBufferCache bufferCache;
+ protected final IPageManager freePageManager;
+ protected final AbstractTreeIndex treeIndex;
+ protected final int fileId;
+ protected final MultiComparator cmp;
+ protected final int slotSize;
+ protected final int leafMaxBytes;
+ protected final int interiorMaxBytes;
+ protected final ArrayList<NodeFrontier> nodeFrontiers = new ArrayList<>();
+ protected final ITreeIndexMetadataFrame metaFrame;
+ protected final ITreeIndexTupleWriter tupleWriter;
+ protected ITreeIndexFrame leafFrame;
+ protected ITreeIndexFrame interiorFrame;
+ // Immutable bulk loaders write their root page at page -2, as needed e.g. by append-only file systems such as
+ // HDFS. Since loading this tree relies on the root page actually being at that point, no further inserts into
+ // that tree are allowed. Currently, this is not enforced.
+ protected boolean releasedLatches;
+ private final IFIFOPageWriter pageWriter;
+ protected List<ICachedPage> pagesToWrite;
+ private final ICompressedPageWriter compressedPageWriter;
+
+ protected AbstractTreeIndexBulkLoader(float fillFactor, IPageWriteCallback callback, ITreeIndex index)
+ throws HyracksDataException {
+ this(fillFactor, callback, index, index.getLeafFrameFactory().createFrame());
+ }
+
+ protected AbstractTreeIndexBulkLoader(float fillFactor, IPageWriteCallback callback, ITreeIndex index,
+ ITreeIndexFrame leafFrame) throws HyracksDataException {
+ this.bufferCache = index.getBufferCache();
+ this.freePageManager = index.getPageManager();
+ this.fileId = index.getFileId();
+ this.treeIndex = (AbstractTreeIndex) index;
+ this.leafFrame = leafFrame;
+ interiorFrame = treeIndex.getInteriorFrameFactory().createFrame();
+ metaFrame = freePageManager.createMetadataFrame();
+
+ pageWriter = bufferCache.createFIFOWriter(callback, this);
+
+ if (!treeIndex.isEmptyTree(leafFrame)) {
+ throw HyracksDataException.create(ErrorCode.CANNOT_BULK_LOAD_NON_EMPTY_TREE);
+ }
+
+ this.cmp = MultiComparator.create(treeIndex.getCmpFactories());
+
+ leafFrame.setMultiComparator(cmp);
+ interiorFrame.setMultiComparator(cmp);
+
+ tupleWriter = leafFrame.getTupleWriter();
+ NodeFrontier leafFrontier = new NodeFrontier(createTupleReference());
+ leafFrontier.pageId = freePageManager.takePage(metaFrame);
+ leafFrontier.page = bufferCache.confiscatePage(BufferedFileHandle.getDiskPageId(fileId, leafFrontier.pageId));
+
+ interiorFrame.setPage(leafFrontier.page);
+ interiorFrame.initBuffer((byte) 0);
+ interiorMaxBytes = (int) (interiorFrame.getBuffer().capacity() * fillFactor);
+
+ leafFrame.setPage(leafFrontier.page);
+ leafFrame.initBuffer((byte) 0);
+ leafMaxBytes = (int) (leafFrame.getBuffer().capacity() * fillFactor);
+ slotSize = leafFrame.getSlotSize();
+
+ nodeFrontiers.add(leafFrontier);
+ pagesToWrite = new ArrayList<>();
+ compressedPageWriter = bufferCache.getCompressedPageWriter(fileId);
+ }
+
+ protected ITreeIndexTupleReference createTupleReference() {
+ return leafFrame.createTupleReference();
+ }
+
+ protected void handleException() {
+ // Unlatch and unpin pages that weren't in the queue to avoid leaking memory.
+ compressedPageWriter.abort();
+ for (NodeFrontier nodeFrontier : nodeFrontiers) {
+ if (nodeFrontier != null && nodeFrontier.page != null) {
+ ICachedPage frontierPage = nodeFrontier.page;
+ if (frontierPage.confiscated()) {
+ bufferCache.returnPage(frontierPage, false);
+ }
+ }
+ }
+ for (ICachedPage pageToDiscard : pagesToWrite) {
+ if (pageToDiscard != null) {
+ bufferCache.returnPage(pageToDiscard, false);
+ }
+ }
+ releasedLatches = true;
+ }
+
+ @Override
+ public void end() throws HyracksDataException {
+ if (hasFailed()) {
+ throw HyracksDataException.create(getFailure());
+ }
+ freePageManager.setRootPageId(treeIndex.getRootPageId());
+ }
+
+ protected void setRootPageId(int rootPage) {
+ treeIndex.rootPage = rootPage;
+ }
+
+ protected void addLevel() throws HyracksDataException {
+ NodeFrontier frontier = new NodeFrontier(tupleWriter.createTupleReference());
+ frontier.page = bufferCache.confiscatePage(IBufferCache.INVALID_DPID);
+ frontier.pageId = -1;
+ frontier.lastTuple.setFieldCount(cmp.getKeyFieldCount());
+ interiorFrame.setPage(frontier.page);
+ interiorFrame.initBuffer((byte) nodeFrontiers.size());
+ nodeFrontiers.add(frontier);
+ }
+
+ public ITreeIndexFrame getLeafFrame() {
+ return leafFrame;
+ }
+
+ public void setLeafFrame(ITreeIndexFrame leafFrame) {
+ this.leafFrame = leafFrame;
+ }
+
+ public void write(ICachedPage cPage) throws HyracksDataException {
+ compressedPageWriter.prepareWrite(cPage);
+ pageWriter.write(cPage);
+ }
+
+ @Override
+ public void force() throws HyracksDataException {
+ bufferCache.force(fileId, false);
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/DefaultTupleProjector.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/DefaultTupleProjector.java
new file mode 100644
index 0000000..c63912b
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/DefaultTupleProjector.java
@@ -0,0 +1,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.hyracks.storage.am.common.impls;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.common.projection.ITupleProjector;
+
+class DefaultTupleProjector implements ITupleProjector {
+ public static final ITupleProjector INSTANCE = new DefaultTupleProjector();
+
+ private DefaultTupleProjector() {
+ }
+
+ @Override
+ public ITupleReference project(ITupleReference tuple, DataOutput dos, ArrayTupleBuilder tb) throws IOException {
+ for (int i = 0; i < tuple.getFieldCount(); i++) {
+ dos.write(tuple.getFieldData(i), tuple.getFieldStart(i), tuple.getFieldLength(i));
+ tb.addFieldEndOffset();
+ }
+ return tuple;
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/DefaultTupleProjectorFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/DefaultTupleProjectorFactory.java
new file mode 100644
index 0000000..092982d
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/DefaultTupleProjectorFactory.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.common.impls;
+
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.common.projection.ITupleProjector;
+import org.apache.hyracks.storage.common.projection.ITupleProjectorFactory;
+
+public class DefaultTupleProjectorFactory implements ITupleProjectorFactory {
+ private static final long serialVersionUID = -4525893018744087821L;
+ public static final DefaultTupleProjectorFactory INSTANCE = new DefaultTupleProjectorFactory();
+
+ private DefaultTupleProjectorFactory() {
+ }
+
+ @Override
+ public ITupleProjector createTupleProjector(IHyracksTaskContext context) throws HyracksDataException {
+ return DefaultTupleProjector.INSTANCE;
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/IndexAccessParameters.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/IndexAccessParameters.java
index 11d3cd5..63902be 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/IndexAccessParameters.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/IndexAccessParameters.java
@@ -59,6 +59,11 @@
return paramMap;
}
+ @Override
+ public <T> T getParameter(String key, Class<T> clazz) {
+ return paramMap != null && clazz.isInstance(paramMap.get(key)) ? clazz.cast(paramMap.get(key)) : null;
+ }
+
public static IIndexAccessParameters createNoOpParams(IIndexCursorStats stats) {
if (stats == NoOpIndexCursorStats.INSTANCE) {
return NoOpIndexAccessParameters.INSTANCE;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/NoOpIndexAccessParameters.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/NoOpIndexAccessParameters.java
index fa7811c..76a1930 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/NoOpIndexAccessParameters.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/NoOpIndexAccessParameters.java
@@ -47,4 +47,9 @@
public Map<String, Object> getParameters() {
return paramMap;
}
+
+ @Override
+ public <T> T getParameter(String key, Class<T> clazz) {
+ return null;
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/pom.xml b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/pom.xml
new file mode 100644
index 0000000..cadc714
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/pom.xml
@@ -0,0 +1,95 @@
+<!--
+ ! Licensed to the Apache Software Foundation (ASF) under one
+ ! or more contributor license agreements. See the NOTICE file
+ ! distributed with this work for additional information
+ ! regarding copyright ownership. The ASF licenses this file
+ ! to you under the Apache License, Version 2.0 (the
+ ! "License"); you may not use this file except in compliance
+ ! with the License. You may obtain a copy of the License at
+ !
+ ! http://www.apache.org/licenses/LICENSE-2.0
+ !
+ ! Unless required by applicable law or agreed to in writing,
+ ! software distributed under the License is distributed on an
+ ! "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ ! KIND, either express or implied. See the License for the
+ ! specific language governing permissions and limitations
+ ! under the License.
+ !-->
+<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>hyracks-storage-am-lsm-btree-column</artifactId>
+ <parent>
+ <groupId>org.apache.hyracks</groupId>
+ <artifactId>hyracks</artifactId>
+ <version>0.3.9-SNAPSHOT</version>
+ </parent>
+ <licenses>
+ <license>
+ <name>Apache License, Version 2.0</name>
+ <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
+ <distribution>repo</distribution>
+ <comments>A business-friendly OSS license</comments>
+ </license>
+ </licenses>
+ <properties>
+ <root.dir>${basedir}/../..</root.dir>
+ </properties>
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.hyracks</groupId>
+ <artifactId>hyracks-storage-am-btree</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hyracks</groupId>
+ <artifactId>hyracks-storage-am-lsm-btree</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hyracks</groupId>
+ <artifactId>hyracks-storage-am-bloomfilter</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hyracks</groupId>
+ <artifactId>hyracks-storage-am-lsm-common</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hyracks</groupId>
+ <artifactId>hyracks-dataflow-common</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hyracks</groupId>
+ <artifactId>hyracks-storage-am-common</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hyracks</groupId>
+ <artifactId>hyracks-api</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hyracks</groupId>
+ <artifactId>hyracks-storage-common</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hyracks</groupId>
+ <artifactId>hyracks-data-std</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hyracks</groupId>
+ <artifactId>hyracks-util</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.logging.log4j</groupId>
+ <artifactId>log4j-api</artifactId>
+ </dependency>
+ </dependencies>
+</project>
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/AbstractColumnTupleReader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/AbstractColumnTupleReader.java
new file mode 100644
index 0000000..7db792b
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/AbstractColumnTupleReader.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.hyracks.storage.am.lsm.btree.column.api;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.lsm.btree.column.impls.btree.AbstractColumnBTreeLeafFrame;
+import org.apache.hyracks.storage.am.lsm.btree.column.impls.btree.ColumnBTreeReadLeafFrame;
+
+/**
+ * Provided for columnar read tuple reference
+ */
+public abstract class AbstractColumnTupleReader extends AbstractTupleWriterDisabledMethods {
+ public abstract IColumnTupleIterator createTupleIterator(ColumnBTreeReadLeafFrame frame, int componentIndex,
+ IColumnReadMultiPageOp multiPageOp);
+
+ /**
+ * Currently fixed to 4-byte per offset
+ *
+ * @param buf buffer of Page0
+ * @param columnIndex column index
+ * @return column offset
+ * @see AbstractColumnTupleWriter#getColumnOffsetsSize()
+ */
+ public final int getColumnOffset(ByteBuffer buf, int columnIndex) {
+ return buf.getInt(AbstractColumnBTreeLeafFrame.HEADER_SIZE + columnIndex * Integer.BYTES);
+ }
+
+ @Override
+ public final int bytesRequired(ITupleReference tuple) {
+ throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/AbstractColumnTupleReaderWriterFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/AbstractColumnTupleReaderWriterFactory.java
new file mode 100644
index 0000000..774bbb9
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/AbstractColumnTupleReaderWriterFactory.java
@@ -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.
+ */
+package org.apache.hyracks.storage.am.lsm.btree.column.api;
+
+import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleWriterFactory;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.projection.IColumnProjectionInfo;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+
+/**
+ * For columns, there are two types for {@link ITreeIndexTupleWriter} one used during write and another during read
+ */
+public abstract class AbstractColumnTupleReaderWriterFactory implements ITreeIndexTupleWriterFactory {
+ private static final long serialVersionUID = -2377235465942457248L;
+ protected final int pageSize;
+ protected final int maxNumberOfTuples;
+ protected final float tolerance;
+
+ /**
+ * Tuple reader/writer factory
+ *
+ * @param pageSize {@link IBufferCache} page size
+ * @param maxNumberOfTuples maximum number of tuples stored per a mega leaf page
+ * @param tolerance percentage of tolerated empty space
+ */
+ protected AbstractColumnTupleReaderWriterFactory(int pageSize, int maxNumberOfTuples, float tolerance) {
+ this.pageSize = pageSize;
+ this.maxNumberOfTuples = maxNumberOfTuples;
+ this.tolerance = tolerance;
+ }
+
+ /**
+ * Create columnar tuple writer
+ *
+ * @param columnMetadata writer column metadata
+ */
+ public abstract AbstractColumnTupleWriter createColumnWriter(IColumnMetadata columnMetadata);
+
+ /**
+ * Create columnar tuple reader
+ *
+ * @param columnProjectionInfo column projection info for either query or merge
+ */
+ public abstract AbstractColumnTupleReader createColumnReader(IColumnProjectionInfo columnProjectionInfo);
+
+ @Override
+ public final ITreeIndexTupleWriter createTupleWriter() {
+ throw new UnsupportedOperationException("Operation is not supported for " + getClass().getName());
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/AbstractColumnTupleWriter.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/AbstractColumnTupleWriter.java
new file mode 100644
index 0000000..0c19ce7
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/AbstractColumnTupleWriter.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.btree.column.api;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+/**
+ * Columnar Tuple Writer:
+ * <p>
+ * The writer does not write directly to the page(s) buffer but write to internal temporary buffers (provided by
+ * {@link IColumnWriteMultiPageOp} until the header page (or page0) is full or the number of tuples equals to the
+ * {@link #getMaxNumberOfTuples()}
+ * Then, the columns are flushed to disk.
+ * <p>
+ * Contract:
+ * - Initially, the writer has to set multiPageOp by calling {@link #init(IColumnWriteMultiPageOp)}
+ * - For each write, the caller should check if adding a tuple does not exceed the {@link #getMaxNumberOfTuples()} or
+ * the on-disk page size (called stopping condition)
+ * - If the stopping condition is reached, then {@link #flush(ByteBuffer)} needed to be called
+ * <p>
+ * Hyracks visibility:
+ * - Columns are written as blobs (i.e., not interpretable by Hyracks)
+ * - Hyracks only aware of where each column at
+ */
+public abstract class AbstractColumnTupleWriter extends AbstractTupleWriterDisabledMethods {
+ /**
+ * Set the writer with {@link IColumnWriteMultiPageOp} to allocate columns for their writers
+ *
+ * @param multiPageOp multiPageOp
+ */
+ public abstract void init(IColumnWriteMultiPageOp multiPageOp) throws HyracksDataException;
+
+ /**
+ * @return The current number of columns
+ */
+ public abstract int getNumberOfColumns();
+
+ /**
+ * Currently, a column offset takes 4-byte (fixed). But in the future, we can reformat the offsets. For example,
+ * we can store index-offset pairs if we encounter a sparse columns (i.e., most columns are just nulls).
+ *
+ * @return the size needed to store columns' offsets
+ */
+ public final int getColumnOffsetsSize() {
+ return Integer.BYTES * getNumberOfColumns();
+ }
+
+ /**
+ * @return maximum number of tuples to be stored per page (i.e., page0)
+ */
+ public abstract int getMaxNumberOfTuples();
+
+ /**
+ * @return page0 occupied space
+ */
+ public abstract int getOccupiedSpace();
+
+ /**
+ * Writes the tuple into a temporary internal buffers
+ *
+ * @param tuple The tuple to be written
+ */
+ public abstract void writeTuple(ITupleReference tuple) throws HyracksDataException;
+
+ /**
+ * Flush all columns from the internal buffers to the page buffer
+ *
+ * @return the allocated space used to write tuples
+ */
+ public abstract int flush(ByteBuffer pageZero) throws HyracksDataException;
+
+ /**
+ * Close the current writer and release all allocated temporary buffers
+ */
+ public abstract void close();
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/AbstractTupleWriterDisabledMethods.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/AbstractTupleWriterDisabledMethods.java
new file mode 100644
index 0000000..abc5ef0
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/AbstractTupleWriterDisabledMethods.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.btree.column.api;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleReference;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
+
+/**
+ * Disable all row write methods
+ */
+public abstract class AbstractTupleWriterDisabledMethods implements ITreeIndexTupleWriter {
+ protected static final String UNSUPPORTED_OPERATION_MSG = "Operation is not supported for columnar tuple reader";
+
+ /* ***********************************************
+ * Disable write-related operations
+ * ***********************************************
+ */
+
+ @Override
+ public final ITreeIndexTupleReference createTupleReference() {
+ throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+ }
+
+ @Override
+ public final int writeTuple(ITupleReference tuple, ByteBuffer targetBuf, int targetOff) {
+ throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+ }
+
+ @Override
+ public final int writeTuple(ITupleReference tuple, byte[] targetBuf, int targetOff) {
+ throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+ }
+
+ @Override
+ public final int writeTupleFields(ITupleReference tuple, int startField, int numFields, byte[] targetBuf,
+ int targetOff) {
+ throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+ }
+
+ @Override
+ public final int bytesRequired(ITupleReference tuple, int startField, int numFields) {
+ throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+ }
+
+ @Override
+ public final int getCopySpaceRequired(ITupleReference tuple) {
+ throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+ }
+
+ @Override
+ public final void setUpdated(boolean isUpdated) {
+ throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+ }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnBufferProvider.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnBufferProvider.java
new file mode 100644
index 0000000..d0b5e12
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnBufferProvider.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.lsm.btree.column.api;
+
+import java.nio.ByteBuffer;
+import java.util.Queue;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.btree.column.impls.btree.ColumnBTreeReadLeafFrame;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+
+/**
+ * A proxy to call {@link IBufferCache} operations. Each column should have its own buffer provider
+ */
+public interface IColumnBufferProvider {
+ /**
+ * Calling this method would pin all the pages of the requested columns from the buffer cache
+ *
+ * @param frame the frame for Page0
+ */
+ void reset(ColumnBTreeReadLeafFrame frame) throws HyracksDataException;
+
+ /**
+ * Return all the pages for a column
+ *
+ * @param buffers queue for all pages of a column
+ */
+ void readAll(Queue<ByteBuffer> buffers) throws HyracksDataException;
+
+ /**
+ * Release all the column pages (i.e., unpin all column pages)
+ */
+ void releaseAll() throws HyracksDataException;
+
+ /**
+ * @return a buffer of a column (in case there is only a single page for a column)
+ */
+ ByteBuffer getBuffer();
+
+ /**
+ * @return the actual length (in bytes) for all the column's pages
+ */
+ int getLength();
+
+ /**
+ * @return the column index
+ */
+ int getColumnIndex();
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnManager.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnManager.java
new file mode 100644
index 0000000..278ea03
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnManager.java
@@ -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.
+ */
+package org.apache.hyracks.storage.am.lsm.btree.column.api;
+
+import java.util.List;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.projection.IColumnTupleProjector;
+
+public interface IColumnManager {
+ /**
+ * Activate the columnar manager for an empty dataset
+ *
+ * @return empty column metadata
+ */
+ IColumnMetadata activate() throws HyracksDataException;
+
+ /**
+ * Activate the column manager for a non-empty dataset
+ *
+ * @param metadata column metadata value from the latest component metadata
+ * @return latest column metadata
+ */
+ IColumnMetadata activate(IValueReference metadata) throws HyracksDataException;
+
+ /**
+ * Create merge column metadata for a newly created merge component
+ *
+ * @param metadata latest column metadata value stored in the metadata page
+ * @param componentsTuples tuples of the merging components
+ * @return column metadata for a new merged component
+ */
+ IColumnMetadata createMergeColumnMetadata(IValueReference metadata, List<IColumnTupleIterator> componentsTuples)
+ throws HyracksDataException;
+
+ /**
+ * Create tuple projector for reading the merging components. The merge tuple projector will return all columns
+ *
+ * @return merge tuple projector
+ */
+ IColumnTupleProjector getMergeColumnProjector();
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnManagerFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnManagerFactory.java
new file mode 100644
index 0000000..a2dfbcf
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnManagerFactory.java
@@ -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.
+ */
+package org.apache.hyracks.storage.am.lsm.btree.column.api;
+
+import java.io.Serializable;
+
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
+
+public interface IColumnManagerFactory extends Serializable, IJsonSerializable {
+ /**
+ * @return a new instance of {@link IColumnManager}
+ */
+ IColumnManager createColumnManager();
+
+ /**
+ * Get column tuple reader/writer for the {@link LSMIOOperationType#LOAD}
+ */
+ AbstractColumnTupleReaderWriterFactory getLoadColumnTupleReaderWriterFactory();
+
+ /**
+ * Get column tuple reader/writer for the {@link LSMIOOperationType#FLUSH}
+ */
+ AbstractColumnTupleReaderWriterFactory getFlushColumnTupleReaderWriterFactory();
+
+ /**
+ * Get column tuple reader/writer for the {@link LSMIOOperationType#MERGE}
+ */
+ AbstractColumnTupleReaderWriterFactory createMergeColumnTupleReaderWriterFactory();
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnMetadata.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnMetadata.java
new file mode 100644
index 0000000..4c23b97
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnMetadata.java
@@ -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.
+ */
+package org.apache.hyracks.storage.am.lsm.btree.column.api;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.util.annotations.NotThreadSafe;
+
+/**
+ * A holder for the columnar metadata.
+ * Modifications on the columnar metadata are not thread safe.
+ */
+@NotThreadSafe
+public interface IColumnMetadata {
+ /**
+ * @return a serialized version of the columns metadata
+ */
+ IValueReference serializeColumnsMetadata() throws HyracksDataException;
+
+ /**
+ * abort in case of an error. This should clean up any artifact
+ */
+ void abort() throws HyracksDataException;
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnReadMultiPageOp.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnReadMultiPageOp.java
new file mode 100644
index 0000000..f43a6e9
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnReadMultiPageOp.java
@@ -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.
+ */
+package org.apache.hyracks.storage.am.lsm.btree.column.api;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.storage.common.buffercache.ICachedPage;
+
+/**
+ * A proxy to call {@link IBufferCache} read columns' pages
+ * Implementer should be aware to unpin all pages in case of an error
+ */
+public interface IColumnReadMultiPageOp {
+ /**
+ * Pin a column page
+ *
+ * @return a page that belongs to a column
+ */
+ ICachedPage pin(int pageId) throws HyracksDataException;
+
+ /**
+ * Unpin a pinned column page
+ */
+ void unpin(ICachedPage page) throws HyracksDataException;
+
+ /**
+ * Return {@link IBufferCache} page size
+ *
+ * @see IBufferCache#getPageSize()
+ */
+ int getPageSize();
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnTupleIterator.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnTupleIterator.java
new file mode 100644
index 0000000..0c95500
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnTupleIterator.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.hyracks.storage.am.lsm.btree.column.api;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.common.api.ILSMIndexCursor;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMTreeTupleReference;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.storage.common.buffercache.ICachedPage;
+
+/**
+ * A tuple representation that combines all the columns. It simply provides a way to iterate over tuples for a given
+ * set that could span multiple pages.
+ */
+public interface IColumnTupleIterator extends ILSMTreeTupleReference, Comparable<IColumnTupleIterator> {
+ /**
+ * Reset the iterator starting at the provided index
+ *
+ * @param startIndex start from the tuple at this index
+ */
+ void reset(int startIndex) throws HyracksDataException;
+
+ /**
+ * Mark {@link IColumnTupleIterator} as consumed
+ */
+ void consume();
+
+ /**
+ * @return true if the {@link IColumnTupleIterator} is consumed, false otherwise
+ */
+ boolean isConsumed();
+
+ /**
+ * Skip a number of tuples
+ *
+ * @param count the number of tuples that needed to be skipped
+ */
+ void skip(int count) throws HyracksDataException;
+
+ /**
+ * Move to the next tuple
+ */
+ void next() throws HyracksDataException;
+
+ /**
+ * Notifies that the last tuple has been consumed
+ */
+ void lastTupleReached() throws HyracksDataException;
+
+ /**
+ * The component index is the same as the index of a component in an {@link ILSMIndexCursor}
+ *
+ * @return From which {@link ILSMComponent} this iterator is for
+ */
+ int getComponentIndex();
+
+ /**
+ * Calls {@link IBufferCache#unpin(ICachedPage)} for all columns' pages
+ */
+ void unpinColumnsPages() throws HyracksDataException;
+
+ void close();
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnWriteMultiPageOp.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnWriteMultiPageOp.java
new file mode 100644
index 0000000..2309fe1
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnWriteMultiPageOp.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.hyracks.storage.am.lsm.btree.column.api;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+
+/**
+ * A proxy to call {@link IBufferCache} writing methods
+ * <p>
+ * An instance of this interface is responsible for returning all confiscated pages back to {@link IBufferCache} upon
+ * failures. Temporary buffers should be returned to the {@link IBufferCache} once the multi-page operation is finished.
+ * <p>
+ * Users of an instance of this interface should not expect the temporary buffers will last after the multi-page
+ * operation is finished.
+ */
+public interface IColumnWriteMultiPageOp {
+ /**
+ * @return a buffer that correspond to a page in a file
+ */
+ ByteBuffer confiscatePersistent() throws HyracksDataException;
+
+ /**
+ * Persist all confiscated persistent buffers to disk
+ */
+ void persist() throws HyracksDataException;
+
+ /**
+ * @return the number confiscated persistent pages
+ */
+ int getNumberOfPersistentBuffers();
+
+ /**
+ * @return a {@link IBufferCache}-backed buffer for temporary use
+ */
+ ByteBuffer confiscateTemporary() throws HyracksDataException;
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/projection/IColumnProjectionInfo.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/projection/IColumnProjectionInfo.java
new file mode 100644
index 0000000..1506433
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/projection/IColumnProjectionInfo.java
@@ -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.
+ */
+package org.apache.hyracks.storage.am.lsm.btree.column.api.projection;
+
+/**
+ * Gets information about the requested columns
+ */
+public interface IColumnProjectionInfo {
+ /**
+ * @param ordinal position of the requested column
+ * @return column index given the ordinal number of the requested column
+ */
+ int getColumnIndex(int ordinal);
+
+ /**
+ * @return total number of requested columns
+ */
+ int getNumberOfProjectedColumns();
+
+ /**
+ * @return number of primary keys
+ */
+ int getNumberOfPrimaryKeys();
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/projection/IColumnTupleProjector.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/projection/IColumnTupleProjector.java
new file mode 100644
index 0000000..c1301da
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/projection/IColumnTupleProjector.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.lsm.btree.column.api.projection;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.storage.am.lsm.common.api.IComponentMetadata;
+import org.apache.hyracks.storage.common.projection.ITupleProjector;
+
+/**
+ * A specialized {@link ITupleProjector} for columnar datasets
+ */
+public interface IColumnTupleProjector extends ITupleProjector {
+ /**
+ * Create projection information
+ *
+ * @param columnMetadata the latest component's raw column metadata as stored in {@link IComponentMetadata}
+ * @return projection information
+ */
+ IColumnProjectionInfo createProjectionInfo(IValueReference columnMetadata) throws HyracksDataException;
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/AbstractColumnBTreeLeafFrame.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/AbstractColumnBTreeLeafFrame.java
new file mode 100644
index 0000000..9aeafa4
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/AbstractColumnBTreeLeafFrame.java
@@ -0,0 +1,267 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.btree.column.impls.btree;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.common.api.ISlotManager;
+import org.apache.hyracks.storage.am.common.api.ISplitKey;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleReference;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
+import org.apache.hyracks.storage.am.common.frames.FrameOpSpaceStatus;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.AbstractColumnTupleWriter;
+import org.apache.hyracks.storage.common.MultiComparator;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.storage.common.buffercache.ICachedPage;
+import org.apache.hyracks.storage.common.buffercache.IExtraPageBlockHelper;
+
+/**
+ * Disable all unsupported/unused operations
+ */
+public abstract class AbstractColumnBTreeLeafFrame implements ITreeIndexFrame {
+ private static final String UNSUPPORTED_OPERATION_MSG = "Operation is not supported";
+
+ /*
+ * Remap the BTreeNSMFrame pointers for columnar pages
+ */
+ //Same as before
+ public static final int TUPLE_COUNT_OFFSET = Constants.TUPLE_COUNT_OFFSET;
+ //Previously Renaming
+ public static final int NUMBER_OF_COLUMNS_OFFSET = Constants.FREE_SPACE_OFFSET;
+ //Previously first four byte of LSN.
+ public static final int LEFT_MOST_KEY_OFFSET = Constants.RESERVED_HEADER_SIZE;
+ //Previously last four byte of LSN.
+ public static final int RIGHT_MOST_KEY_OFFSET = LEFT_MOST_KEY_OFFSET + 4;
+ /**
+ * Currently, a column offset takes 4-byte (fixed). But in the future, we can reformat the offsets. For example,
+ * we can store index-offset pairs if we encounter a sparse columns (i.e., most columns are just nulls). This
+ * reformatting could be indicated by the FLAG byte.
+ *
+ * @see AbstractColumnTupleWriter#getColumnOffsetsSize()
+ */
+ public static final int SIZE_OF_COLUMNS_OFFSETS_OFFSET = RIGHT_MOST_KEY_OFFSET + 4;
+ //Total number of columns pages
+ public static final int NUMBER_OF_COLUMN_PAGES = SIZE_OF_COLUMNS_OFFSETS_OFFSET + 4;
+ //A flag (used in NSM to indicate small and large pages). We can reuse it as explained above
+ public static final int FLAG_OFFSET = NUMBER_OF_COLUMN_PAGES + 4;
+ public static final int NEXT_LEAF_OFFSET = FLAG_OFFSET + 1;
+ public static final int HEADER_SIZE = NEXT_LEAF_OFFSET + 4;
+
+ protected final ITreeIndexTupleWriter rowTupleWriter;
+
+ protected MultiComparator cmp;
+ protected ICachedPage page;
+ protected ByteBuffer buf;
+
+ AbstractColumnBTreeLeafFrame(ITreeIndexTupleWriter rowTupleWriter) {
+ this.rowTupleWriter = rowTupleWriter;
+ }
+
+ /* ****************************************************************************
+ * Needed by both read and write
+ * ****************************************************************************
+ */
+
+ @Override
+ public final ITreeIndexTupleWriter getTupleWriter() {
+ return rowTupleWriter;
+ }
+
+ @Override
+ public final void setMultiComparator(MultiComparator cmp) {
+ this.cmp = cmp;
+ }
+
+ @Override
+ public final void setPage(ICachedPage page) {
+ this.page = page;
+ this.buf = page.getBuffer();
+ buf.clear();
+ buf.position(HEADER_SIZE);
+ }
+
+ @Override
+ public final ICachedPage getPage() {
+ return page;
+ }
+
+ @Override
+ public final ByteBuffer getBuffer() {
+ return buf;
+ }
+
+ @Override
+ public final boolean isLeaf() {
+ return true;
+ }
+
+ @Override
+ public final boolean isInterior() {
+ return false;
+ }
+
+ @Override
+ public final int getPageHeaderSize() {
+ return HEADER_SIZE;
+ }
+
+ /* ****************************************************************************
+ * Operations that are needed by either read or write
+ * ****************************************************************************
+ */
+
+ @Override
+ public void initBuffer(byte level) {
+ throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+ }
+
+ @Override
+ public int getTupleCount() {
+ throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+ }
+
+ @Override
+ public ITreeIndexTupleReference createTupleReference() {
+ throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+ }
+
+ @Override
+ public ITupleReference getLeftmostTuple() throws HyracksDataException {
+ throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+ }
+
+ @Override
+ public ITupleReference getRightmostTuple() throws HyracksDataException {
+ throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+ }
+
+ /* ****************************************************************************
+ * Unsupported Operations
+ * ****************************************************************************
+ */
+
+ @Override
+ public final String printHeader() {
+ throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+ }
+
+ @Override
+ public final byte getLevel() {
+ throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+ }
+
+ @Override
+ public final void setLevel(byte level) {
+ throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+ }
+
+ @Override
+ public final int getBytesRequiredToWriteTuple(ITupleReference tuple) {
+ throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+ }
+
+ @Override
+ public final FrameOpSpaceStatus hasSpaceInsert(ITupleReference tuple) throws HyracksDataException {
+ throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+ }
+
+ @Override
+ public final void insert(ITupleReference tuple, int tupleIndex) {
+ throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+ }
+
+ @Override
+ public final FrameOpSpaceStatus hasSpaceUpdate(ITupleReference newTuple, int oldTupleIndex) {
+ throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+ }
+
+ @Override
+ public final void update(ITupleReference newTuple, int oldTupleIndex, boolean inPlace) {
+ throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+ }
+
+ @Override
+ public final void delete(ITupleReference tuple, int tupleIndex) {
+ throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+ }
+
+ @Override
+ public final boolean compact() {
+ throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+ }
+
+ @Override
+ public final boolean compress() throws HyracksDataException {
+ throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+ }
+
+ @Override
+ public final int getTupleOffset(int slotNum) {
+ throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+ }
+
+ @Override
+ public final int getTotalFreeSpace() {
+ throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+ }
+
+ @Override
+ public final void setPageLsn(long pageLsn) {
+ throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+ }
+
+ @Override
+ public final long getPageLsn() {
+ throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+ }
+
+ @Override
+ public final int getMaxTupleSize(int pageSize) {
+ throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+ }
+
+ @Override
+ public final void split(ITreeIndexFrame rightFrame, ITupleReference tuple, ISplitKey splitKey,
+ IExtraPageBlockHelper extraPageBlockHelper, IBufferCache bufferCache) throws HyracksDataException {
+ throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+ }
+
+ @Override
+ public final ISlotManager getSlotManager() {
+ throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+ }
+
+ @Override
+ public final int getSlotSize() {
+ return 0;
+ }
+
+ @Override
+ public final int getFreeSpaceOff() {
+ throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+ }
+
+ @Override
+ public final void setFreeSpaceOff(int freeSpace) {
+ throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTree.java
new file mode 100644
index 0000000..fcee22c
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTree.java
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.btree.column.impls.btree;
+
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.api.util.HyracksConstants;
+import org.apache.hyracks.storage.am.btree.impls.DiskBTree;
+import org.apache.hyracks.storage.am.common.api.IPageManager;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexCursor;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnMetadata;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.projection.IColumnProjectionInfo;
+import org.apache.hyracks.storage.common.IIndexAccessParameters;
+import org.apache.hyracks.storage.common.IIndexBulkLoader;
+import org.apache.hyracks.storage.common.IIndexCursorStats;
+import org.apache.hyracks.storage.common.NoOpIndexCursorStats;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
+
+public class ColumnBTree extends DiskBTree {
+ public ColumnBTree(IBufferCache bufferCache, IPageManager freePageManager,
+ ITreeIndexFrameFactory interiorFrameFactory, ITreeIndexFrameFactory leafFrameFactory,
+ IBinaryComparatorFactory[] cmpFactories, int fieldCount, FileReference file) {
+ super(bufferCache, freePageManager, interiorFrameFactory, leafFrameFactory, cmpFactories, fieldCount, file);
+ }
+
+ @Override
+ public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
+ boolean checkIfEmptyIndex, IPageWriteCallback callback) {
+ throw new IllegalAccessError("Missing write column metadata");
+ }
+
+ public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, IPageWriteCallback callback,
+ IColumnMetadata columnMetadata) throws HyracksDataException {
+ ColumnBTreeLeafFrameFactory columnLeafFrameFactory = (ColumnBTreeLeafFrameFactory) leafFrameFactory;
+ ColumnBTreeWriteLeafFrame writeLeafFrame = columnLeafFrameFactory.createWriterFrame(columnMetadata);
+ return new ColumnBTreeBulkloader(fillFactor, verifyInput, callback, this, writeLeafFrame);
+ }
+
+ @Override
+ public BTreeAccessor createAccessor(IIndexAccessParameters iap) {
+ throw new IllegalArgumentException("Use createAccessor(IIndexAccessParameters, int, IColumnTupleProjector)");
+ }
+
+ public BTreeAccessor createAccessor(IIndexAccessParameters iap, int index, IColumnProjectionInfo projectionInfo) {
+ return new ColumnBTreeAccessor(this, iap, index, projectionInfo);
+ }
+
+ public class ColumnBTreeAccessor extends DiskBTreeAccessor {
+ private final int index;
+ private final IColumnProjectionInfo projectionInfo;
+
+ public ColumnBTreeAccessor(ColumnBTree btree, IIndexAccessParameters iap, int index,
+ IColumnProjectionInfo projectionInfo) {
+ super(btree, iap);
+ this.index = index;
+ this.projectionInfo = projectionInfo;
+ }
+
+ @Override
+ public ITreeIndexCursor createSearchCursor(boolean exclusive) {
+ ColumnBTreeLeafFrameFactory columnLeafFrameFactory = (ColumnBTreeLeafFrameFactory) leafFrameFactory;
+ ColumnBTreeReadLeafFrame readLeafFrame = columnLeafFrameFactory.createReadFrame(projectionInfo);
+ return new ColumnBTreeRangeSearchCursor(readLeafFrame, (IIndexCursorStats) iap.getParameters()
+ .getOrDefault(HyracksConstants.INDEX_CURSOR_STATS, NoOpIndexCursorStats.INSTANCE), index);
+ }
+
+ @Override
+ public ITreeIndexCursor createPointCursor(boolean exclusive, boolean stateful) {
+ ColumnBTreeLeafFrameFactory columnLeafFrameFactory = (ColumnBTreeLeafFrameFactory) leafFrameFactory;
+ ColumnBTreeReadLeafFrame readLeafFrame = columnLeafFrameFactory.createReadFrame(projectionInfo);
+ return new ColumnBTreePointSearchCursor(readLeafFrame, (IIndexCursorStats) iap.getParameters()
+ .getOrDefault(HyracksConstants.INDEX_CURSOR_STATS, NoOpIndexCursorStats.INSTANCE), index);
+ }
+ }
+}
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreeBulkloader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreeBulkloader.java
new file mode 100644
index 0000000..48bd180
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreeBulkloader.java
@@ -0,0 +1,227 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.btree.column.impls.btree;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.btree.impls.BTreeNSMBulkLoader;
+import org.apache.hyracks.storage.am.btree.impls.BTreeSplitKey;
+import org.apache.hyracks.storage.am.common.api.ISplitKey;
+import org.apache.hyracks.storage.am.common.api.ITreeIndex;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleReference;
+import org.apache.hyracks.storage.am.common.impls.NodeFrontier;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.AbstractColumnTupleWriter;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnWriteMultiPageOp;
+import org.apache.hyracks.storage.common.buffercache.CachedPage;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.storage.common.buffercache.ICachedPage;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
+import org.apache.hyracks.storage.common.file.BufferedFileHandle;
+
+public final class ColumnBTreeBulkloader extends BTreeNSMBulkLoader implements IColumnWriteMultiPageOp {
+ private final List<CachedPage> columnsPages;
+ private final List<CachedPage> tempConfiscatedPages;
+ private final ColumnBTreeWriteLeafFrame columnarFrame;
+ private final AbstractColumnTupleWriter columnWriter;
+ private final ISplitKey lowKey;
+ private boolean setLowKey;
+ private int tupleCount;
+
+ public ColumnBTreeBulkloader(float fillFactor, boolean verifyInput, IPageWriteCallback callback, ITreeIndex index,
+ ITreeIndexFrame leafFrame) throws HyracksDataException {
+ super(fillFactor, verifyInput, callback, index, leafFrame);
+ columnsPages = new ArrayList<>();
+ tempConfiscatedPages = new ArrayList<>();
+ columnarFrame = (ColumnBTreeWriteLeafFrame) leafFrame;
+ columnWriter = columnarFrame.getColumnTupleWriter();
+ columnWriter.init(this);
+ lowKey = new BTreeSplitKey(tupleWriter.createTupleReference());
+ lowKey.getTuple().setFieldCount(cmp.getKeyFieldCount());
+ setLowKey = true;
+ }
+
+ @Override
+ public void add(ITupleReference tuple) throws HyracksDataException {
+ if (isFull(tuple)) {
+ writeFullLeafPage();
+ confiscateNewLeafPage();
+ }
+ //Save the key of the last inserted tuple
+ setMinMaxKeys(tuple);
+ columnWriter.writeTuple(tuple);
+ tupleCount++;
+ }
+
+ @Override
+ protected ITreeIndexTupleReference createTupleReference() {
+ return tupleWriter.createTupleReference();
+ }
+
+ private boolean isFull(ITupleReference tuple) {
+ if (tupleCount == 0) {
+ return false;
+ } else if (tupleCount >= columnWriter.getMaxNumberOfTuples()) {
+ //We reached the maximum number of tuples
+ return true;
+ }
+ int requiredFreeSpace = AbstractColumnBTreeLeafFrame.HEADER_SIZE;
+ //Columns' Offsets
+ requiredFreeSpace += columnWriter.getColumnOffsetsSize();
+ //Occupied space from previous writes
+ requiredFreeSpace += columnWriter.getOccupiedSpace();
+ //min and max tuples' sizes
+ requiredFreeSpace += lowKey.getTuple().getTupleSize() + splitKey.getTuple().getTupleSize();
+ //New tuple required space
+ requiredFreeSpace += columnWriter.bytesRequired(tuple);
+ return bufferCache.getPageSize() <= requiredFreeSpace;
+ }
+
+ private void setMinMaxKeys(ITupleReference tuple) {
+ //Set max key
+ setSplitKey(splitKey, tuple);
+ if (setLowKey) {
+ setSplitKey(lowKey, tuple);
+ lowKey.getTuple().resetByTupleOffset(lowKey.getBuffer().array(), 0);
+ setLowKey = false;
+ }
+ }
+
+ @Override
+ public void end() throws HyracksDataException {
+ if (tupleCount > 0) {
+ splitKey.getTuple().resetByTupleOffset(splitKey.getBuffer().array(), 0);
+ columnarFrame.flush(columnWriter, tupleCount, this, lowKey.getTuple(), splitKey.getTuple());
+ }
+ columnWriter.close();
+ //We are done, return any temporary confiscated pages
+ for (ICachedPage page : tempConfiscatedPages) {
+ bufferCache.returnPage(page, false);
+ }
+ tempConfiscatedPages.clear();
+ //Where Page0 and columns pages will be written
+ super.end();
+ }
+
+ @Override
+ protected void writeFullLeafPage() throws HyracksDataException {
+ NodeFrontier leafFrontier = nodeFrontiers.get(0);
+ splitKey.getTuple().resetByTupleOffset(splitKey.getBuffer().array(), 0);
+ splitKey.setLeftPage(leafFrontier.pageId);
+ if (tupleCount > 0) {
+ //We need to flush columns to confiscate all columns pages first before calling propagateBulk
+ columnarFrame.flush(columnWriter, tupleCount, this, lowKey.getTuple(), splitKey.getTuple());
+ }
+
+ propagateBulk(1, pagesToWrite);
+
+ //Take a page for the next leaf
+ leafFrontier.pageId = freePageManager.takePage(metaFrame);
+ columnarFrame.setNextLeaf(leafFrontier.pageId);
+
+ /*
+ * Write columns' pages first to ensure they (columns' pages) are written before pageZero.
+ * It ensures pageZero does not land in between columns' pages if compression is enabled
+ */
+ writeColumnsPages();
+ //Then write page0
+ write(leafFrontier.page);
+
+ //Write interior nodes after writing columns pages
+ for (ICachedPage c : pagesToWrite) {
+ write(c);
+ }
+
+ pagesToWrite.clear();
+ splitKey.setRightPage(leafFrontier.pageId);
+ setLowKey = true;
+ tupleCount = 0;
+ }
+
+ @Override
+ protected void writeLastLeaf(ICachedPage page) throws HyracksDataException {
+ /*
+ * Write columns' pages first to ensure they (columns' pages) are written before pageZero.
+ * It ensures pageZero does not land in between columns' pages if compression is enabled
+ */
+ writeColumnsPages();
+ super.writeLastLeaf(page);
+ }
+
+ private void writeColumnsPages() throws HyracksDataException {
+ for (ICachedPage c : columnsPages) {
+ write(c);
+ }
+ columnsPages.clear();
+ }
+
+ @Override
+ public void abort() throws HyracksDataException {
+ for (ICachedPage page : columnsPages) {
+ bufferCache.returnPage(page, false);
+ }
+
+ for (ICachedPage page : tempConfiscatedPages) {
+ bufferCache.returnPage(page, false);
+ }
+ super.abort();
+ }
+
+ private void setSplitKey(ISplitKey splitKey, ITupleReference tuple) {
+ int splitKeySize = tupleWriter.bytesRequired(tuple, 0, cmp.getKeyFieldCount());
+ splitKey.initData(splitKeySize);
+ tupleWriter.writeTupleFields(tuple, 0, cmp.getKeyFieldCount(), splitKey.getBuffer().array(), 0);
+ }
+
+ /*
+ * ***********************************************************
+ * IColumnWriteMultiPageOp
+ * ***********************************************************
+ */
+
+ @Override
+ public ByteBuffer confiscatePersistent() throws HyracksDataException {
+ int pageId = freePageManager.takePage(metaFrame);
+ long dpid = BufferedFileHandle.getDiskPageId(fileId, pageId);
+ CachedPage page = (CachedPage) bufferCache.confiscatePage(dpid);
+ columnsPages.add(page);
+ return page.getBuffer();
+ }
+
+ @Override
+ public void persist() throws HyracksDataException {
+ writeColumnsPages();
+ }
+
+ @Override
+ public int getNumberOfPersistentBuffers() {
+ return columnsPages.size();
+ }
+
+ @Override
+ public ByteBuffer confiscateTemporary() throws HyracksDataException {
+ CachedPage page = (CachedPage) bufferCache.confiscatePage(IBufferCache.INVALID_DPID);
+ tempConfiscatedPages.add(page);
+ return page.getBuffer();
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreeFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreeFactory.java
new file mode 100644
index 0000000..1b9e198
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreeFactory.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.storage.am.lsm.btree.column.impls.btree;
+
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.api.io.IIOManager;
+import org.apache.hyracks.storage.am.common.api.IPageManagerFactory;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import org.apache.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+
+public class ColumnBTreeFactory extends TreeIndexFactory<ColumnBTree> {
+ public ColumnBTreeFactory(IIOManager ioManager, IBufferCache bufferCache,
+ IPageManagerFactory freePageManagerFactory, ITreeIndexFrameFactory interiorFrameFactory,
+ ITreeIndexFrameFactory leafFrameFactory, IBinaryComparatorFactory[] cmpFactories, int fieldCount) {
+ super(ioManager, bufferCache, freePageManagerFactory, interiorFrameFactory, leafFrameFactory, cmpFactories,
+ fieldCount);
+ }
+
+ @Override
+ public ColumnBTree createIndexInstance(FileReference file) throws HyracksDataException {
+ return new ColumnBTree(bufferCache, freePageManagerFactory.createPageManager(bufferCache), interiorFrameFactory,
+ leafFrameFactory, cmpFactories, fieldCount, file);
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreeLeafFrameFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreeLeafFrameFactory.java
new file mode 100644
index 0000000..31d85bd
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreeLeafFrameFactory.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.lsm.btree.column.impls.btree;
+
+import org.apache.hyracks.storage.am.btree.frames.BTreeNSMLeafFrame;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleWriterFactory;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.AbstractColumnTupleReader;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.AbstractColumnTupleReaderWriterFactory;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.AbstractColumnTupleWriter;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnMetadata;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.projection.IColumnProjectionInfo;
+
+public class ColumnBTreeLeafFrameFactory implements ITreeIndexFrameFactory {
+ private static final long serialVersionUID = 4136035898137820322L;
+ private final ITreeIndexTupleWriterFactory rowTupleWriterFactory;
+ private final AbstractColumnTupleReaderWriterFactory columnTupleWriterFactory;
+
+ public ColumnBTreeLeafFrameFactory(ITreeIndexTupleWriterFactory rowTupleWriterFactory,
+ AbstractColumnTupleReaderWriterFactory columnTupleWriterFactory) {
+ this.rowTupleWriterFactory = rowTupleWriterFactory;
+ this.columnTupleWriterFactory = columnTupleWriterFactory;
+ }
+
+ @Override
+ public ITreeIndexFrame createFrame() {
+ //Create a dummy leaf frame
+ return new BTreeNSMLeafFrame(rowTupleWriterFactory.createTupleWriter());
+ }
+
+ @Override
+ public ITreeIndexTupleWriterFactory getTupleWriterFactory() {
+ return rowTupleWriterFactory;
+ }
+
+ public ColumnBTreeWriteLeafFrame createWriterFrame(IColumnMetadata columnMetadata) {
+ ITreeIndexTupleWriter rowTupleWriter = rowTupleWriterFactory.createTupleWriter();
+ AbstractColumnTupleWriter columnTupleWriter = columnTupleWriterFactory.createColumnWriter(columnMetadata);
+ return new ColumnBTreeWriteLeafFrame(rowTupleWriter, columnTupleWriter);
+ }
+
+ public ColumnBTreeReadLeafFrame createReadFrame(IColumnProjectionInfo columnProjectionInfo) {
+ ITreeIndexTupleWriter rowTupleWriter = rowTupleWriterFactory.createTupleWriter();
+ AbstractColumnTupleReader columnTupleReader = columnTupleWriterFactory.createColumnReader(columnProjectionInfo);
+ return new ColumnBTreeReadLeafFrame(rowTupleWriter, columnTupleReader);
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreePointSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreePointSearchCursor.java
new file mode 100644
index 0000000..c93e77e
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreePointSearchCursor.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.hyracks.storage.am.lsm.btree.column.impls.btree;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.btree.api.IDiskBTreeStatefulPointSearchCursor;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
+import org.apache.hyracks.storage.common.IIndexCursorStats;
+import org.apache.hyracks.storage.common.ISearchPredicate;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+
+public class ColumnBTreePointSearchCursor extends ColumnBTreeRangeSearchCursor
+ implements IDiskBTreeStatefulPointSearchCursor {
+
+ public ColumnBTreePointSearchCursor(ColumnBTreeReadLeafFrame frame, IIndexCursorStats stats, int index) {
+ super(frame, stats, index);
+ }
+
+ @Override
+ public void doClose() throws HyracksDataException {
+ pageId = IBufferCache.INVALID_PAGEID;
+ super.doClose();
+ }
+
+ @Override
+ public int getLastPageId() {
+ return pageId;
+ }
+
+ @Override
+ public void setCursorToNextKey(ISearchPredicate searchPred) throws HyracksDataException {
+ initCursorPosition(searchPred);
+ }
+
+ @Override
+ public ITreeIndexFrame getFrame() {
+ return frame;
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreeRangeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreeRangeSearchCursor.java
new file mode 100644
index 0000000..fe980cc
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreeRangeSearchCursor.java
@@ -0,0 +1,238 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.btree.column.impls.btree;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.btree.impls.BTreeCursorInitialState;
+import org.apache.hyracks.storage.am.btree.impls.RangePredicate;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexCursor;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnReadMultiPageOp;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnTupleIterator;
+import org.apache.hyracks.storage.common.EnforcedIndexCursor;
+import org.apache.hyracks.storage.common.ICursorInitialState;
+import org.apache.hyracks.storage.common.IIndexCursorStats;
+import org.apache.hyracks.storage.common.ISearchPredicate;
+import org.apache.hyracks.storage.common.MultiComparator;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.storage.common.buffercache.ICachedPage;
+import org.apache.hyracks.storage.common.file.BufferedFileHandle;
+
+public class ColumnBTreeRangeSearchCursor extends EnforcedIndexCursor
+ implements ITreeIndexCursor, IColumnReadMultiPageOp {
+
+ protected final ColumnBTreeReadLeafFrame frame;
+ protected final IColumnTupleIterator frameTuple;
+
+ protected IBufferCache bufferCache = null;
+ protected int fileId;
+
+ protected int pageId;
+ protected ICachedPage page0 = null;
+
+ protected final RangePredicate reusablePredicate;
+ protected MultiComparator originalKeyCmp;
+
+ protected RangePredicate pred;
+ protected ITupleReference lowKey;
+ protected ITupleReference highKey;
+ protected boolean firstNextCall;
+
+ protected final IIndexCursorStats stats;
+
+ public ColumnBTreeRangeSearchCursor(ColumnBTreeReadLeafFrame frame, IIndexCursorStats stats, int index) {
+ this.frame = frame;
+ this.frameTuple = frame.createTupleReference(index, this);
+ this.reusablePredicate = new RangePredicate();
+ this.stats = stats;
+ fileId = -1;
+ pageId = IBufferCache.INVALID_PAGEID;
+ }
+
+ @Override
+ public void doDestroy() throws HyracksDataException {
+ // No Op all resources are released in the close call
+ }
+
+ @Override
+ public ITupleReference doGetTuple() {
+ return frameTuple;
+ }
+
+ private void fetchNextLeafPage(int leafPage) throws HyracksDataException {
+ int nextLeafPage = leafPage;
+ do {
+ ICachedPage nextLeaf = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, nextLeafPage), false);
+ stats.getPageCounter().update(1);
+ bufferCache.unpin(page0);
+ page0 = nextLeaf;
+ frame.setPage(page0);
+ frameTuple.reset(0);
+ nextLeafPage = frame.getNextLeaf();
+ } while (frame.getTupleCount() == 0 && nextLeafPage > 0);
+ }
+
+ @Override
+ public boolean doHasNext() throws HyracksDataException {
+ int nextLeafPage;
+ if (frameTuple.isConsumed() && !firstNextCall) {
+ frameTuple.lastTupleReached();
+ nextLeafPage = frame.getNextLeaf();
+ if (nextLeafPage >= 0) {
+ fetchNextLeafPage(nextLeafPage);
+ } else {
+ return false;
+ }
+ }
+ return isNextIncluded();
+ }
+
+ @Override
+ public void doNext() throws HyracksDataException {
+ //NoOp
+ }
+
+ @Override
+ public void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
+ // in case open is called multiple times without closing
+ if (page0 != null) {
+ releasePages();
+ }
+ originalKeyCmp = initialState.getOriginalKeyComparator();
+ page0 = initialState.getPage();
+ pageId = ((BTreeCursorInitialState) initialState).getPageId();
+ frame.setPage(page0);
+ frame.setMultiComparator(originalKeyCmp);
+ frameTuple.reset(0);
+ initCursorPosition(searchPred);
+ }
+
+ protected void initCursorPosition(ISearchPredicate searchPred) throws HyracksDataException {
+ pred = (RangePredicate) searchPred;
+ lowKey = pred.getLowKey();
+ highKey = pred.getHighKey();
+
+ reusablePredicate.setLowKeyComparator(originalKeyCmp);
+ reusablePredicate.setHighKeyComparator(pred.getHighKeyComparator());
+ reusablePredicate.setHighKey(pred.getHighKey(), pred.isHighKeyInclusive());
+ firstNextCall = true;
+ advanceTupleToLowKey();
+ }
+
+ protected boolean isNextIncluded() throws HyracksDataException {
+ if (firstNextCall) {
+ //The first call of frameTuple.next() was done during the opening of the cursor
+ firstNextCall = false;
+ return true;
+ } else if (frameTuple.isConsumed()) {
+ //All tuple were consumed
+ return false;
+ }
+ //Next tuple
+ frameTuple.next();
+ //Check whether the frameTuple is not consumed and also include the search key
+ return highKey == null || isLessOrEqual(frameTuple, highKey, pred.isHighKeyInclusive());
+ }
+
+ protected void advanceTupleToLowKey() throws HyracksDataException {
+ if (highKey != null && isLessOrEqual(highKey, frame.getLeftmostTuple(), !pred.isHighKeyInclusive())) {
+ /*
+ * Lowest key from the frame is greater than the requested highKey. No tuple will satisfy the search
+ * key. Consume the frameTuple to stop the search
+ */
+ firstNextCall = false;
+ frameTuple.consume();
+ return;
+ } else if (lowKey == null) {
+ //No range was specified.
+ frameTuple.next();
+ return;
+ }
+
+ //The lowKey is somewhere within the frame tuples
+ boolean stop = false;
+ int counter = 0;
+ while (!stop && !frameTuple.isConsumed()) {
+ frameTuple.next();
+ stop = isLessOrEqual(lowKey, frameTuple, pred.isLowKeyInclusive());
+ counter++;
+ }
+ //Advance all columns to the proper position
+ frameTuple.skip(counter - 1);
+ }
+
+ protected void releasePages() throws HyracksDataException {
+ //Unpin all column pages first
+ frameTuple.unpinColumnsPages();
+ if (page0 != null) {
+ bufferCache.unpin(page0);
+ }
+ }
+
+ private boolean isLessOrEqual(ITupleReference left, ITupleReference right, boolean inclusive)
+ throws HyracksDataException {
+ int cmp = originalKeyCmp.compare(left, right);
+ return cmp < 0 || cmp == 0 && inclusive;
+ }
+
+ @Override
+ public void doClose() throws HyracksDataException {
+ frameTuple.close();
+ releasePages();
+ page0 = null;
+ pred = null;
+ }
+
+ @Override
+ public void setBufferCache(IBufferCache bufferCache) {
+ this.bufferCache = bufferCache;
+ }
+
+ @Override
+ public void setFileId(int fileId) {
+ this.fileId = fileId;
+ }
+
+ @Override
+ public boolean isExclusiveLatchNodes() {
+ return false;
+ }
+
+ /*
+ * ***********************************************************
+ * IColumnReadMultiPageOp
+ * ***********************************************************
+ */
+ @Override
+ public ICachedPage pin(int pageId) throws HyracksDataException {
+ stats.getPageCounter().update(1);
+ return bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false);
+ }
+
+ @Override
+ public void unpin(ICachedPage page) throws HyracksDataException {
+ bufferCache.unpin(page);
+ }
+
+ @Override
+ public int getPageSize() {
+ return bufferCache.getPageSize();
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreeReadLeafFrame.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreeReadLeafFrame.java
new file mode 100644
index 0000000..8872613
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreeReadLeafFrame.java
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.btree.column.impls.btree;
+
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleReference;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.AbstractColumnTupleReader;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnReadMultiPageOp;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnTupleIterator;
+import org.apache.hyracks.storage.common.buffercache.CachedPage;
+import org.apache.hyracks.storage.common.file.BufferedFileHandle;
+
+public final class ColumnBTreeReadLeafFrame extends AbstractColumnBTreeLeafFrame {
+ private final AbstractColumnTupleReader columnarTupleReader;
+ private final ITreeIndexTupleReference leftMostTuple;
+ private final ITreeIndexTupleReference rightMostTuple;
+
+ public ColumnBTreeReadLeafFrame(ITreeIndexTupleWriter rowTupleWriter,
+ AbstractColumnTupleReader columnarTupleReader) {
+ super(rowTupleWriter);
+ this.columnarTupleReader = columnarTupleReader;
+ leftMostTuple = rowTupleWriter.createTupleReference();
+ rightMostTuple = rowTupleWriter.createTupleReference();
+ }
+
+ @Override
+ public ITupleReference getLeftmostTuple() {
+ if (getTupleCount() == 0) {
+ return null;
+ }
+ leftMostTuple.setFieldCount(cmp.getKeyFieldCount());
+ leftMostTuple.resetByTupleOffset(buf.array(), buf.getInt(LEFT_MOST_KEY_OFFSET));
+ return leftMostTuple;
+ }
+
+ @Override
+ public ITupleReference getRightmostTuple() {
+ if (getTupleCount() == 0) {
+ return null;
+ }
+ rightMostTuple.setFieldCount(cmp.getKeyFieldCount());
+ rightMostTuple.resetByTupleOffset(buf.array(), buf.getInt(RIGHT_MOST_KEY_OFFSET));
+ return rightMostTuple;
+ }
+
+ public IColumnTupleIterator createTupleReference(int index, IColumnReadMultiPageOp multiPageOp) {
+ return columnarTupleReader.createTupleIterator(this, index, multiPageOp);
+ }
+
+ @Override
+ public int getTupleCount() {
+ return buf.getInt(Constants.TUPLE_COUNT_OFFSET);
+ }
+
+ public int getPageId() {
+ return BufferedFileHandle.getPageId(((CachedPage) page).getDiskPageId());
+ }
+
+ public int getNumberOfColumns() {
+ return buf.getInt(NUMBER_OF_COLUMNS_OFFSET);
+ }
+
+ public int getColumnOffset(int columnIndex) {
+ if (columnIndex >= getNumberOfColumns()) {
+ throw new IndexOutOfBoundsException(columnIndex + " >= " + getNumberOfColumns());
+ }
+ return columnarTupleReader.getColumnOffset(buf, columnIndex);
+ }
+
+ AbstractColumnTupleReader getColumnarTupleReader() {
+ return columnarTupleReader;
+ }
+
+ int getNextLeaf() {
+ return buf.getInt(NEXT_LEAF_OFFSET);
+ }
+
+ @Override
+ public ITreeIndexTupleReference createTupleReference() {
+ throw new IllegalArgumentException("Use createTupleReference(int)");
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreeWriteLeafFrame.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreeWriteLeafFrame.java
new file mode 100644
index 0000000..275fb0e
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreeWriteLeafFrame.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.btree.column.impls.btree;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.AbstractColumnTupleWriter;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnWriteMultiPageOp;
+
+public class ColumnBTreeWriteLeafFrame extends AbstractColumnBTreeLeafFrame {
+ private final AbstractColumnTupleWriter columnTupleWriter;
+
+ public ColumnBTreeWriteLeafFrame(ITreeIndexTupleWriter rowTupleWriter,
+ AbstractColumnTupleWriter columnTupleWriter) {
+ super(rowTupleWriter);
+ this.columnTupleWriter = columnTupleWriter;
+ }
+
+ @Override
+ public void initBuffer(byte level) {
+ buf.putInt(TUPLE_COUNT_OFFSET, 0);
+ buf.put(Constants.LEVEL_OFFSET, level);
+ buf.putInt(NUMBER_OF_COLUMNS_OFFSET, 0);
+ buf.putInt(LEFT_MOST_KEY_OFFSET, -1);
+ buf.putInt(RIGHT_MOST_KEY_OFFSET, -1);
+ buf.putInt(SIZE_OF_COLUMNS_OFFSETS_OFFSET, 0);
+ buf.putInt(NUMBER_OF_COLUMN_PAGES, 0);
+ buf.put(FLAG_OFFSET, (byte) 0);
+ buf.putInt(NEXT_LEAF_OFFSET, -1);
+ }
+
+ void flush(AbstractColumnTupleWriter columnWriter, int numberOfTuples, IColumnWriteMultiPageOp multiPageOp,
+ ITupleReference minKey, ITupleReference maxKey) throws HyracksDataException {
+ //Prepare the space for writing the columns' information such as the primary keys
+ buf.position(HEADER_SIZE);
+ //Write the columns' information including the columns' offsets and the primary keys
+ columnWriter.flush(buf);
+
+ //Write min and max keys
+ int offset = buf.position();
+ buf.putInt(LEFT_MOST_KEY_OFFSET, offset);
+ offset += rowTupleWriter.writeTuple(minKey, buf.array(), offset);
+ buf.putInt(RIGHT_MOST_KEY_OFFSET, offset);
+ rowTupleWriter.writeTuple(maxKey, buf.array(), offset);
+
+ //Write page information
+ int numberOfColumns = columnWriter.getNumberOfColumns();
+ buf.putInt(TUPLE_COUNT_OFFSET, numberOfTuples);
+ buf.putInt(NUMBER_OF_COLUMNS_OFFSET, numberOfColumns);
+ buf.putInt(SIZE_OF_COLUMNS_OFFSETS_OFFSET, columnWriter.getColumnOffsetsSize());
+ buf.putInt(NUMBER_OF_COLUMN_PAGES, multiPageOp.getNumberOfPersistentBuffers());
+ }
+
+ public AbstractColumnTupleWriter getColumnTupleWriter() {
+ return columnTupleWriter;
+ }
+
+ void setNextLeaf(int pageId) {
+ buf.putInt(NEXT_LEAF_OFFSET, pageId);
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnBTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnBTree.java
new file mode 100644
index 0000000..048d9de
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnBTree.java
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.btree.column.impls.lsm;
+
+import java.util.List;
+
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IIOManager;
+import org.apache.hyracks.storage.am.common.api.IExtendedModificationOperationCallback;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnManager;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnMetadata;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.projection.IColumnTupleProjector;
+import org.apache.hyracks.storage.am.lsm.btree.column.utils.ColumnUtil;
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTree;
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeBatchPointSearchCursor;
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeRangeSearchCursor;
+import org.apache.hyracks.storage.am.lsm.common.api.IComponentMetadata;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMPageWriteCallbackFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexOperationContext;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMTreeIndexAccessor.ICursorFactory;
+import org.apache.hyracks.storage.common.IIndexAccessParameters;
+import org.apache.hyracks.storage.common.IIndexCursorStats;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.util.trace.ITracer;
+
+public class LSMColumnBTree extends LSMBTree {
+ private static final ICursorFactory CURSOR_FACTORY = LSMColumnBTreeSearchCursor::new;
+ private final IColumnManager columnManager;
+ private final ILSMDiskComponentFactory mergeComponentFactory;
+ /**
+ * This column metadata only used during flush and dataset bulkload operations. We cannot have more than one
+ * thread to do a flush/dataset bulkload. Do not use it for search/scan. Instead, use the latest component
+ * metadata of the operational disk components.
+ *
+ * @see LSMColumnBTreeOpContext#createProjectionInfo()
+ */
+ private IColumnMetadata columnMetadata;
+
+ public LSMColumnBTree(IIOManager ioManager, List<IVirtualBufferCache> virtualBufferCaches,
+ ITreeIndexFrameFactory interiorFrameFactory, ITreeIndexFrameFactory insertLeafFrameFactory,
+ ITreeIndexFrameFactory deleteLeafFrameFactory, IBufferCache diskBufferCache,
+ ILSMIndexFileManager fileManager, ILSMDiskComponentFactory componentFactory,
+ ILSMDiskComponentFactory mergeComponentFactory, ILSMDiskComponentFactory bulkloadComponentFactory,
+ double bloomFilterFalsePositiveRate, int fieldCount, IBinaryComparatorFactory[] cmpFactories,
+ ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
+ ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
+ int[] btreeFields, ITracer tracer, IColumnManager columnManager) throws HyracksDataException {
+ super(ioManager, virtualBufferCaches, interiorFrameFactory, insertLeafFrameFactory, deleteLeafFrameFactory,
+ diskBufferCache, fileManager, componentFactory, bulkloadComponentFactory, null, null, null,
+ bloomFilterFalsePositiveRate, fieldCount, cmpFactories, mergePolicy, opTracker, ioScheduler,
+ ioOpCallbackFactory, pageWriteCallbackFactory, true, true, btreeFields, null, true, false, tracer);
+ this.columnManager = columnManager;
+ this.mergeComponentFactory = mergeComponentFactory;
+ }
+
+ @Override
+ public synchronized void activate() throws HyracksDataException {
+ super.activate();
+ if (diskComponents.isEmpty()) {
+ columnMetadata = columnManager.activate();
+ } else {
+ IComponentMetadata componentMetadata = diskComponents.get(0).getMetadata();
+ columnMetadata = columnManager.activate(ColumnUtil.getColumnMetadataCopy(componentMetadata));
+ }
+ }
+
+ @Override
+ public LSMColumnBTreeOpContext createOpContext(IIndexAccessParameters iap) {
+ int numBloomFilterKeyFields =
+ ((LSMColumnBTreeWithBloomFilterDiskComponentFactory) componentFactory).getBloomFilterKeyFields().length;
+ IColumnTupleProjector tupleProjector =
+ ColumnUtil.getTupleProjector(iap, columnManager.getMergeColumnProjector());
+ return new LSMColumnBTreeOpContext(this, memoryComponents, insertLeafFrameFactory, deleteLeafFrameFactory,
+ (IExtendedModificationOperationCallback) iap.getModificationCallback(),
+ iap.getSearchOperationCallback(), numBloomFilterKeyFields, getTreeFields(), getFilterFields(),
+ getHarness(), getFilterCmpFactories(), tracer, tupleProjector);
+ }
+
+ protected IColumnManager getColumnManager() {
+ return columnManager;
+ }
+
+ protected IColumnMetadata getColumnMetadata() {
+ return columnMetadata;
+ }
+
+ @Override
+ protected LSMBTreeRangeSearchCursor createCursor(AbstractLSMIndexOperationContext opCtx,
+ boolean returnDeletedTuples, IIndexCursorStats stats) {
+ return new LSMColumnBTreeRangeSearchCursor(opCtx, returnDeletedTuples, stats);
+ }
+
+ @Override
+ public LSMBTreeBatchPointSearchCursor createBatchPointSearchCursor(ILSMIndexOperationContext opCtx) {
+ return new LSMColumnBatchPointSearchCursor(opCtx);
+ }
+
+ @Override
+ protected ILSMDiskComponentFactory getMergeComponentFactory() {
+ return mergeComponentFactory;
+ }
+
+ @Override
+ public ICursorFactory getCursorFactory() {
+ return CURSOR_FACTORY;
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnBTreeOpContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnBTreeOpContext.java
new file mode 100644
index 0000000..8a33de1
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnBTreeOpContext.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.hyracks.storage.am.lsm.btree.column.impls.lsm;
+
+import java.util.List;
+
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.storage.am.common.api.IExtendedModificationOperationCallback;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.projection.IColumnProjectionInfo;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.projection.IColumnTupleProjector;
+import org.apache.hyracks.storage.am.lsm.btree.column.impls.lsm.tuples.ColumnAwareMultiComparator;
+import org.apache.hyracks.storage.am.lsm.btree.column.utils.ColumnUtil;
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeOpContext;
+import org.apache.hyracks.storage.am.lsm.common.api.IComponentMetadata;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent.LSMComponentType;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMHarness;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
+import org.apache.hyracks.storage.common.ISearchOperationCallback;
+import org.apache.hyracks.storage.common.MultiComparator;
+import org.apache.hyracks.util.trace.ITracer;
+
+public class LSMColumnBTreeOpContext extends LSMBTreeOpContext {
+ private final IColumnTupleProjector projector;
+
+ public LSMColumnBTreeOpContext(ILSMIndex index, List<ILSMMemoryComponent> mutableComponents,
+ ITreeIndexFrameFactory insertLeafFrameFactory, ITreeIndexFrameFactory deleteLeafFrameFactory,
+ IExtendedModificationOperationCallback modificationCallback, ISearchOperationCallback searchCallback,
+ int numBloomFilterKeyFields, int[] btreeFields, int[] filterFields, ILSMHarness lsmHarness,
+ IBinaryComparatorFactory[] filterCmpFactories, ITracer tracer, IColumnTupleProjector projector) {
+ super(index, mutableComponents, insertLeafFrameFactory, deleteLeafFrameFactory, modificationCallback,
+ searchCallback, numBloomFilterKeyFields, btreeFields, filterFields, lsmHarness, filterCmpFactories,
+ tracer);
+ this.projector = projector;
+ }
+
+ public IColumnProjectionInfo createProjectionInfo() throws HyracksDataException {
+ List<ILSMComponent> operationalComponents = getComponentHolder();
+ IComponentMetadata componentMetadata = null;
+ for (int i = 0; i < operationalComponents.size() && componentMetadata == null; i++) {
+ ILSMComponent component = operationalComponents.get(i);
+ if (component.getType() == LSMComponentType.DISK) {
+ //Find the first on-disk component, which has the most recent column metadata.
+ componentMetadata = component.getMetadata();
+ }
+ }
+ if (componentMetadata != null) {
+ IValueReference columnMetadata = ColumnUtil.getColumnMetadataCopy(componentMetadata);
+ return projector.createProjectionInfo(columnMetadata);
+ }
+ //In-memory components only
+ return null;
+ }
+
+ @Override
+ protected MultiComparator createMultiComparator(IBinaryComparatorFactory[] cmpFactories) {
+ IBinaryComparator[] comparators = new IBinaryComparator[cmpFactories.length];
+ for (int i = 0; i < comparators.length; i++) {
+ comparators[i] = cmpFactories[i].createBinaryComparator();
+ }
+ return new ColumnAwareMultiComparator(comparators);
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnBTreeRangeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnBTreeRangeSearchCursor.java
new file mode 100644
index 0000000..75b2c72
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnBTreeRangeSearchCursor.java
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.btree.column.impls.lsm;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.btree.impls.BTree;
+import org.apache.hyracks.storage.am.btree.impls.BTree.BTreeAccessor;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnTupleIterator;
+import org.apache.hyracks.storage.am.lsm.btree.column.impls.btree.ColumnBTree;
+import org.apache.hyracks.storage.am.lsm.btree.column.impls.btree.ColumnBTreeRangeSearchCursor;
+import org.apache.hyracks.storage.am.lsm.btree.column.impls.lsm.tuples.ColumnAwareDiskOnlyMultiComparator;
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeRangeSearchCursor;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent.LSMComponentType;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+import org.apache.hyracks.storage.common.IIndexCursor;
+import org.apache.hyracks.storage.common.IIndexCursorStats;
+import org.apache.hyracks.storage.common.NoOpIndexCursorStats;
+
+public class LSMColumnBTreeRangeSearchCursor extends LSMBTreeRangeSearchCursor {
+ private final List<IColumnTupleIterator> componentTupleList;
+
+ public LSMColumnBTreeRangeSearchCursor(ILSMIndexOperationContext opCtx) {
+ this(opCtx, false, NoOpIndexCursorStats.INSTANCE);
+ }
+
+ public LSMColumnBTreeRangeSearchCursor(ILSMIndexOperationContext opCtx, boolean returnDeletedTuples,
+ IIndexCursorStats stats) {
+ super(opCtx, returnDeletedTuples, stats);
+ componentTupleList = new ArrayList<>();
+ }
+
+ @Override
+ protected BTreeAccessor createAccessor(LSMComponentType type, BTree btree, int index) throws HyracksDataException {
+ if (type == LSMComponentType.MEMORY) {
+ return super.createAccessor(type, btree, index);
+ }
+ ColumnBTree columnBTree = (ColumnBTree) btree;
+ LSMColumnBTreeOpContext columnOpCtx = (LSMColumnBTreeOpContext) opCtx;
+ return columnBTree.createAccessor(iap, index, columnOpCtx.createProjectionInfo());
+ }
+
+ @Override
+ protected IIndexCursor createCursor(LSMComponentType type, BTreeAccessor accessor) {
+ if (type == LSMComponentType.MEMORY) {
+ return super.createCursor(type, accessor);
+ }
+ ColumnBTreeRangeSearchCursor cursor = (ColumnBTreeRangeSearchCursor) accessor.createSearchCursor(false);
+ componentTupleList.add((IColumnTupleIterator) cursor.doGetTuple());
+ return cursor;
+ }
+
+ @Override
+ protected void markAsDeleted(PriorityQueueElement e) throws HyracksDataException {
+ if (isMemoryComponent[e.getCursorIndex()]) {
+ super.markAsDeleted(e);
+ return;
+ }
+ IColumnTupleIterator columnTuple = (IColumnTupleIterator) e.getTuple();
+ columnTuple.skip(1);
+ }
+
+ @Override
+ protected void setPriorityQueueComparator() {
+ if (!includeMutableComponent) {
+ cmp = new ColumnAwareDiskOnlyMultiComparator(cmp);
+ }
+ if (pqCmp == null || cmp != pqCmp.getMultiComparator()) {
+ pqCmp = new PriorityQueueComparator(cmp);
+ }
+ }
+
+ @Override
+ protected void excludeMemoryComponent() {
+ //Replace the comparator with disk only comparator
+ pqCmp.setMultiComparator(new ColumnAwareDiskOnlyMultiComparator(cmp));
+ }
+
+ @Override
+ protected int replaceFrom() {
+ //Disable replacing the in-memory component to disk component as the schema may change
+ //TODO at least allow the replacement when no schema changes occur
+ return -1;
+ }
+
+ /**
+ * @return we need the tuple references for vertical merges
+ */
+ public List<IColumnTupleIterator> getComponentTupleList() {
+ return componentTupleList;
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnBTreeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnBTreeSearchCursor.java
new file mode 100644
index 0000000..6ca41d0
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnBTreeSearchCursor.java
@@ -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.
+ */
+package org.apache.hyracks.storage.am.lsm.btree.column.impls.lsm;
+
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeDiskComponentScanCursor;
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeSearchCursor;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+
+public class LSMColumnBTreeSearchCursor extends LSMBTreeSearchCursor {
+ public LSMColumnBTreeSearchCursor(ILSMIndexOperationContext opCtx) {
+ super(new LSMColumnPointSearchCursor(opCtx), new LSMColumnBTreeRangeSearchCursor(opCtx),
+ new LSMBTreeDiskComponentScanCursor(opCtx));
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnBTreeWithBloomFilterDiskComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnBTreeWithBloomFilterDiskComponent.java
new file mode 100644
index 0000000..57e162d
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnBTreeWithBloomFilterDiskComponent.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.storage.am.lsm.btree.column.impls.lsm;
+
+import java.util.List;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilter;
+import org.apache.hyracks.storage.am.btree.impls.BTree;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnMetadata;
+import org.apache.hyracks.storage.am.lsm.btree.column.impls.btree.ColumnBTree;
+import org.apache.hyracks.storage.am.lsm.btree.column.utils.ColumnUtil;
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeMergeOperation;
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeWithBloomFilterDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.IComponentMetadata;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.impls.ChainedLSMDiskComponentBulkLoader;
+import org.apache.hyracks.storage.am.lsm.common.impls.IChainedComponentBulkLoader;
+import org.apache.hyracks.storage.common.IIndexBulkLoader;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
+
+public class LSMColumnBTreeWithBloomFilterDiskComponent extends LSMBTreeWithBloomFilterDiskComponent {
+
+ public LSMColumnBTreeWithBloomFilterDiskComponent(AbstractLSMIndex lsmIndex, BTree btree, BloomFilter bloomFilter,
+ ILSMComponentFilter filter) {
+ super(lsmIndex, btree, bloomFilter, filter);
+ }
+
+ @Override
+ public ChainedLSMDiskComponentBulkLoader createBulkLoader(ILSMIOOperation operation, float fillFactor,
+ boolean verifyInput, long numElementsHint, boolean checkIfEmptyIndex, boolean withFilter,
+ boolean cleanupEmptyComponent, IPageWriteCallback callback) throws HyracksDataException {
+ ChainedLSMDiskComponentBulkLoader chainedBulkLoader =
+ new ChainedLSMDiskComponentBulkLoader(operation, this, cleanupEmptyComponent);
+ if (withFilter && getLsmIndex().getFilterFields() != null) {
+ //Add filter writer if exists
+ chainedBulkLoader.addBulkLoader(createFilterBulkLoader());
+ }
+ //Add index bulkloader
+ chainedBulkLoader.addBulkLoader(createColumnIndexBulkLoader(operation, fillFactor, verifyInput, callback));
+
+ if (numElementsHint > 0) {
+ chainedBulkLoader.addBulkLoader(createBloomFilterBulkLoader(numElementsHint, callback));
+ }
+
+ callback.initialize(chainedBulkLoader);
+ return chainedBulkLoader;
+ }
+
+ private IChainedComponentBulkLoader createColumnIndexBulkLoader(ILSMIOOperation operation, float fillFactor,
+ boolean verifyInput, IPageWriteCallback callback) throws HyracksDataException {
+ LSMIOOperationType operationType = operation.getIOOpertionType();
+ LSMColumnBTree lsmColumnBTree = (LSMColumnBTree) getLsmIndex();
+ ColumnBTree columnBTree = (ColumnBTree) getIndex();
+ IColumnMetadata columnMetadata;
+ if (operationType == LSMIOOperationType.FLUSH || operationType == LSMIOOperationType.LOAD) {
+ columnMetadata = lsmColumnBTree.getColumnMetadata();
+ } else {
+ //Merge
+ LSMBTreeMergeOperation mergeOp = (LSMBTreeMergeOperation) operation;
+ LSMColumnBTreeRangeSearchCursor cursor = (LSMColumnBTreeRangeSearchCursor) mergeOp.getCursor();
+ List<ILSMComponent> mergingComponents = mergeOp.getMergingComponents();
+ IComponentMetadata componentMetadata = mergingComponents.get(0).getMetadata();
+ IValueReference columnMetadataValue = ColumnUtil.getColumnMetadataCopy(componentMetadata);
+ columnMetadata = lsmColumnBTree.getColumnManager().createMergeColumnMetadata(columnMetadataValue,
+ cursor.getComponentTupleList());
+ }
+ IIndexBulkLoader bulkLoader = columnBTree.createBulkLoader(fillFactor, verifyInput, callback, columnMetadata);
+ return new LSMColumnIndexBulkloader(bulkLoader, columnMetadata, getMetadata());
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnBTreeWithBloomFilterDiskComponentFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnBTreeWithBloomFilterDiskComponentFactory.java
new file mode 100644
index 0000000..2d86a73
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnBTreeWithBloomFilterDiskComponentFactory.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.hyracks.storage.am.lsm.btree.column.impls.lsm;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilterFactory;
+import org.apache.hyracks.storage.am.lsm.btree.column.impls.btree.ColumnBTree;
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeWithBloomFilterDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentFactory;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
+import org.apache.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
+
+/**
+ * We only create a BTree with a bloom filter
+ */
+public class LSMColumnBTreeWithBloomFilterDiskComponentFactory implements ILSMDiskComponentFactory {
+ private final TreeIndexFactory<ColumnBTree> btreeFactory;
+ private final BloomFilterFactory bloomFilterFactory;
+
+ public LSMColumnBTreeWithBloomFilterDiskComponentFactory(TreeIndexFactory<ColumnBTree> btreeFactory,
+ BloomFilterFactory bloomFilterFactory) {
+ this.btreeFactory = btreeFactory;
+ this.bloomFilterFactory = bloomFilterFactory;
+ }
+
+ @Override
+ public LSMBTreeWithBloomFilterDiskComponent createComponent(AbstractLSMIndex lsmIndex,
+ LSMComponentFileReferences cfr) throws HyracksDataException {
+ return new LSMColumnBTreeWithBloomFilterDiskComponent(lsmIndex,
+ btreeFactory.createIndexInstance(cfr.getInsertIndexFileReference()),
+ bloomFilterFactory.createBloomFiltertInstance(cfr.getBloomFilterFileReference()), null);
+ }
+
+ public int[] getBloomFilterKeyFields() {
+ return bloomFilterFactory.getBloomFilterKeyFields();
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnBatchPointSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnBatchPointSearchCursor.java
new file mode 100644
index 0000000..65b292b
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnBatchPointSearchCursor.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.lsm.btree.column.impls.lsm;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.btree.impls.BTree;
+import org.apache.hyracks.storage.am.btree.impls.BTree.BTreeAccessor;
+import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
+import org.apache.hyracks.storage.am.lsm.btree.column.impls.btree.ColumnBTree;
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeBatchPointSearchCursor;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent.LSMComponentType;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+
+public class LSMColumnBatchPointSearchCursor extends LSMBTreeBatchPointSearchCursor {
+
+ public LSMColumnBatchPointSearchCursor(ILSMIndexOperationContext opCtx) {
+ super(opCtx);
+ }
+
+ @Override
+ protected BTreeAccessor createAccessor(LSMComponentType type, BTree btree, int index) throws HyracksDataException {
+ if (type == LSMComponentType.MEMORY) {
+ return super.createAccessor(type, btree, index);
+ }
+ ColumnBTree columnBTree = (ColumnBTree) btree;
+ LSMColumnBTreeOpContext columnOpCtx = (LSMColumnBTreeOpContext) opCtx;
+ return columnBTree.createAccessor(NoOpIndexAccessParameters.INSTANCE, index,
+ columnOpCtx.createProjectionInfo());
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnIndexBulkloader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnIndexBulkloader.java
new file mode 100644
index 0000000..ba41227
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnIndexBulkloader.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.hyracks.storage.am.lsm.btree.column.impls.lsm;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnMetadata;
+import org.apache.hyracks.storage.am.lsm.btree.column.utils.ColumnUtil;
+import org.apache.hyracks.storage.am.lsm.common.api.IComponentMetadata;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMIndexBulkLoader;
+import org.apache.hyracks.storage.common.IIndexBulkLoader;
+
+public class LSMColumnIndexBulkloader extends LSMIndexBulkLoader {
+ private final IColumnMetadata columnMetadata;
+ private final IComponentMetadata componentMetadata;
+
+ public LSMColumnIndexBulkloader(IIndexBulkLoader bulkLoader, IColumnMetadata columnMetadata,
+ IComponentMetadata componentMetadata) {
+ super(bulkLoader);
+ this.columnMetadata = columnMetadata;
+ this.componentMetadata = componentMetadata;
+ }
+
+ @Override
+ public void end() throws HyracksDataException {
+ ColumnUtil.putColumnsMetadataValue(columnMetadata.serializeColumnsMetadata(), componentMetadata);
+ super.end();
+ }
+
+ @Override
+ public void abort() throws HyracksDataException {
+ columnMetadata.abort();
+ super.abort();
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnPointSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnPointSearchCursor.java
new file mode 100644
index 0000000..e193232
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnPointSearchCursor.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.lsm.btree.column.impls.lsm;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.btree.impls.BTree;
+import org.apache.hyracks.storage.am.btree.impls.BTree.BTreeAccessor;
+import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
+import org.apache.hyracks.storage.am.lsm.btree.column.impls.btree.ColumnBTree;
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreePointSearchCursor;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent.LSMComponentType;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+
+public class LSMColumnPointSearchCursor extends LSMBTreePointSearchCursor {
+
+ public LSMColumnPointSearchCursor(ILSMIndexOperationContext opCtx) {
+ super(opCtx);
+ }
+
+ @Override
+ protected BTreeAccessor createAccessor(LSMComponentType type, BTree btree, int index) throws HyracksDataException {
+ if (type == LSMComponentType.MEMORY) {
+ return super.createAccessor(type, btree, index);
+ }
+ ColumnBTree columnBTree = (ColumnBTree) btree;
+ LSMColumnBTreeOpContext columnOpCtx = (LSMColumnBTreeOpContext) opCtx;
+ return columnBTree.createAccessor(NoOpIndexAccessParameters.INSTANCE, index,
+ columnOpCtx.createProjectionInfo());
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/tuples/AbstractColumnTupleReference.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/tuples/AbstractColumnTupleReference.java
new file mode 100644
index 0000000..d39f94e
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/tuples/AbstractColumnTupleReference.java
@@ -0,0 +1,181 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.btree.column.impls.lsm.tuples;
+
+import static org.apache.hyracks.storage.am.lsm.btree.column.impls.btree.AbstractColumnBTreeLeafFrame.HEADER_SIZE;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnBufferProvider;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnReadMultiPageOp;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnTupleIterator;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.projection.IColumnProjectionInfo;
+import org.apache.hyracks.storage.am.lsm.btree.column.impls.btree.ColumnBTreeReadLeafFrame;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+public abstract class AbstractColumnTupleReference implements IColumnTupleIterator {
+ private static final Logger LOGGER = LogManager.getLogger();
+ private static final String UNSUPPORTED_OPERATION_MSG = "Operation is not supported for column tuples";
+ private final int componentIndex;
+ private final ColumnBTreeReadLeafFrame frame;
+ private final IColumnBufferProvider[] primaryKeyBufferProviders;
+ private final IColumnBufferProvider[] buffersProviders;
+ private final int numberOfPrimaryKeys;
+ private int totalNumberOfPages;
+ private int numOfSkippedPages;
+ protected int tupleIndex;
+
+ /**
+ * Column tuple reference
+ *
+ * @param componentIndex LSM component index
+ * @param frame page0 frame
+ * @param info projection info
+ */
+ protected AbstractColumnTupleReference(int componentIndex, ColumnBTreeReadLeafFrame frame,
+ IColumnProjectionInfo info, IColumnReadMultiPageOp multiPageOp) {
+ this.componentIndex = componentIndex;
+ this.frame = frame;
+ numberOfPrimaryKeys = info.getNumberOfPrimaryKeys();
+
+ primaryKeyBufferProviders = new IColumnBufferProvider[numberOfPrimaryKeys];
+
+ for (int i = 0; i < numberOfPrimaryKeys; i++) {
+ primaryKeyBufferProviders[i] = new ColumnSingleBufferProvider(i);
+ }
+
+ int numberOfRequestedColumns = info.getNumberOfProjectedColumns();
+ buffersProviders = new IColumnBufferProvider[numberOfRequestedColumns];
+ for (int i = 0; i < numberOfRequestedColumns; i++) {
+ int columnIndex = info.getColumnIndex(i);
+ if (columnIndex >= numberOfPrimaryKeys) {
+ buffersProviders[i] = new ColumnMultiBufferProvider(columnIndex, multiPageOp);
+ } else {
+ buffersProviders[i] = new ColumnSingleBufferProvider(columnIndex);
+ }
+ }
+ totalNumberOfPages = 0;
+ numOfSkippedPages = 0;
+ }
+
+ @Override
+ public final void reset(int startIndex) throws HyracksDataException {
+ tupleIndex = startIndex;
+ ByteBuffer pageZero = frame.getBuffer();
+ pageZero.clear();
+ pageZero.position(HEADER_SIZE);
+
+ int numberOfTuples = frame.getTupleCount();
+ //Start new page and check whether we should skip reading non-key columns or not
+ boolean readColumnPages = startNewPage(pageZero, frame.getNumberOfColumns(), numberOfTuples);
+
+ //Start primary keys
+ for (int i = 0; i < numberOfPrimaryKeys; i++) {
+ IColumnBufferProvider provider = primaryKeyBufferProviders[i];
+ provider.reset(frame);
+ startPrimaryKey(provider, tupleIndex, i, numberOfTuples);
+ }
+
+ if (readColumnPages) {
+ for (int i = 0; i < buffersProviders.length; i++) {
+ IColumnBufferProvider provider = buffersProviders[i];
+ //Release previous pinned pages if any
+ provider.releaseAll();
+ provider.reset(frame);
+ startColumn(provider, tupleIndex, i, numberOfTuples);
+ }
+ } else {
+ numOfSkippedPages++;
+ }
+ totalNumberOfPages++;
+ }
+
+ protected abstract boolean startNewPage(ByteBuffer pageZero, int numberOfColumns, int numberOfTuples);
+
+ protected abstract void startPrimaryKey(IColumnBufferProvider bufferProvider, int startIndex, int ordinal,
+ int numberOfTuples) throws HyracksDataException;
+
+ protected abstract void startColumn(IColumnBufferProvider buffersProvider, int startIndex, int ordinal,
+ int numberOfTuples) throws HyracksDataException;
+
+ protected abstract void onNext() throws HyracksDataException;
+
+ @Override
+ public final void next() throws HyracksDataException {
+ onNext();
+ tupleIndex++;
+ }
+
+ @Override
+ public final void consume() {
+ tupleIndex = frame.getTupleCount();
+ }
+
+ @Override
+ public final boolean isConsumed() {
+ return tupleIndex >= frame.getTupleCount();
+ }
+
+ @Override
+ public final int getComponentIndex() {
+ return componentIndex;
+ }
+
+ @Override
+ public final void unpinColumnsPages() throws HyracksDataException {
+ for (int i = 0; i < buffersProviders.length; i++) {
+ buffersProviders[i].releaseAll();
+ }
+ }
+
+ @Override
+ public final void close() {
+ if (LOGGER.isInfoEnabled()) {
+ LOGGER.info("Skipped {} pages out of {} in total", numOfSkippedPages, totalNumberOfPages);
+ }
+ }
+
+ /* *************************************************************
+ * Unsupported Operations
+ * *************************************************************
+ */
+
+ @Override
+ public final void setFieldCount(int fieldCount) {
+ throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+ }
+
+ @Override
+ public final void setFieldCount(int fieldStartIndex, int fieldCount) {
+ throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+ }
+
+ @Override
+ public final void resetByTupleOffset(byte[] buf, int tupleStartOffset) {
+ throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+ }
+
+ @Override
+ public final void resetByTupleIndex(ITreeIndexFrame frame, int tupleIndex) {
+ throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/tuples/ColumnAwareDiskOnlyMultiComparator.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/tuples/ColumnAwareDiskOnlyMultiComparator.java
new file mode 100644
index 0000000..297b740
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/tuples/ColumnAwareDiskOnlyMultiComparator.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.hyracks.storage.am.lsm.btree.column.impls.lsm.tuples;
+
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnTupleIterator;
+import org.apache.hyracks.storage.common.MultiComparator;
+
+public class ColumnAwareDiskOnlyMultiComparator extends MultiComparator {
+ public ColumnAwareDiskOnlyMultiComparator(MultiComparator comparator) {
+ super(comparator.getComparators());
+ }
+
+ @Override
+ public int compare(ITupleReference tupleA, ITupleReference tupleB) {
+ return ((IColumnTupleIterator) tupleA).compareTo((IColumnTupleIterator) tupleB);
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/tuples/ColumnAwareMultiComparator.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/tuples/ColumnAwareMultiComparator.java
new file mode 100644
index 0000000..0438f41
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/tuples/ColumnAwareMultiComparator.java
@@ -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.
+ */
+package org.apache.hyracks.storage.am.lsm.btree.column.impls.lsm.tuples;
+
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnTupleIterator;
+import org.apache.hyracks.storage.common.MultiComparator;
+
+public class ColumnAwareMultiComparator extends MultiComparator {
+ public ColumnAwareMultiComparator(IBinaryComparator[] cmps) {
+ super(cmps);
+ }
+
+ @Override
+ public int compare(ITupleReference tupleA, ITupleReference tupleB) throws HyracksDataException {
+ if (tupleA instanceof IColumnTupleIterator && tupleB instanceof IColumnTupleIterator) {
+ //Avoid comparing serialized data
+ return ((IColumnTupleIterator) tupleA).compareTo((IColumnTupleIterator) tupleB);
+ }
+ return super.compare(tupleA, tupleB);
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/tuples/ColumnMultiBufferProvider.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/tuples/ColumnMultiBufferProvider.java
new file mode 100644
index 0000000..0c17d6b
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/tuples/ColumnMultiBufferProvider.java
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.btree.column.impls.lsm.tuples;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayDeque;
+import java.util.Queue;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnBufferProvider;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnReadMultiPageOp;
+import org.apache.hyracks.storage.am.lsm.btree.column.impls.btree.ColumnBTreeReadLeafFrame;
+import org.apache.hyracks.storage.common.buffercache.ICachedPage;
+
+public final class ColumnMultiBufferProvider implements IColumnBufferProvider {
+ private final int columnIndex;
+ private final IColumnReadMultiPageOp multiPageOp;
+ private final Queue<ICachedPage> pages;
+ private int numberOfPages;
+ private int startPage;
+ private int startOffset;
+ private int length;
+
+ public ColumnMultiBufferProvider(int columnIndex, IColumnReadMultiPageOp multiPageOp) {
+ this.columnIndex = columnIndex;
+ this.multiPageOp = multiPageOp;
+ pages = new ArrayDeque<>();
+ }
+
+ @Override
+ public void reset(ColumnBTreeReadLeafFrame frame) throws HyracksDataException {
+ if (columnIndex >= frame.getNumberOfColumns()) {
+ numberOfPages = 0;
+ length = 0;
+ return;
+ }
+
+ int offset = frame.getColumnOffset(columnIndex);
+ startPage = frame.getPageId() + getColumnPageIndex(offset);
+ startOffset = offset % multiPageOp.getPageSize();
+ //Duplicate as the buffer could be shared by more than one column
+ ByteBuffer firstPage = readNext().duplicate();
+ firstPage.position(startOffset);
+ //Read the length
+ length = firstPage.getInt();
+ int remainingLength = length - firstPage.remaining();
+ numberOfPages = (int) Math.ceil((double) remainingLength / multiPageOp.getPageSize());
+ //+4-bytes after reading the length
+ startOffset += Integer.BYTES;
+ //-4-bytes after reading the length
+ length -= Integer.BYTES;
+ }
+
+ @Override
+ public void readAll(Queue<ByteBuffer> buffers) throws HyracksDataException {
+ ByteBuffer buffer = pages.peek().getBuffer().duplicate();
+ buffer.clear();
+ buffer.position(startOffset);
+ buffers.add(buffer);
+ for (int i = 0; i < numberOfPages; i++) {
+ buffer = readNext().duplicate();
+ buffer.clear();
+ buffers.add(buffer);
+ }
+ numberOfPages = 0;
+ }
+
+ @Override
+ public void releaseAll() throws HyracksDataException {
+ while (!pages.isEmpty()) {
+ ICachedPage page = pages.poll();
+ multiPageOp.unpin(page);
+ }
+ }
+
+ @Override
+ public int getLength() {
+ return length;
+ }
+
+ @Override
+ public ByteBuffer getBuffer() {
+ throw new UnsupportedOperationException("Use readAll() for multi-buffer");
+ }
+
+ @Override
+ public int getColumnIndex() {
+ return columnIndex;
+ }
+
+ private ByteBuffer readNext() throws HyracksDataException {
+ ICachedPage columnPage = multiPageOp.pin(startPage++);
+ pages.add(columnPage);
+ return columnPage.getBuffer();
+ }
+
+ private int getColumnPageIndex(int columnOffset) {
+ return (int) Math.floor((double) columnOffset / multiPageOp.getPageSize());
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/tuples/ColumnSingleBufferProvider.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/tuples/ColumnSingleBufferProvider.java
new file mode 100644
index 0000000..3ae5c7d
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/tuples/ColumnSingleBufferProvider.java
@@ -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.
+ */
+package org.apache.hyracks.storage.am.lsm.btree.column.impls.lsm.tuples;
+
+import java.nio.ByteBuffer;
+import java.util.Queue;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnBufferProvider;
+import org.apache.hyracks.storage.am.lsm.btree.column.impls.btree.ColumnBTreeReadLeafFrame;
+
+/**
+ * Buffer provider for Primary Keys
+ */
+public final class ColumnSingleBufferProvider implements IColumnBufferProvider {
+ private final int columnIndex;
+ private ByteBuffer buffer;
+
+ public ColumnSingleBufferProvider(int columnIndex) {
+ this.columnIndex = columnIndex;
+ }
+
+ @Override
+ public void reset(ColumnBTreeReadLeafFrame frame) {
+ int offset = frame.getColumnOffset(columnIndex);
+ this.buffer = frame.getBuffer().duplicate();
+ buffer.position(offset);
+ }
+
+ @Override
+ public void readAll(Queue<ByteBuffer> buffers) {
+ throw new UnsupportedOperationException("Use getBuffer() for single-buffer");
+ }
+
+ @Override
+ public void releaseAll() throws HyracksDataException {
+ //NoOp
+ }
+
+ @Override
+ public ByteBuffer getBuffer() {
+ return buffer;
+ }
+
+ @Override
+ public int getLength() {
+ return -1;
+ }
+
+ @Override
+ public int getColumnIndex() {
+ return columnIndex;
+ }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/utils/ColumnUtil.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/utils/ColumnUtil.java
new file mode 100644
index 0000000..25ed3db
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/utils/ColumnUtil.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.hyracks.storage.am.lsm.btree.column.utils;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.util.HyracksConstants;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.storage.am.common.freepage.MutableArrayValueReference;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnManager;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.projection.IColumnTupleProjector;
+import org.apache.hyracks.storage.am.lsm.btree.column.impls.lsm.LSMColumnBTree;
+import org.apache.hyracks.storage.am.lsm.common.api.IComponentMetadata;
+import org.apache.hyracks.storage.common.IIndexAccessParameters;
+
+public class ColumnUtil {
+ /**
+ * Used to get the columns info from {@link IComponentMetadata#get(IValueReference, ArrayBackedValueStorage)}
+ *
+ * @see LSMColumnBTree#activate()
+ * @see IColumnManager#activate(IValueReference)
+ */
+ private static final MutableArrayValueReference COLUMNS_METADATA_KEY =
+ new MutableArrayValueReference("COLUMNS_METADATA".getBytes());
+
+ private ColumnUtil() {
+ }
+
+ public static IValueReference getColumnMetadataCopy(IComponentMetadata src) throws HyracksDataException {
+ ArrayBackedValueStorage storage = new ArrayBackedValueStorage();
+ src.get(COLUMNS_METADATA_KEY, storage);
+ return storage;
+ }
+
+ public static void putColumnsMetadataValue(IValueReference columnsMetadataValue, IComponentMetadata dest)
+ throws HyracksDataException {
+ dest.put(COLUMNS_METADATA_KEY, columnsMetadataValue);
+ }
+
+ public static IColumnTupleProjector getTupleProjector(IIndexAccessParameters iap,
+ IColumnTupleProjector defaultProjector) {
+ IColumnTupleProjector projector =
+ iap.getParameter(HyracksConstants.TUPLE_PROJECTOR, IColumnTupleProjector.class);
+ return projector == null ? defaultProjector : projector;
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/utils/LSMColumnBTreeUtil.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/utils/LSMColumnBTreeUtil.java
new file mode 100644
index 0000000..1a55447
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/utils/LSMColumnBTreeUtil.java
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.btree.column.utils;
+
+import java.util.List;
+
+import org.apache.hyracks.api.compression.ICompressorDecompressorFactory;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.api.io.IIOManager;
+import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilterFactory;
+import org.apache.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
+import org.apache.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
+import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnManagerFactory;
+import org.apache.hyracks.storage.am.lsm.btree.column.impls.btree.ColumnBTree;
+import org.apache.hyracks.storage.am.lsm.btree.column.impls.btree.ColumnBTreeFactory;
+import org.apache.hyracks.storage.am.lsm.btree.column.impls.btree.ColumnBTreeLeafFrameFactory;
+import org.apache.hyracks.storage.am.lsm.btree.column.impls.lsm.LSMColumnBTree;
+import org.apache.hyracks.storage.am.lsm.btree.column.impls.lsm.LSMColumnBTreeWithBloomFilterDiskComponentFactory;
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTree;
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeFileManager;
+import org.apache.hyracks.storage.am.lsm.btree.tuples.LSMBTreeCopyTupleWriterFactory;
+import org.apache.hyracks.storage.am.lsm.btree.tuples.LSMBTreeTupleWriterFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMPageWriteCallbackFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
+import org.apache.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.util.trace.ITracer;
+
+public class LSMColumnBTreeUtil {
+
+ public static LSMBTree createLSMTree(IIOManager ioManager, List<IVirtualBufferCache> virtualBufferCaches,
+ FileReference file, IBufferCache diskBufferCache, ITypeTraits[] typeTraits,
+ IBinaryComparatorFactory[] cmpFactories, int[] bloomFilterKeyFields, double bloomFilterFalsePositiveRate,
+ ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
+ ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
+ int[] btreeFields, IMetadataPageManagerFactory freePageManagerFactory, boolean updateAware, ITracer tracer,
+ ICompressorDecompressorFactory compressorDecompressorFactory, ITypeTraits nullTypeTraits,
+ INullIntrospector nullIntrospector, IColumnManagerFactory columnManagerFactory)
+ throws HyracksDataException {
+
+ //Tuple writers
+ LSMBTreeTupleWriterFactory insertTupleWriterFactory = new LSMBTreeTupleWriterFactory(typeTraits,
+ cmpFactories.length, false, updateAware, nullTypeTraits, nullIntrospector);
+ LSMBTreeTupleWriterFactory deleteTupleWriterFactory = new LSMBTreeTupleWriterFactory(typeTraits,
+ cmpFactories.length, true, updateAware, nullTypeTraits, nullIntrospector);
+ LSMBTreeCopyTupleWriterFactory copyTupleWriterFactory = new LSMBTreeCopyTupleWriterFactory(typeTraits,
+ cmpFactories.length, updateAware, nullTypeTraits, nullIntrospector);
+ LSMBTreeTupleWriterFactory bulkLoadTupleWriterFactory = new LSMBTreeTupleWriterFactory(typeTraits,
+ cmpFactories.length, false, updateAware, nullTypeTraits, nullIntrospector);
+
+ //Leaf frames
+ ITreeIndexFrameFactory flushLeafFrameFactory = new ColumnBTreeLeafFrameFactory(copyTupleWriterFactory,
+ columnManagerFactory.getFlushColumnTupleReaderWriterFactory());
+ ITreeIndexFrameFactory mergeLeafFrameFactory = new ColumnBTreeLeafFrameFactory(copyTupleWriterFactory,
+ columnManagerFactory.createMergeColumnTupleReaderWriterFactory());
+ ITreeIndexFrameFactory bulkLoadLeafFrameFactory = new ColumnBTreeLeafFrameFactory(bulkLoadTupleWriterFactory,
+ columnManagerFactory.getLoadColumnTupleReaderWriterFactory());
+ ITreeIndexFrameFactory insertLeafFrameFactory = new BTreeNSMLeafFrameFactory(insertTupleWriterFactory);
+ ITreeIndexFrameFactory deleteLeafFrameFactory = new BTreeNSMLeafFrameFactory(deleteTupleWriterFactory);
+ ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(insertTupleWriterFactory);
+
+ //BTree factory
+ TreeIndexFactory<ColumnBTree> flushBTreeFactory = new ColumnBTreeFactory(ioManager, diskBufferCache,
+ freePageManagerFactory, interiorFrameFactory, flushLeafFrameFactory, cmpFactories, typeTraits.length);
+ TreeIndexFactory<ColumnBTree> mergeBTreeFactory = new ColumnBTreeFactory(ioManager, diskBufferCache,
+ freePageManagerFactory, interiorFrameFactory, mergeLeafFrameFactory, cmpFactories, typeTraits.length);
+ TreeIndexFactory<ColumnBTree> bulkloadBTreeFactory =
+ new ColumnBTreeFactory(ioManager, diskBufferCache, freePageManagerFactory, interiorFrameFactory,
+ bulkLoadLeafFrameFactory, cmpFactories, typeTraits.length);
+
+ ILSMIndexFileManager fileNameManager =
+ new LSMBTreeFileManager(ioManager, file, flushBTreeFactory, true, compressorDecompressorFactory);
+
+ BloomFilterFactory bloomFilterFactory = new BloomFilterFactory(diskBufferCache, bloomFilterKeyFields);
+ ILSMDiskComponentFactory flushComponentFactory =
+ new LSMColumnBTreeWithBloomFilterDiskComponentFactory(flushBTreeFactory, bloomFilterFactory);
+ ILSMDiskComponentFactory mergeComponentFactory =
+ new LSMColumnBTreeWithBloomFilterDiskComponentFactory(mergeBTreeFactory, bloomFilterFactory);
+ ILSMDiskComponentFactory bulkLoadComponentFactory =
+ new LSMColumnBTreeWithBloomFilterDiskComponentFactory(bulkloadBTreeFactory, bloomFilterFactory);
+
+ return new LSMColumnBTree(ioManager, virtualBufferCaches, interiorFrameFactory, insertLeafFrameFactory,
+ deleteLeafFrameFactory, diskBufferCache, fileNameManager, flushComponentFactory, mergeComponentFactory,
+ bulkLoadComponentFactory, bloomFilterFalsePositiveRate, typeTraits.length, cmpFactories, mergePolicy,
+ opTracker, ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, btreeFields, tracer,
+ columnManagerFactory.createColumnManager());
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResource.java
index e16baf9..ee4c4f7 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResource.java
@@ -56,19 +56,21 @@
ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
IMetadataPageManagerFactory metadataPageManagerFactory,
ILSMIOOperationSchedulerProvider ioSchedulerProvider, boolean durable, boolean hasBloomFilter,
- ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
+ ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector,
+ boolean isSecondaryNoIncrementalMaintenance) {
super(typeTraits, cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate, isPrimary, path,
storageManager, mergePolicyFactory, mergePolicyProperties, filterTypeTraits, filterCmpFactories,
btreeFields, filterFields, opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory,
metadataPageManagerFactory, null, ioSchedulerProvider, durable,
- NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter, nullTypeTraits, nullIntrospector);
+ NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter, nullTypeTraits, nullIntrospector,
+ isSecondaryNoIncrementalMaintenance);
}
private ExternalBTreeLocalResource(IPersistedResourceRegistry registry, JsonNode json, int[] bloomFilterKeyFields,
- double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields, boolean hasBloomFilter)
- throws HyracksDataException {
+ double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields, boolean hasBloomFilter,
+ boolean isSecondaryNoIncrementalMaintenance) throws HyracksDataException {
super(registry, json, bloomFilterKeyFields, bloomFilterFalsePositiveRate, isPrimary, btreeFields,
- NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter);
+ NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter, isSecondaryNoIncrementalMaintenance);
}
@Override
@@ -97,6 +99,6 @@
LSMBTreeLocalResource lsmBtree = (LSMBTreeLocalResource) LSMBTreeLocalResource.fromJson(registry, json);
return new ExternalBTreeLocalResource(registry, json, lsmBtree.bloomFilterKeyFields,
lsmBtree.bloomFilterFalsePositiveRate, lsmBtree.isPrimary, lsmBtree.btreeFields,
- lsmBtree.hasBloomFilter);
+ lsmBtree.hasBloomFilter, lsmBtree.isSecondaryNoIncrementalMaintenance);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResourceFactory.java
index e4c29ba..3a28057 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResourceFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResourceFactory.java
@@ -46,13 +46,13 @@
IMetadataPageManagerFactory metadataPageManagerFactory,
ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMMergePolicyFactory mergePolicyFactory,
Map<String, String> mergePolicyProperties, boolean durable, int[] bloomFilterKeyFields,
- double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields, boolean hasBloomFilter,
- ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
+ double bloomFilterFalsePositiveRate, int[] btreeFields, boolean hasBloomFilter, ITypeTraits nullTypeTraits,
+ INullIntrospector nullIntrospector) {
super(storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
opTrackerFactory, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory, null,
ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, bloomFilterKeyFields,
- bloomFilterFalsePositiveRate, isPrimary, btreeFields, NoOpCompressorDecompressorFactory.INSTANCE,
- hasBloomFilter, nullTypeTraits, nullIntrospector);
+ bloomFilterFalsePositiveRate, false, btreeFields, NoOpCompressorDecompressorFactory.INSTANCE,
+ hasBloomFilter, nullTypeTraits, nullIntrospector, true);
}
@Override
@@ -61,6 +61,7 @@
bloomFilterFalsePositiveRate, isPrimary, fileRef.getRelativePath(), storageManager, mergePolicyFactory,
mergePolicyProperties, filterTypeTraits, filterCmpFactories, btreeFields, bloomFilterKeyFields,
opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
- ioSchedulerProvider, durable, hasBloomFilter, nullTypeTraits, nullIntrospector);
+ ioSchedulerProvider, durable, hasBloomFilter, nullTypeTraits, nullIntrospector,
+ isSecondaryNoIncrementalMaintenance);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResource.java
index 863ae56..f2c5263 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResource.java
@@ -59,19 +59,21 @@
ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
IMetadataPageManagerFactory metadataPageManagerFactory,
ILSMIOOperationSchedulerProvider ioSchedulerProvider, boolean durable, boolean hasBloomFilter,
- ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
+ ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector,
+ boolean isSecondaryNoIncrementalMaintenance) {
super(typeTraits, cmpFactories, buddyBtreeFields, bloomFilterFalsePositiveRate, isPrimary, path, storageManager,
mergePolicyFactory, mergePolicyProperties, filterTypeTraits, filterCmpFactories, btreeFields,
filterFields, opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory,
metadataPageManagerFactory, null, ioSchedulerProvider, durable,
- NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter, nullTypeTraits, nullIntrospector);
+ NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter, nullTypeTraits, nullIntrospector,
+ isSecondaryNoIncrementalMaintenance);
}
private ExternalBTreeWithBuddyLocalResource(IPersistedResourceRegistry registry, JsonNode json,
int[] bloomFilterKeyFields, double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields,
- boolean hasBloomFilter) throws HyracksDataException {
+ boolean hasBloomFilter, boolean isSecondaryNoIncrementalMaintenance) throws HyracksDataException {
super(registry, json, bloomFilterKeyFields, bloomFilterFalsePositiveRate, isPrimary, btreeFields,
- NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter);
+ NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter, isSecondaryNoIncrementalMaintenance);
}
@Override
@@ -100,6 +102,6 @@
LSMBTreeLocalResource lsmBtree = (LSMBTreeLocalResource) LSMBTreeLocalResource.fromJson(registry, json);
return new ExternalBTreeWithBuddyLocalResource(registry, json, lsmBtree.bloomFilterKeyFields,
lsmBtree.bloomFilterFalsePositiveRate, lsmBtree.isPrimary, lsmBtree.btreeFields,
- lsmBtree.hasBloomFilter);
+ lsmBtree.hasBloomFilter, lsmBtree.isSecondaryNoIncrementalMaintenance);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResourceFactory.java
index 39c8aed..995a23b 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResourceFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResourceFactory.java
@@ -46,13 +46,13 @@
IMetadataPageManagerFactory metadataPageManagerFactory,
ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMMergePolicyFactory mergePolicyFactory,
Map<String, String> mergePolicyProperties, boolean durable, int[] buddyBtreeFields,
- double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields, boolean hasBloomFilter,
- ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
+ double bloomFilterFalsePositiveRate, int[] btreeFields, boolean hasBloomFilter, ITypeTraits nullTypeTraits,
+ INullIntrospector nullIntrospector) {
super(storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory, null,
ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, buddyBtreeFields,
- bloomFilterFalsePositiveRate, isPrimary, btreeFields, NoOpCompressorDecompressorFactory.INSTANCE,
- hasBloomFilter, nullTypeTraits, nullIntrospector);
+ bloomFilterFalsePositiveRate, false, btreeFields, NoOpCompressorDecompressorFactory.INSTANCE,
+ hasBloomFilter, nullTypeTraits, nullIntrospector, true);
}
@Override
@@ -61,6 +61,7 @@
bloomFilterFalsePositiveRate, isPrimary, fileRef.getRelativePath(), storageManager, mergePolicyFactory,
mergePolicyProperties, filterTypeTraits, filterCmpFactories, btreeFields, filterFields,
opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
- ioSchedulerProvider, durable, hasBloomFilter, nullTypeTraits, nullIntrospector);
+ ioSchedulerProvider, durable, hasBloomFilter, nullTypeTraits, nullIntrospector,
+ isSecondaryNoIncrementalMaintenance);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorDescriptor.java
index 3800d17..3e03e5c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorDescriptor.java
@@ -29,6 +29,7 @@
import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import org.apache.hyracks.storage.common.projection.ITupleProjectorFactory;
public class LSMBTreeBatchPointSearchOperatorDescriptor extends BTreeSearchOperatorDescriptor {
private static final long serialVersionUID = 1L;
@@ -38,10 +39,11 @@
IIndexDataflowHelperFactory indexHelperFactory, boolean retainInput, boolean retainMissing,
IMissingWriterFactory missingWriterFactory, ISearchOperationCallbackFactory searchCallbackFactory,
int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, ITupleFilterFactory tupleFilterFactory,
- long outputLimit) {
+ long outputLimit, ITupleProjectorFactory tupleProjectorFactory) {
super(spec, outRecDesc, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory,
retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, minFilterFieldIndexes,
- maxFilterFieldIndexes, false, null, tupleFilterFactory, outputLimit, false, null, null);
+ maxFilterFieldIndexes, false, null, tupleFilterFactory, outputLimit, false, null, null,
+ tupleProjectorFactory);
}
@Override
@@ -51,7 +53,7 @@
recordDescProvider.getInputRecordDescriptor(getActivityId(), 0), lowKeyFields, highKeyFields,
lowKeyInclusive, highKeyInclusive, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, tupleFilterFactory,
- outputLimit);
+ outputLimit, tupleProjectorFactory);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorNodePushable.java
index 30813ef..f6f97b7 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorNodePushable.java
@@ -34,10 +34,12 @@
import org.apache.hyracks.storage.am.common.api.ITreeIndex;
import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTree;
import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeBatchPointSearchCursor;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
import org.apache.hyracks.storage.common.IIndexCursor;
import org.apache.hyracks.storage.common.ISearchPredicate;
+import org.apache.hyracks.storage.common.projection.ITupleProjectorFactory;
public class LSMBTreeBatchPointSearchOperatorNodePushable extends BTreeSearchOperatorNodePushable {
@@ -48,18 +50,19 @@
boolean highKeyInclusive, int[] minFilterKeyFields, int[] maxFilterKeyFields,
IIndexDataflowHelperFactory indexHelperFactory, boolean retainInput, boolean retainMissing,
IMissingWriterFactory missingWriterFactory, ISearchOperationCallbackFactory searchCallbackFactory,
- ITupleFilterFactory tupleFilterFactory, long outputLimit) throws HyracksDataException {
+ ITupleFilterFactory tupleFilterFactory, long outputLimit, ITupleProjectorFactory tupleProjectorFactory)
+ throws HyracksDataException {
super(ctx, partition, inputRecDesc, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive,
minFilterKeyFields, maxFilterKeyFields, indexHelperFactory, retainInput, retainMissing,
missingWriterFactory, searchCallbackFactory, false, null, tupleFilterFactory, outputLimit, false, null,
- null);
+ null, tupleProjectorFactory);
this.keyFields = lowKeyFields;
}
@Override
protected IIndexCursor createCursor() throws HyracksDataException {
ILSMIndexAccessor lsmAccessor = (ILSMIndexAccessor) indexAccessor;
- return new LSMBTreeBatchPointSearchCursor(lsmAccessor.getOpContext());
+ return ((LSMBTree) index).createBatchPointSearchCursor(lsmAccessor.getOpContext());
}
@Override
@@ -123,7 +126,7 @@
break;
}
}
- stats.getTupleCounter().update(matchingTupleCount);
+ stats.getInputTupleCounter().update(matchingTupleCount);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResource.java
index 656ae02..a7e433c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResource.java
@@ -59,6 +59,7 @@
protected final boolean isPrimary;
protected final int[] btreeFields;
protected final ICompressorDecompressorFactory compressorDecompressorFactory;
+ protected final boolean isSecondaryNoIncrementalMaintenance;
public LSMBTreeLocalResource(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories,
int[] bloomFilterKeyFields, double bloomFilterFalsePositiveRate, boolean isPrimary, String path,
@@ -70,7 +71,8 @@
IMetadataPageManagerFactory metadataPageManagerFactory, IVirtualBufferCacheProvider vbcProvider,
ILSMIOOperationSchedulerProvider ioSchedulerProvider, boolean durable,
ICompressorDecompressorFactory compressorDecompressorFactory, boolean hasBloomFilter,
- ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
+ ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector,
+ boolean isSecondaryNoIncrementalMaintenance) {
super(path, storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, nullTypeTraits,
@@ -81,12 +83,13 @@
this.btreeFields = btreeFields;
this.compressorDecompressorFactory = compressorDecompressorFactory;
this.hasBloomFilter = hasBloomFilter;
+ this.isSecondaryNoIncrementalMaintenance = isSecondaryNoIncrementalMaintenance;
}
protected LSMBTreeLocalResource(IPersistedResourceRegistry registry, JsonNode json, int[] bloomFilterKeyFields,
double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields,
- ICompressorDecompressorFactory compressorDecompressorFactory, boolean hasBloomFilter)
- throws HyracksDataException {
+ ICompressorDecompressorFactory compressorDecompressorFactory, boolean hasBloomFilter,
+ boolean isSecondaryNoIncrementalMaintenance) throws HyracksDataException {
super(registry, json);
this.bloomFilterKeyFields = bloomFilterKeyFields;
this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
@@ -94,6 +97,7 @@
this.btreeFields = btreeFields;
this.compressorDecompressorFactory = compressorDecompressorFactory;
this.hasBloomFilter = hasBloomFilter;
+ this.isSecondaryNoIncrementalMaintenance = isSecondaryNoIncrementalMaintenance;
}
@Override
@@ -114,6 +118,10 @@
compressorDecompressorFactory, hasBloomFilter, nullTypeTraits, nullIntrospector);
}
+ public boolean isSecondaryNoIncrementalMaintenance() {
+ return isSecondaryNoIncrementalMaintenance;
+ }
+
@Override
public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
final ObjectNode jsonObject = registry.getClassIdentifier(getClass(), serialVersionUID);
@@ -131,8 +139,10 @@
final JsonNode compressorDecompressorNode = json.get("compressorDecompressorFactory");
final ICompressorDecompressorFactory compDecompFactory = (ICompressorDecompressorFactory) registry
.deserializeOrDefault(compressorDecompressorNode, NoOpCompressorDecompressorFactory.class);
+ boolean isSecondaryNoIncrementalMaintenance =
+ getOrDefaultBoolean(json, "isSecondaryNoIncrementalMaintenance", false);
return new LSMBTreeLocalResource(registry, json, bloomFilterKeyFields, bloomFilterFalsePositiveRate, isPrimary,
- btreeFields, compDecompFactory, hasBloomFilter);
+ btreeFields, compDecompFactory, hasBloomFilter, isSecondaryNoIncrementalMaintenance);
}
@Override
@@ -145,13 +155,16 @@
json.put("isPrimary", isPrimary);
json.putPOJO("btreeFields", btreeFields);
json.putPOJO("compressorDecompressorFactory", compressorDecompressorFactory.toJson(registry));
+ json.put("isSecondaryNoIncrementalMaintenance", isSecondaryNoIncrementalMaintenance);
}
private static boolean getOrDefaultHasBloomFilter(JsonNode json, boolean isPrimary) {
- if (json.has(HAS_BLOOM_FILTER_FIELD)) {
- return json.get(HAS_BLOOM_FILTER_FIELD).asBoolean();
- }
- // for backward compatibiliy, only primary indexes have bloom filters
- return isPrimary;
+ // for backward compatibility, only primary indexes have bloom filters
+ return getOrDefaultBoolean(json, HAS_BLOOM_FILTER_FIELD, isPrimary);
}
+
+ private static boolean getOrDefaultBoolean(JsonNode jsonNode, String fieldName, boolean defaultValue) {
+ return jsonNode.has(fieldName) ? jsonNode.get(fieldName).asBoolean() : defaultValue;
+ }
+
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResourceFactory.java
index e926a48..6695e90 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResourceFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResourceFactory.java
@@ -48,6 +48,7 @@
protected final boolean isPrimary;
protected final int[] btreeFields;
protected final ICompressorDecompressorFactory compressorDecompressorFactory;
+ protected final boolean isSecondaryNoIncrementalMaintenance;
public LSMBTreeLocalResourceFactory(IStorageManager storageManager, ITypeTraits[] typeTraits,
IBinaryComparatorFactory[] cmpFactories, ITypeTraits[] filterTypeTraits,
@@ -59,7 +60,8 @@
Map<String, String> mergePolicyProperties, boolean durable, int[] bloomFilterKeyFields,
double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields,
ICompressorDecompressorFactory compressorDecompressorFactory, boolean hasBloomFilter,
- ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
+ ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector,
+ boolean isSecondaryNoIncrementalMaintenance) {
super(storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
opTrackerFactory, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, nullTypeTraits,
@@ -70,6 +72,7 @@
this.isPrimary = isPrimary;
this.btreeFields = btreeFields;
this.compressorDecompressorFactory = compressorDecompressorFactory;
+ this.isSecondaryNoIncrementalMaintenance = isSecondaryNoIncrementalMaintenance;
}
@Override
@@ -78,7 +81,8 @@
isPrimary, fileRef.getRelativePath(), storageManager, mergePolicyFactory, mergePolicyProperties,
filterTypeTraits, filterCmpFactories, btreeFields, filterFields, opTrackerProvider, ioOpCallbackFactory,
pageWriteCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider, durable,
- compressorDecompressorFactory, hasBloomFilter, nullTypeTraits, nullIntrospector);
+ compressorDecompressorFactory, hasBloomFilter, nullTypeTraits, nullIntrospector,
+ isSecondaryNoIncrementalMaintenance);
}
private void readObject(java.io.ObjectInputStream in) throws IOException, ClassNotFoundException {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
index c3d1416..d2fbbef 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
@@ -341,7 +341,7 @@
try {
List<ILSMComponent> mergedComponents = mergeOp.getMergingComponents();
long numElements = getNumberOfElements(mergedComponents);
- mergedComponent = createDiskComponent(componentFactory, mergeOp.getTarget(), null,
+ mergedComponent = createDiskComponent(getMergeComponentFactory(), mergeOp.getTarget(), null,
mergeOp.getBloomFilterTarget(), true);
IPageWriteCallback pageWriteCallback = pageWriteCallbackFactory.createPageWriteCallback();
componentBulkLoader = mergedComponent.createBulkLoader(operation, 1.0f, false, numElements, false,
@@ -418,7 +418,7 @@
}
public ILSMIndexAccessor createAccessor(AbstractLSMIndexOperationContext opCtx) {
- return new LSMTreeIndexAccessor(getHarness(), opCtx, cursorFactory);
+ return new LSMTreeIndexAccessor(getHarness(), opCtx, getCursorFactory());
}
@Override
@@ -483,8 +483,28 @@
returnDeletedTuples = true;
}
IIndexCursorStats stats = new IndexCursorStats();
- LSMBTreeRangeSearchCursor cursor = new LSMBTreeRangeSearchCursor(opCtx, returnDeletedTuples, stats);
+ LSMBTreeRangeSearchCursor cursor = createCursor(opCtx, returnDeletedTuples, stats);
return new LSMBTreeMergeOperation(accessor, cursor, stats, mergeFileRefs.getInsertIndexFileReference(),
mergeFileRefs.getBloomFilterFileReference(), callback, getIndexIdentifier());
}
+
+ public LSMBTreeBatchPointSearchCursor createBatchPointSearchCursor(ILSMIndexOperationContext opCtx) {
+ return new LSMBTreeBatchPointSearchCursor(opCtx);
+ }
+
+ protected LSMBTreeRangeSearchCursor createCursor(AbstractLSMIndexOperationContext opCtx,
+ boolean returnDeletedTuples, IIndexCursorStats stats) {
+ return new LSMBTreeRangeSearchCursor(opCtx, returnDeletedTuples, stats);
+ }
+
+ /**
+ * @return Merge component factory (could be different from {@link #componentFactory}
+ */
+ protected ILSMDiskComponentFactory getMergeComponentFactory() {
+ return componentFactory;
+ }
+
+ protected ICursorFactory getCursorFactory() {
+ return cursorFactory;
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeBatchPointSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeBatchPointSearchCursor.java
index 8ab6fb1..73d06d2 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeBatchPointSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeBatchPointSearchCursor.java
@@ -21,14 +21,15 @@
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.btree.impls.BTree.BTreeAccessor;
import org.apache.hyracks.storage.am.btree.impls.BatchPredicate;
-import org.apache.hyracks.storage.am.btree.impls.DiskBTreePointSearchCursor;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexCursor;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent.LSMComponentType;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
/**
* This cursor performs point searches for each batch of search keys.
* Assumption: the search keys must be sorted into the increasing order.
- *
*/
public class LSMBTreeBatchPointSearchCursor extends LSMBTreePointSearchCursor {
@@ -42,7 +43,9 @@
while (!foundTuple && batchPred.hasNext()) {
batchPred.next();
if (foundIn >= 0) {
- btreeCursors[foundIn].close();
+ if (operationalComponents.get(foundIn).getType() == LSMComponentType.MEMORY) {
+ btreeCursors[foundIn].close();
+ }
foundIn = -1;
}
foundTuple = super.doHasNext();
@@ -56,6 +59,11 @@
}
@Override
+ protected ITreeIndexCursor createCursor(LSMComponentType type, BTreeAccessor btreeAccessor) {
+ return btreeAccessor.createPointCursor(false, true);
+ }
+
+ @Override
protected boolean isSearchCandidate(int componentIndex) throws HyracksDataException {
if (!super.isSearchCandidate(componentIndex)) {
return false;
@@ -68,21 +76,6 @@
maxFileterKey, opCtx.getFilterCmp());
}
- @Override
- protected void closeCursors() throws HyracksDataException {
- super.closeCursors();
- if (btreeCursors != null) {
- // clear search states of btree cursors
- for (int i = 0; i < numBTrees; ++i) {
- if (btreeCursors[i] != null) {
- if (btreeCursors[i] instanceof DiskBTreePointSearchCursor) {
- ((DiskBTreePointSearchCursor) btreeCursors[i]).clearSearchState();
- }
- }
- }
- }
- }
-
public int getKeyIndex() {
return ((BatchPredicate) predicate).getKeyIndex();
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java
index 1312e30..a00e10e 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java
@@ -43,7 +43,7 @@
import org.apache.hyracks.storage.common.MultiComparator;
import org.apache.hyracks.util.trace.ITracer;
-public final class LSMBTreeOpContext extends AbstractLSMIndexOperationContext {
+public class LSMBTreeOpContext extends AbstractLSMIndexOperationContext {
/*
* Finals
@@ -74,9 +74,9 @@
IBinaryComparatorFactory[] filterCmpFactories, ITracer tracer) {
super(index, btreeFields, filterFields, filterCmpFactories, searchCallback, modificationCallback, tracer);
LSMBTreeMemoryComponent c = (LSMBTreeMemoryComponent) mutableComponents.get(0);
- IBinaryComparatorFactory cmpFactories[] = c.getIndex().getComparatorFactories();
+ IBinaryComparatorFactory[] cmpFactories = c.getIndex().getComparatorFactories();
if (cmpFactories[0] != null) {
- this.cmp = MultiComparator.create(c.getIndex().getComparatorFactories());
+ this.cmp = createMultiComparator(c.getIndex().getComparatorFactories());
} else {
this.cmp = null;
}
@@ -112,6 +112,10 @@
insertSearchCursor = new LSMBTreePointSearchCursor(this);
}
+ protected MultiComparator createMultiComparator(IBinaryComparatorFactory[] cmpFactories) {
+ return MultiComparator.create(cmpFactories);
+ }
+
@Override
public void setOperation(IndexOperation newOp) {
reset();
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java
index d4903d9..9740bc4 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java
@@ -199,7 +199,8 @@
for (int i = 0; i < numBTrees; i++) {
ILSMComponent component = operationalComponents.get(i);
BTree btree = (BTree) component.getIndex();
- if (component.getType() == LSMComponentType.MEMORY) {
+ LSMComponentType type = component.getType();
+ if (type == LSMComponentType.MEMORY) {
includeMutableComponent = true;
if (bloomFilters[i] != null) {
destroyAndNullifyCursorAtIndex(i);
@@ -212,8 +213,8 @@
}
if (btreeAccessors[i] == null) {
- btreeAccessors[i] = btree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
- btreeCursors[i] = btreeAccessors[i].createPointCursor(false, false);
+ btreeAccessors[i] = createAccessor(type, btree, i);
+ btreeCursors[i] = createCursor(type, btreeAccessors[i]);
} else {
// re-use
btreeAccessors[i].reset(btree, NoOpIndexAccessParameters.INSTANCE);
@@ -225,6 +226,14 @@
hashComputed = false;
}
+ protected BTreeAccessor createAccessor(LSMComponentType type, BTree btree, int i) throws HyracksDataException {
+ return btree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
+ }
+
+ protected ITreeIndexCursor createCursor(LSMComponentType type, BTreeAccessor btreeAccessor) {
+ return btreeAccessor.createPointCursor(false, false);
+ }
+
private void destroyAndNullifyCursorAtIndex(int i) throws HyracksDataException {
// component at location i was a disk component before, and is now a memory component, or vise versa
bloomFilters[i] = null;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java
index 2c5fb50..968416c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java
@@ -49,7 +49,7 @@
private final RangePredicate reusablePred;
private ISearchOperationCallback searchCallback;
private BTreeAccessor[] btreeAccessors;
- private boolean[] isMemoryComponent;
+ protected boolean[] isMemoryComponent;
private ArrayTupleBuilder tupleBuilder;
private boolean canCallProceed = true;
private boolean resultOfSearchCallbackProceed = false;
@@ -149,6 +149,7 @@
// There are no more elements in the memory component.. can safely skip locking for the
// remaining operations
includeMutableComponent = false;
+ excludeMemoryComponent();
}
}
} else {
@@ -180,6 +181,7 @@
// the tree of head tuple
// the head element of PQ is useless now
PriorityQueueElement e = outputPriorityQueue.poll();
+ markAsDeleted(e);
pushIntoQueueFromCursorAndReplaceThisElement(e);
} else {
// If the previous tuple and the head tuple are different
@@ -200,6 +202,14 @@
}
+ protected void excludeMemoryComponent() {
+ //NoOp
+ }
+
+ protected void markAsDeleted(PriorityQueueElement e) throws HyracksDataException {
+ //NoOp
+ }
+
private void pushOutputElementIntoQueueIfNeeded() throws HyracksDataException {
if (needPushElementIntoQueue) {
pushIntoQueueFromCursorAndReplaceThisElement(outputElement);
@@ -250,7 +260,7 @@
}
}
- private int replaceFrom() throws HyracksDataException {
+ protected int replaceFrom() throws HyracksDataException {
int replaceFrom = -1;
if (!switchPossible) {
return replaceFrom;
@@ -386,20 +396,21 @@
}
for (int i = 0; i < numBTrees; i++) {
ILSMComponent component = operationalComponents.get(i);
+ LSMComponentType type = component.getType();
BTree btree;
if (component.getType() == LSMComponentType.MEMORY) {
includeMutableComponent = true;
}
btree = (BTree) component.getIndex();
if (btreeAccessors[i] == null || destroyIncompatible(component, i)) {
- btreeAccessors[i] = btree.createAccessor(iap);
- rangeCursors[i] = btreeAccessors[i].createSearchCursor(false);
+ btreeAccessors[i] = createAccessor(type, btree, i);
+ rangeCursors[i] = createCursor(type, btreeAccessors[i]);
} else {
// re-use
btreeAccessors[i].reset(btree, iap);
rangeCursors[i].close();
}
- isMemoryComponent[i] = component.getType() == LSMComponentType.MEMORY;
+ isMemoryComponent[i] = type == LSMComponentType.MEMORY;
}
IndexCursorUtils.open(btreeAccessors, rangeCursors, searchPred);
try {
@@ -433,4 +444,12 @@
return resultOfSearchCallbackProceed;
}
+ protected BTreeAccessor createAccessor(LSMComponentType type, BTree btree, int index) throws HyracksDataException {
+ return btree.createAccessor(iap);
+ }
+
+ protected IIndexCursor createCursor(LSMComponentType type, BTreeAccessor accessor) {
+ return accessor.createSearchCursor(false);
+ }
+
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeSearchCursor.java
index efacad1..aa72267 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeSearchCursor.java
@@ -46,6 +46,13 @@
scanCursor = new LSMBTreeDiskComponentScanCursor(opCtx);
}
+ protected LSMBTreeSearchCursor(LSMBTreePointSearchCursor pointCursor, LSMBTreeRangeSearchCursor rangeCursor,
+ LSMBTreeDiskComponentScanCursor scanCursor) {
+ this.pointCursor = pointCursor;
+ this.rangeCursor = rangeCursor;
+ this.scanCursor = scanCursor;
+ }
+
@Override
public void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
LSMBTreeCursorInitialState lsmInitialState = (LSMBTreeCursorInitialState) initialState;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexBulkLoader.java
index 13a0e27..acb84e1 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexBulkLoader.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexBulkLoader.java
@@ -20,7 +20,7 @@
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
-import org.apache.hyracks.storage.am.common.impls.AbstractTreeIndex.AbstractTreeIndexBulkLoader;
+import org.apache.hyracks.storage.am.common.impls.AbstractTreeIndexBulkLoader;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMTreeTupleWriter;
import org.apache.hyracks.storage.common.IIndexBulkLoader;
import org.apache.hyracks.storage.common.buffercache.ICachedPage;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java
index 27875c0..2c97221 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java
@@ -236,7 +236,7 @@
} else {
// If the previous tuple and the head tuple are different
// the info of previous tuple is useless
- if (needPushElementIntoQueue == true) {
+ if (needPushElementIntoQueue) {
pushIntoQueueFromCursorAndReplaceThisElement(outputElement);
needPushElementIntoQueue = false;
}
@@ -304,6 +304,10 @@
public MultiComparator getMultiComparator() {
return cmp;
}
+
+ public void setMultiComparator(MultiComparator cmp) {
+ this.cmp = cmp;
+ }
}
protected void setPriorityQueueComparator() {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeAbstractCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeAbstractCursor.java
index b7eb115..11385de 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeAbstractCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeAbstractCursor.java
@@ -28,6 +28,7 @@
import org.apache.hyracks.storage.am.btree.impls.BTreeRangeSearchCursor;
import org.apache.hyracks.storage.am.btree.impls.RangePredicate;
import org.apache.hyracks.storage.am.common.api.ILSMIndexCursor;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexCursor;
import org.apache.hyracks.storage.am.common.impls.IndexAccessParameters;
import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
@@ -50,7 +51,7 @@
protected boolean open;
protected RTreeSearchCursor[] rtreeCursors;
- protected BTreeRangeSearchCursor[] btreeCursors;
+ protected ITreeIndexCursor[] btreeCursors;
protected RTreeAccessor[] rtreeAccessors;
protected BTreeAccessor[] btreeAccessors;
protected BloomFilter[] bloomFilters;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java
index 8e5cb35..729ca74 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java
@@ -142,7 +142,8 @@
bTreeTupleSorter = new TreeTupleSorter(flushingComponent.getBuddyIndex().getFileId(), linearizerArray,
btreeLeafFrameFactory.createFrame(), btreeLeafFrameFactory.createFrame(),
flushingComponent.getBuddyIndex().getBufferCache(), comparatorFields);
- BTreeRangeSearchCursor btreeScanCursor = memBTreeAccessor.createSearchCursor(false);
+ BTreeRangeSearchCursor btreeScanCursor =
+ (BTreeRangeSearchCursor) memBTreeAccessor.createSearchCursor(false);
try {
isEmpty = true;
memBTreeAccessor.search(btreeScanCursor, btreeNullPredicate);
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java
index a8a4252..e1c6f5b 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java
@@ -28,6 +28,7 @@
import org.apache.hyracks.storage.am.common.api.ITreeIndex;
import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
import org.apache.hyracks.storage.am.common.dataflow.IndexSearchOperatorNodePushable;
+import org.apache.hyracks.storage.am.common.impls.DefaultTupleProjectorFactory;
import org.apache.hyracks.storage.am.rtree.impls.SearchPredicate;
import org.apache.hyracks.storage.am.rtree.util.RTreeUtils;
import org.apache.hyracks.storage.common.IIndexAccessParameters;
@@ -59,7 +60,7 @@
super(ctx, inputRecDesc, partition, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter,
nonFilterWriterFactory, null, -1, appendOpCallbackProceedResult, searchCallbackProceedResultFalseValue,
- searchCallbackProceedResultTrueValue);
+ searchCallbackProceedResultTrueValue, DefaultTupleProjectorFactory.INSTANCE);
if (keyFields != null && keyFields.length > 0) {
searchKey = new PermutingFrameTupleReference();
searchKey.setFieldPermutation(keyFields);
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTree.java
index 7e8f249..d85200f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTree.java
@@ -41,6 +41,7 @@
import org.apache.hyracks.storage.am.common.frames.AbstractSlotManager;
import org.apache.hyracks.storage.am.common.frames.FrameOpSpaceStatus;
import org.apache.hyracks.storage.am.common.impls.AbstractTreeIndex;
+import org.apache.hyracks.storage.am.common.impls.AbstractTreeIndexBulkLoader;
import org.apache.hyracks.storage.am.common.impls.NodeFrontier;
import org.apache.hyracks.storage.am.common.impls.TreeIndexDiskOrderScanCursor;
import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
@@ -902,7 +903,7 @@
return new RTreeBulkLoader(fillFactor, callback);
}
- public class RTreeBulkLoader extends AbstractTreeIndex.AbstractTreeIndexBulkLoader {
+ public class RTreeBulkLoader extends AbstractTreeIndexBulkLoader {
ITreeIndexFrame lowerFrame, prevInteriorFrame;
RTreeTypeAwareTupleWriter interiorFrameTupleWriter =
((RTreeTypeAwareTupleWriter) interiorFrame.getTupleWriter());
@@ -911,7 +912,7 @@
List<Integer> prevNodeFrontierPages = new ArrayList<>();
public RTreeBulkLoader(float fillFactor, IPageWriteCallback callback) throws HyracksDataException {
- super(fillFactor, callback);
+ super(fillFactor, callback, RTree.this);
prevInteriorFrame = interiorFrameFactory.createFrame();
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndexAccessParameters.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndexAccessParameters.java
index a3745d0..8f553d9 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndexAccessParameters.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndexAccessParameters.java
@@ -40,4 +40,14 @@
* Gets additional parameters.
*/
Map<String, Object> getParameters();
+
+ /**
+ * Gets a parameter.
+ *
+ * @param key of a parameter
+ * @param clazz used to explicitly cast the requested parameter to the required type
+ * @param <T> the required type
+ * @return the requested parameter
+ */
+ <T> T getParameter(String key, Class<T> clazz);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java
index f1fe86f..5578d27 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java
@@ -552,6 +552,10 @@
BufferedFileHandle fInfo = getFileHandle(cPage);
cPage.buffer.clear();
fInfo.read(cPage);
+ final IThreadStats threadStats = statsSubscribers.get(Thread.currentThread());
+ if (threadStats != null) {
+ threadStats.coldRead();
+ }
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/projection/ITupleProjector.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/projection/ITupleProjector.java
new file mode 100644
index 0000000..ba23e30
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/projection/ITupleProjector.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.common.projection;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+public interface ITupleProjector {
+ ITupleReference project(ITupleReference tuple, DataOutput dos, ArrayTupleBuilder tb) throws IOException;
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/projection/ITupleProjectorFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/projection/ITupleProjectorFactory.java
new file mode 100644
index 0000000..ff9ecf9
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/projection/ITupleProjectorFactory.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.common.projection;
+
+import java.io.Serializable;
+
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * Tuple projector allows the data source to project the values needed before it pushed up
+ * to the upper operator.
+ */
+public interface ITupleProjectorFactory extends Serializable {
+ ITupleProjector createTupleProjector(IHyracksTaskContext context) throws HyracksDataException;
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/LicensingTestBase.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/LicensingTestBase.java
index 9e52785..283c12c 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/LicensingTestBase.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/LicensingTestBase.java
@@ -22,7 +22,6 @@
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
-import java.util.Arrays;
import java.util.List;
import org.apache.commons.io.FileUtils;
@@ -47,8 +46,7 @@
}
installerDir = FileUtil.joinPath(targetDir, list[0], topLevel[0]);
Assert.assertNotNull("installerDir", list);
- Assert.assertFalse("Ambiguous install dir (" + pattern + "): " + Arrays.toString(topLevel),
- list.length > 1);
+ Assert.assertFalse("Ambiguous install dir (" + pattern + "): " + list, list.length > 1);
Assert.assertEquals("Can't find install dir (" + pattern + ")", 1, topLevel.length);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestTaskContext.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestTaskContext.java
index dcb85f6..c73ebee 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestTaskContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestTaskContext.java
@@ -49,15 +49,17 @@
public class TestTaskContext implements IHyracksTaskContext {
private final TestJobletContext jobletContext;
private final TaskAttemptId taskId;
+ private int partitionCount;
private WorkspaceFileFactory fileFactory;
private Map<Object, IStateObject> stateObjectMap = new HashMap<>();
private Object sharedObject;
private final IStatsCollector statsCollector = new StatsCollector();
private final ThreadStats threadStats = new ThreadStats();
- public TestTaskContext(TestJobletContext jobletContext, TaskAttemptId taskId) {
+ public TestTaskContext(TestJobletContext jobletContext, TaskAttemptId taskId, int partitionCount) {
this.jobletContext = jobletContext;
this.taskId = taskId;
+ this.partitionCount = partitionCount;
fileFactory = new WorkspaceFileFactory(this, getIoManager());
}
@@ -129,6 +131,11 @@
}
@Override
+ public int getPartitionCount() {
+ return partitionCount;
+ }
+
+ @Override
public synchronized void setStateObject(IStateObject taskState) {
stateObjectMap.put(taskState.getId(), taskState);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestUtils.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestUtils.java
index 3f78234..2348a1a 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestUtils.java
@@ -38,7 +38,7 @@
import org.apache.hyracks.api.dataflow.TaskId;
import org.apache.hyracks.api.exceptions.HyracksException;
import org.apache.hyracks.api.exceptions.IWarningCollector;
-import org.apache.hyracks.api.exceptions.Warning;
+import org.apache.hyracks.api.exceptions.NoOpWarningCollector;
import org.apache.hyracks.api.io.IODeviceHandle;
import org.apache.hyracks.api.job.JobId;
import org.apache.hyracks.api.util.CleanupUtils;
@@ -52,22 +52,7 @@
public class TestUtils {
private static final int DEFAULT_FRAME_SIZE = 32768;
- public static final IWarningCollector NOOP_WARNING_COLLECTOR = new IWarningCollector() {
- @Override
- public void warn(Warning warning) {
- // no-op
- }
-
- @Override
- public boolean shouldWarn() {
- return false;
- }
-
- @Override
- public long getTotalWarningsCount() {
- return 0;
- }
- };
+ public static final IWarningCollector NOOP_WARNING_COLLECTOR = NoOpWarningCollector.INSTANCE;
public static IHyracksTaskContext createHyracksTask() {
return create(DEFAULT_FRAME_SIZE);
@@ -91,7 +76,7 @@
INCServiceContext serviceCtx = new TestNCServiceContext(ioManager, null);
TestJobletContext jobletCtx = new TestJobletContext(frameSize, serviceCtx, new JobId(0));
TaskAttemptId tid = new TaskAttemptId(new TaskId(new ActivityId(new OperatorDescriptorId(0), 0), 0), 0);
- IHyracksTaskContext taskCtx = new TestTaskContext(jobletCtx, tid);
+ IHyracksTaskContext taskCtx = new TestTaskContext(jobletCtx, tid, 1);
return taskCtx;
} catch (HyracksException e) {
throw new RuntimeException(e);
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResource.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResource.java
index 8e2d6dd..c7d3dff 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResource.java
@@ -56,19 +56,21 @@
ILSMOperationTrackerFactory opTrackerProvider, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
IMetadataPageManagerFactory metadataPageManagerFactory, IVirtualBufferCacheProvider vbcProvider,
- ILSMIOOperationSchedulerProvider ioSchedulerProvider, boolean durable, boolean hasBloomFilter) {
+ ILSMIOOperationSchedulerProvider ioSchedulerProvider, boolean durable, boolean hasBloomFilter,
+ boolean isSecondaryNoIncrementalMaintenance) {
super(typeTraits, cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate, isPrimary, path,
storageManager, mergePolicyFactory, mergePolicyProperties, filterTypeTraits, filterCmpFactories,
btreeFields, filterFields, opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory,
metadataPageManagerFactory, vbcProvider, ioSchedulerProvider, durable,
- NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter, null, null);
+ NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter, null, null,
+ isSecondaryNoIncrementalMaintenance);
}
protected TestLsmBtreeLocalResource(IPersistedResourceRegistry registry, JsonNode json, int[] bloomFilterKeyFields,
- double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields, boolean hasBloomFilter)
- throws HyracksDataException {
+ double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields, boolean hasBloomFilter,
+ boolean isSecondaryNoIncrementalMaintenance) throws HyracksDataException {
super(registry, json, bloomFilterKeyFields, bloomFilterFalsePositiveRate, isPrimary, btreeFields,
- NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter);
+ NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter, isSecondaryNoIncrementalMaintenance);
}
@Override
@@ -107,7 +109,8 @@
final boolean isPrimary = json.get("isPrimary").asBoolean();
final boolean hasBloomFilter = json.get("hasBloomFilter").asBoolean();
final int[] btreeFields = OBJECT_MAPPER.convertValue(json.get("btreeFields"), int[].class);
+ boolean isSecondaryNoIncrementalMaintenance = json.get("isSecondaryNoIncrementalMaintenance").asBoolean();
return new TestLsmBtreeLocalResource(registry, json, bloomFilterKeyFields, bloomFilterFalsePositiveRate,
- isPrimary, btreeFields, hasBloomFilter);
+ isPrimary, btreeFields, hasBloomFilter, isSecondaryNoIncrementalMaintenance);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResourceFactory.java
index 7d9b24a..48adf91 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResourceFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResourceFactory.java
@@ -46,12 +46,14 @@
IMetadataPageManagerFactory metadataPageManagerFactory, IVirtualBufferCacheProvider vbcProvider,
ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMMergePolicyFactory mergePolicyFactory,
Map<String, String> mergePolicyProperties, boolean durable, int[] bloomFilterKeyFields,
- double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields, boolean hasBloomFilter) {
+ double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields, boolean hasBloomFilter,
+ boolean isSecondaryNoIncrementalMaintenance) {
super(storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
opTrackerFactory, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable,
bloomFilterKeyFields, bloomFilterFalsePositiveRate, isPrimary, btreeFields,
- NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter, null, null);
+ NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter, null, null,
+ isSecondaryNoIncrementalMaintenance);
}
@Override
@@ -60,6 +62,6 @@
bloomFilterFalsePositiveRate, isPrimary, fileRef.getRelativePath(), storageManager, mergePolicyFactory,
mergePolicyProperties, filterTypeTraits, filterCmpFactories, btreeFields, filterFields,
opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
- vbcProvider, ioSchedulerProvider, durable, hasBloomFilter);
+ vbcProvider, ioSchedulerProvider, durable, hasBloomFilter, isSecondaryNoIncrementalMaintenance);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java
index aa9cd5a..144f1ad 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java
@@ -740,6 +740,11 @@
}
@Override
+ public int getPartitionCount() {
+ return 1;
+ }
+
+ @Override
public ICounterContext getCounterContext() {
return null;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ExitUtil.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ExitUtil.java
index beabb5d..8f8e8f6 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ExitUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ExitUtil.java
@@ -55,6 +55,7 @@
public static final int EC_ACTIVE_SUSPEND_FAILURE = 17;
public static final int EC_ACTIVE_RESUME_FAILURE = 18;
public static final int EC_NC_FAILED_TO_NOTIFY_TASKS_COMPLETED = 19;
+ public static final int EC_ACTIVE_RECOVERY_FAILURE = 20;
public static final int EC_FAILED_TO_CANCEL_ACTIVE_START_STOP = 22;
public static final int EC_INCONSISTENT_STORAGE_REFERENCES = 23;
public static final int EC_IMMEDIATE_HALT = 33;
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/IThreadStats.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/IThreadStats.java
index dc63ac4..e782a69 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/IThreadStats.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/IThreadStats.java
@@ -31,4 +31,16 @@
* @return the pinned pages count
*/
long getPinnedPagesCount();
+
+ /**
+ * Indicates that this thread caused a cold read from disk
+ */
+ void coldRead();
+
+ /**
+ * Gets the count of pages read in from disk
+ *
+ * @return the cold read count
+ */
+ long getColdReadCount();
}
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ThreadStats.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ThreadStats.java
index c79eefc..9af3fb1 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ThreadStats.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ThreadStats.java
@@ -26,6 +26,7 @@
public class ThreadStats implements IThreadStats {
private AtomicLong pinnedPagesCount = new AtomicLong();
+ private AtomicLong coldReadCount = new AtomicLong();
@Override
public void pagePinned() {
@@ -36,4 +37,14 @@
public long getPinnedPagesCount() {
return pinnedPagesCount.get();
}
+
+ @Override
+ public long getColdReadCount() {
+ return coldReadCount.get();
+ }
+
+ @Override
+ public void coldRead() {
+ coldReadCount.incrementAndGet();
+ }
}
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
index 3eb8687..cde79cb 100644
--- 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
@@ -221,6 +221,10 @@
return VarLenIntEncoderDecoder.decode(b, s);
}
+ public static int getNumBytesToStoreLength(byte[] bytes, int start) {
+ return getNumBytesToStoreLength(getUTFLength(bytes, start));
+ }
+
public static int getNumBytesToStoreLength(int strlen) {
return VarLenIntEncoderDecoder.getBytesRequired(strlen);
}
@@ -244,12 +248,13 @@
* consistent with the comparison result.
*/
public static int normalize(byte[] bytes, int start) {
- int len = getUTFLength(bytes, start);
long nk = 0;
+ int len = getUTFLength(bytes, start);
int offset = start + getNumBytesToStoreLength(len);
+ int end = offset + len;
for (int i = 0; i < 2; ++i) {
nk <<= 16;
- if (i < len) {
+ if (offset < end) {
nk += (charAt(bytes, offset)) & 0xffff;
offset += charSize(bytes, offset);
}
@@ -498,19 +503,15 @@
* are exactly the same as for the <code>readUTF</code>
* method of <code>DataInput</code>.
*
- * @param in
- * a data input stream.
+ * @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.
+ * @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 {
@@ -602,10 +603,8 @@
/**
* Write a UTF8 String <code>str</code> into the DataOutput <code>out</code>
*
- * @param str,
- * a Unicode string;
- * @param out,
- * a Data output stream.
+ * @param str, a Unicode string;
+ * @param out, a Data output stream.
* @throws IOException
*/
public static void writeUTF8(CharSequence str, DataOutput out) throws IOException {
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/test/java/org/apache/hyracks/util/string/UTF8StringSample.java b/hyracks-fullstack/hyracks/hyracks-util/src/test/java/org/apache/hyracks/util/string/UTF8StringSample.java
index b114351..eb3a5b6 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/test/java/org/apache/hyracks/util/string/UTF8StringSample.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/test/java/org/apache/hyracks/util/string/UTF8StringSample.java
@@ -35,7 +35,8 @@
public static final String STRING_LEN_3 = "xyz";
public static final String STRING_UTF8_3 = "锟斤拷";
- public static final String STRING_UTF8_MIX = "\uD841\uDF0E\uD841\uDF31锟X斤Y拷Zà"; // one, two, three, and four bytes
+ // one, two, three, and four bytes
+ public static final String STRING_UTF8_MIX = "\uD841\uDF0E\uD841\uDF31锟X斤Y拷Zà";
public static final String STRING_UTF8_MIX_LOWERCASE = "\uD841\uDF0E\uD841\uDF31锟x斤y拷zà";
public static final String STRING_NEEDS_2_JAVA_CHARS_1 = "\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89";
public static final String STRING_NEEDS_2_JAVA_CHARS_2 = "😢😢💉💉";
@@ -44,6 +45,8 @@
public static final String STRING_EMOJI_FAMILY_OF_2 = "\uD83D\uDC68\u200D\uD83D\uDC66";
public static final String EMOJI_BASKETBALL = "\uD83C\uDFC0";
+ public static final String THREE_BYTES_UTF8_CHAR = "ह";
+
public static final String STRING_LEN_127 = generateStringRepeatBy(ONE_ASCII_CHAR, 127);
public static final String STRING_LEN_128 = generateStringRepeatBy(ONE_ASCII_CHAR, 128);
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
index c7468d2..4eb1fc3 100644
--- 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
@@ -25,6 +25,7 @@
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.UTF8StringSample.THREE_BYTES_UTF8_CHAR;
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;
@@ -77,13 +78,14 @@
}
@Test
- public void testCompareToAndNormolize() throws Exception {
+ public void testCompareToAndNormalize() 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);
+ testCompare(THREE_BYTES_UTF8_CHAR, THREE_BYTES_UTF8_CHAR, OPTION.STANDARD);
}
- public boolean isSameSign(int r1, int r2) {
+ private static boolean isSameSign(int r1, int r2) {
if (r1 > 0) {
return r2 > 0;
}
@@ -99,7 +101,7 @@
LOWERCASE
}
- public void testCompare(String str1, String str2, OPTION option) throws IOException {
+ private static void testCompare(String str1, String str2, OPTION option) {
byte[] buffer1 = writeStringToBytes(str1);
byte[] buffer2 = writeStringToBytes(str2);
@@ -117,7 +119,6 @@
assertEquals(str1.compareToIgnoreCase(str2), lowerCaseCompareTo(buffer1, 0, buffer2, 0));
break;
}
-
}
@Test
diff --git a/hyracks-fullstack/hyracks/pom.xml b/hyracks-fullstack/hyracks/pom.xml
index 8816de0..a35a61e 100644
--- a/hyracks-fullstack/hyracks/pom.xml
+++ b/hyracks-fullstack/hyracks/pom.xml
@@ -88,6 +88,7 @@
<module>hyracks-storage-am-lsm-invertedindex</module>
<module>hyracks-storage-am-lsm-common</module>
<module>hyracks-storage-am-lsm-btree</module>
+ <module>hyracks-storage-am-lsm-btree-column</module>
<module>hyracks-storage-am-lsm-rtree</module>
<module>hyracks-storage-am-rtree</module>
<module>hyracks-test-support</module>
diff --git a/hyracks-fullstack/pom.xml b/hyracks-fullstack/pom.xml
index d082076..348b5bf 100644
--- a/hyracks-fullstack/pom.xml
+++ b/hyracks-fullstack/pom.xml
@@ -72,9 +72,11 @@
<!-- Versions under dependencymanagement or used in many projects via properties -->
<hadoop.version>3.3.1</hadoop.version>
<jacoco.version>0.7.6.201602180812</jacoco.version>
- <log4j.version>2.17.1</log4j.version>
+ <log4j.version>2.19.0</log4j.version>
<snappy.version>1.1.8.4</snappy.version>
- <jackson.version>2.13.2</jackson.version>
+ <jackson.version>2.14.1</jackson.version>
+ <jackson-databind.version>${jackson.version}</jackson-databind.version>
+ <netty.version>4.1.87.Final</netty.version>
<implementation.title>Apache Hyracks and Algebricks - ${project.name}</implementation.title>
<implementation.url>https://asterixdb.apache.org/</implementation.url>
@@ -86,47 +88,52 @@
<dependency>
<groupId>io.netty</groupId>
<artifactId>netty-buffer</artifactId>
- <version>4.1.73.Final</version>
+ <version>${netty.version}</version>
</dependency>
<dependency>
<groupId>io.netty</groupId>
<artifactId>netty-common</artifactId>
- <version>4.1.73.Final</version>
+ <version>${netty.version}</version>
</dependency>
<dependency>
<groupId>io.netty</groupId>
<artifactId>netty-codec-http</artifactId>
- <version>4.1.73.Final</version>
+ <version>${netty.version}</version>
</dependency>
<dependency>
<groupId>io.netty</groupId>
<artifactId>netty-transport</artifactId>
- <version>4.1.73.Final</version>
+ <version>${netty.version}</version>
</dependency>
<dependency>
<groupId>io.netty</groupId>
<artifactId>netty-codec</artifactId>
- <version>4.1.73.Final</version>
+ <version>${netty.version}</version>
</dependency>
<dependency>
<groupId>io.netty</groupId>
<artifactId>netty-handler</artifactId>
- <version>4.1.73.Final</version>
+ <version>${netty.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-transport-classes-epoll</artifactId>
+ <version>${netty.version}</version>
</dependency>
<dependency>
<groupId>io.netty</groupId>
<artifactId>netty-resolver-dns</artifactId>
- <version>4.1.73.Final</version>
+ <version>${netty.version}</version>
</dependency>
<dependency>
<groupId>io.netty</groupId>
<artifactId>netty-codec-http2</artifactId>
- <version>4.1.73.Final</version>
+ <version>${netty.version}</version>
</dependency>
<dependency>
<groupId>io.netty</groupId>
<artifactId>netty-transport-native-unix-common</artifactId>
- <version>4.1.73.Final</version>
+ <version>${netty.version}</version>
</dependency>
<dependency>
<groupId>junit</groupId>
@@ -165,16 +172,21 @@
<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>
+ <groupId>ch.qos.reload4j</groupId>
+ <artifactId>reload4j</artifactId>
+ </exclusion>
+ <!-- TODO(htowaileb): Remove after updating to hadoop 3.3.4 -->
+ <exclusion>
+ <groupId>org.eclipse.jetty</groupId>
+ <artifactId>jetty-util</artifactId>
</exclusion>
<exclusion>
- <groupId>org.slf4j</groupId>
- <artifactId>slf4j-log4j12</artifactId>
+ <groupId>org.eclipse.jetty</groupId>
+ <artifactId>jetty-util-ajax</artifactId>
</exclusion>
</exclusions>
</dependency>
@@ -188,8 +200,21 @@
<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>
+ <!-- TODO(htowaileb): Remove after updating to hadoop 3.3.4 -->
+ <exclusion>
+ <groupId>org.eclipse.jetty</groupId>
+ <artifactId>jetty-util</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.eclipse.jetty</groupId>
+ <artifactId>jetty-util-ajax</artifactId>
</exclusion>
</exclusions>
</dependency>
@@ -199,8 +224,21 @@
<version>${hadoop.version}</version>
<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>
+ <!-- TODO(htowaileb): Remove after updating to hadoop 3.3.4 -->
+ <exclusion>
+ <groupId>org.eclipse.jetty</groupId>
+ <artifactId>jetty-util</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.eclipse.jetty</groupId>
+ <artifactId>jetty-util-ajax</artifactId>
</exclusion>
</exclusions>
</dependency>
@@ -208,6 +246,13 @@
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-mapreduce-client-core</artifactId>
<version>${hadoop.version}</version>
+ <!-- TODO(htowaileb): Remove after updating to hadoop 3.3.4 -->
+ <exclusions>
+ <exclusion>
+ <groupId>org.eclipse.jetty</groupId>
+ <artifactId>jetty-util</artifactId>
+ </exclusion>
+ </exclusions>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
@@ -217,12 +262,26 @@
<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>
+ <!-- TODO(htowaileb): Remove after updating to hadoop 3.3.4 -->
+ <exclusion>
+ <groupId>org.eclipse.jetty</groupId>
+ <artifactId>jetty-util</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
+ <groupId>com.e-movimento.tinytools</groupId>
+ <artifactId>privilegedaccessor</artifactId>
+ <version>1.3</version>
+ </dependency>
+ <dependency>
<groupId>commons-io</groupId>
<artifactId>commons-io</artifactId>
<version>2.11.0</version>
@@ -230,12 +289,12 @@
<dependency>
<groupId>org.apache.commons</groupId>
<artifactId>commons-text</artifactId>
- <version>1.9</version>
+ <version>1.10.0</version>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-databind</artifactId>
- <version>2.13.2.2</version>
+ <version>${jackson-databind.version}</version>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.core</groupId>
@@ -250,7 +309,7 @@
<dependency>
<groupId>com.google.guava</groupId>
<artifactId>guava</artifactId>
- <version>31.0.1-jre</version>
+ <version>31.1-jre</version>
<exclusions>
<exclusion>
<groupId>com.google.code.findbugs</groupId>
@@ -409,6 +468,18 @@
<artifactId>snappy-java</artifactId>
<version>${snappy.version}</version>
</dependency>
+ <!-- TODO(htowaileb): removed from hadoop transitively and added separately to avoid CVEs, can
+ be removed once upgraded to hadoop 3.3.4 as it addresses the CVEs -->
+ <dependency>
+ <groupId>org.eclipse.jetty</groupId>
+ <artifactId>jetty-util</artifactId>
+ <version>9.4.48.v20220622</version>
+ </dependency>
+ <dependency>
+ <groupId>org.eclipse.jetty</groupId>
+ <artifactId>jetty-util-ajax</artifactId>
+ <version>9.4.48.v20220622</version>
+ </dependency>
</dependencies>
</dependencyManagement>
<build>