Merge branch 'gerrit/neo'
Change-Id: I1944b0ce55e9aeaae4ec5c3c2947201a73902752
diff --git a/asterixdb/asterix-algebra/pom.xml b/asterixdb/asterix-algebra/pom.xml
index 7a32f42..f542af3 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..76404ff 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);
@@ -76,6 +86,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 +106,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..43963dc 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/SqlppCompilationProvider.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/SqlppCompilationProvider.java
@@ -18,14 +18,27 @@
*/
package org.apache.asterix.compiler.provider;
+import java.util.HashSet;
+import java.util.Set;
+
import org.apache.asterix.algebra.base.ILangExpressionToPlanTranslatorFactory;
import org.apache.asterix.algebra.base.ILangExtension;
+import org.apache.asterix.common.config.CompilerProperties;
+import org.apache.asterix.external.feed.watch.FeedActivityDetails;
import org.apache.asterix.lang.common.base.IAstPrintVisitorFactory;
import org.apache.asterix.lang.common.base.IParserFactory;
import org.apache.asterix.lang.common.base.IRewriterFactory;
+import org.apache.asterix.lang.common.statement.StartFeedStatement;
+import org.apache.asterix.lang.common.util.FunctionUtil;
import org.apache.asterix.lang.sqlpp.parser.SqlppParserFactory;
+import org.apache.asterix.lang.sqlpp.rewrites.SqlppQueryRewriter;
import org.apache.asterix.lang.sqlpp.rewrites.SqlppRewriterFactory;
import org.apache.asterix.lang.sqlpp.visitor.SqlppAstPrintVisitorFactory;
+import org.apache.asterix.optimizer.base.FuzzyUtils;
+import org.apache.asterix.optimizer.rules.DisjunctivePredicateToJoinRule;
+import org.apache.asterix.optimizer.rules.SetAsterixPhysicalOperatorsRule;
+import org.apache.asterix.optimizer.rules.util.EquivalenceClassUtils;
+import org.apache.asterix.translator.SqlppExpressionToPlanTranslator;
import org.apache.asterix.translator.SqlppExpressionToPlanTranslatorFactory;
public class SqlppCompilationProvider implements ILangCompilationProvider {
@@ -59,4 +72,24 @@
public IRuleSetFactory getRuleSetFactory() {
return new DefaultRuleSetFactory();
}
+
+ @Override
+ public Set<String> getCompilerOptions() {
+ return new HashSet<>(Set.of(CompilerProperties.COMPILER_JOINMEMORY_KEY,
+ CompilerProperties.COMPILER_GROUPMEMORY_KEY, CompilerProperties.COMPILER_SORTMEMORY_KEY,
+ CompilerProperties.COMPILER_WINDOWMEMORY_KEY, CompilerProperties.COMPILER_TEXTSEARCHMEMORY_KEY,
+ CompilerProperties.COMPILER_PARALLELISM_KEY, CompilerProperties.COMPILER_SORT_PARALLEL_KEY,
+ CompilerProperties.COMPILER_SORT_SAMPLES_KEY, CompilerProperties.COMPILER_EXTERNALSCANMEMORY_KEY,
+ CompilerProperties.COMPILER_INDEXONLY_KEY, CompilerProperties.COMPILER_INTERNAL_SANITYCHECK_KEY,
+ CompilerProperties.COMPILER_EXTERNAL_FIELD_PUSHDOWN_KEY, CompilerProperties.COMPILER_SUBPLAN_MERGE_KEY,
+ CompilerProperties.COMPILER_SUBPLAN_NESTEDPUSHDOWN_KEY, CompilerProperties.COMPILER_ARRAYINDEX_KEY,
+ CompilerProperties.COMPILER_MIN_MEMORY_ALLOCATION_KEY, FunctionUtil.IMPORT_PRIVATE_FUNCTIONS,
+ FuzzyUtils.SIM_FUNCTION_PROP_NAME, FuzzyUtils.SIM_THRESHOLD_PROP_NAME,
+ StartFeedStatement.WAIT_FOR_COMPLETION, FeedActivityDetails.FEED_POLICY_NAME,
+ FeedActivityDetails.COLLECT_LOCATIONS, SqlppQueryRewriter.INLINE_WITH_OPTION,
+ SqlppExpressionToPlanTranslator.REWRITE_IN_AS_OR_OPTION, "hash_merge", "output-record-type",
+ DisjunctivePredicateToJoinRule.REWRITE_OR_AS_JOIN_OPTION,
+ SetAsterixPhysicalOperatorsRule.REWRITE_ATTEMPT_BATCH_ASSIGN,
+ EquivalenceClassUtils.REWRITE_INTERNAL_QUERYUID_PK, SqlppQueryRewriter.SQL_COMPAT_OPTION));
+ }
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
index b1d0b47..b8685d5 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
@@ -26,7 +26,6 @@
import org.apache.asterix.external.library.ExternalFunctionDescriptorProvider;
import org.apache.asterix.metadata.declared.MetadataProvider;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IExternalFunctionInfo;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionManager;
import org.apache.asterix.om.functions.IFunctionTypeInferer;
@@ -138,7 +137,7 @@
throws AlgebricksException {
IScalarEvaluatorFactory[] args = codegenArguments(expr, env, inputSchemas, context);
IFunctionDescriptor fd;
- if (expr.getFunctionInfo() instanceof IExternalFunctionInfo) {
+ if (expr.getFunctionInfo().isExternal()) {
// Expr is an external function
fd = ExternalFunctionDescriptorProvider.resolveExternalFunction(expr, env, context);
} else {
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/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/rules/ConstantFoldingRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
index 10aedc7..7202f74 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());
@@ -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
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/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/am/AbstractIntroduceAccessMethodRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
index 7f1ff4a..5be9d66 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
@@ -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..ddc4964 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
@@ -24,7 +24,6 @@
import static org.apache.asterix.om.functions.BuiltinFunctions.FIELD_ACCESS_NESTED;
import java.util.ArrayList;
-import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
@@ -147,6 +146,8 @@
BuiltinFunctions.YEAR_MONTH_DURATION_DEFAULT_NULL_CONSTRUCTOR,
BuiltinFunctions.UUID_DEFAULT_NULL_CONSTRUCTOR, BuiltinFunctions.BINARY_BASE64_DEFAULT_NULL_CONSTRUCTOR);
+ private final static Pair<List<String>, Integer> NO_FIELD_NAME = new Pair<>(Collections.emptyList(), 0);
+
public static void appendPrimaryIndexTypes(Dataset dataset, IAType itemType, IAType metaItemType,
List<Object> target) throws AlgebricksException {
ARecordType recordType = (ARecordType) itemType;
@@ -2898,17 +2899,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 +2918,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 +2930,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 +2959,21 @@
if (funcIdent == BuiltinFunctions.FIELD_ACCESS_BY_NAME) {
fieldName = ConstantExpressionUtil.getStringArgument(funcExpr, 1);
if (fieldName == null) {
- return Collections.emptyList();
+ return NO_FIELD_NAME;
}
isFieldAccess = true;
isByName = true;
} else if (funcIdent == BuiltinFunctions.FIELD_ACCESS_BY_INDEX) {
Integer idx = ConstantExpressionUtil.getIntArgument(funcExpr, 1);
if (idx == null) {
- return Collections.emptyList();
+ return NO_FIELD_NAME;
}
fieldIndex = idx;
isFieldAccess = true;
} else if (funcIdent == BuiltinFunctions.FIELD_ACCESS_NESTED) {
ILogicalExpression nameArg = funcExpr.getArguments().get(1).getValue();
if (nameArg.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
- return Collections.emptyList();
+ return NO_FIELD_NAME;
}
ConstantExpression constExpr = (ConstantExpression) nameArg;
AOrderedList orderedNestedFieldName =
@@ -2989,11 +2988,6 @@
if (isFieldAccess) {
LogicalVariable sourceVar =
((VariableReferenceExpression) funcExpr.getArguments().get(0).getValue()).getVariableReference();
- if (sourceVar.equals(metaVar)) {
- fieldSource.setValue(1);
- } else {
- fieldSource.setValue(0);
- }
if (optFuncExpr != null) {
optFuncExpr.setLogicalExpr(funcVarIndex, parentFuncExpr);
optFuncExpr.addStepExpr(funcVarIndex, funcExpr);
@@ -3024,8 +3018,7 @@
if (parentVars.contains(var)) {
//Found the variable we are looking for.
//return assign and index of expression
- int[] returnValues = { i, varIndex };
- assignAndExpressionIndexes = returnValues;
+ assignAndExpressionIndexes = new int[] { i, varIndex };
}
}
}
@@ -3033,34 +3026,25 @@
//We found the nested assign
//Recursive call on nested assign
- List<String> parentFieldNames = getFieldNameAndStepsFromSubTree(optFuncExpr, subTree,
- assignAndExpressionIndexes[0], assignAndExpressionIndexes[1], recordType, funcVarIndex,
- parentFuncExpr, metaType, metaVar, fieldSource, isUnnestOverVarAllowed);
+ Pair<List<String>, Integer> parentFieldNames =
+ getFieldNameAndStepsFromSubTree(optFuncExpr, subTree, assignAndExpressionIndexes[0],
+ assignAndExpressionIndexes[1], funcVarIndex, parentFuncExpr, context);
- boolean isPreviousOperatorLegalUnnest = isUnnestOverVarAllowed && subTree.getAssignsAndUnnests()
- .get(assignAndExpressionIndexes[0]).getOperatorTag().equals(LogicalOperatorTag.UNNEST);
- if (parentFieldNames.isEmpty() && !isPreviousOperatorLegalUnnest) {
+ if (parentFieldNames.first.isEmpty()) {
//Nested assign was not a field access.
//We will not use index
- return Collections.emptyList();
- } else if (isPreviousOperatorLegalUnnest) {
- parentFieldNames = new ArrayList<>();
+ return NO_FIELD_NAME;
}
if (!isByName) {
- IAType subFieldType;
- if (isUnnestOverVarAllowed && isPreviousOperatorLegalUnnest) {
- // In the case of UNNESTing over a variable, we use the record type given by our caller instead.
- subFieldType = sourceVar.equals(metaVar) ? metaType : recordType;
- } else {
- subFieldType = sourceVar.equals(metaVar) ? metaType.getSubFieldType(parentFieldNames)
- : recordType.getSubFieldType(parentFieldNames);
- // Sub-field type can be AUnionType in case if optional. Thus, needs to get the actual type.
- subFieldType = TypeComputeUtils.getActualType(subFieldType);
- if (subFieldType.getTypeTag() != ATypeTag.OBJECT) {
- throw CompilationException.create(ErrorCode.TYPE_CONVERT, subFieldType,
- ARecordType.class.getName());
- }
+ IVariableTypeEnvironment outputTypeEnvironment = context.getOutputTypeEnvironment(
+ subTree.getAssignsAndUnnests().get(assignAndExpressionIndexes[0]));
+ IAType subFieldType = (IAType) outputTypeEnvironment.getVarType(sourceVar);
+ // Sub-field type can be AUnionType in case if optional. Thus, needs to get the actual type.
+ subFieldType = TypeComputeUtils.getActualType(subFieldType);
+ if (subFieldType.getTypeTag() != ATypeTag.OBJECT) {
+ throw CompilationException.create(ErrorCode.TYPE_CONVERT, subFieldType,
+ ARecordType.class.getName());
}
fieldName = ((ARecordType) subFieldType).getFieldNames()[fieldIndex];
@@ -3070,11 +3054,9 @@
}
//add fieldName to the nested fieldName, return
if (nestedAccessFieldName != null) {
- for (int i = 0; i < nestedAccessFieldName.size(); i++) {
- parentFieldNames.add(nestedAccessFieldName.get(i));
- }
+ parentFieldNames.first.addAll(nestedAccessFieldName);
} else {
- parentFieldNames.add(fieldName);
+ parentFieldNames.first.add(fieldName);
}
return (parentFieldNames);
}
@@ -3083,15 +3065,15 @@
optFuncExpr.setSourceVar(funcVarIndex, ((AssignOperator) op).getVariables().get(assignVarIndex));
}
//no nested assign, we are at the lowest level.
+ OptimizableOperatorSubTree.RecordTypeSource recType = subTree.getRecordTypeFor(sourceVar);
if (isByName) {
if (nestedAccessFieldName != null) {
- return nestedAccessFieldName;
+ return new Pair<>(nestedAccessFieldName, recType.sourceIndicator);
}
- return new ArrayList<>(Arrays.asList(fieldName));
+ return new Pair<>(new ArrayList<>(List.of(fieldName)), recType.sourceIndicator);
}
- return new ArrayList<>(Arrays.asList(sourceVar.equals(metaVar) ? metaType.getFieldNames()[fieldIndex]
- : recordType.getFieldNames()[fieldIndex]));
-
+ return new Pair<>(new ArrayList<>(List.of(recType.recordType.getFieldNames()[fieldIndex])),
+ recType.sourceIndicator);
}
// We use a part of the field in edit distance computation
@@ -3101,17 +3083,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 +3113,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,
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..751e216 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
@@ -382,10 +382,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.
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..1cf4b0c 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
@@ -453,7 +453,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,
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/translator/AbstractLangTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AbstractLangTranslator.java
index ac7fabe..b0f9a1c 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,7 @@
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.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 +372,17 @@
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;
}
if (invalidOperation) {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/ResultMetadata.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ResultMetadata.java
similarity index 96%
rename from asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/ResultMetadata.java
rename to asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ResultMetadata.java
index 94360a1..78f84ff 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/ResultMetadata.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ResultMetadata.java
@@ -16,12 +16,11 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.asterix.api.common;
+package org.apache.asterix.translator;
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;
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..9b9dfac 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());
}
@@ -237,7 +214,8 @@
&& 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);
@@ -245,8 +223,9 @@
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());
@@ -264,6 +243,24 @@
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();
@@ -312,32 +309,16 @@
return null;
}
- builder.setBinaryBooleanInspectorFactory(format.getBinaryBooleanInspectorFactory());
- builder.setBinaryIntegerInspectorFactory(format.getBinaryIntegerInspectorFactory());
- builder.setComparatorFactoryProvider(format.getBinaryComparatorFactoryProvider());
- builder.setExpressionRuntimeProvider(
- new ExpressionRuntimeProvider(new QueryLogicalExpressionJobGen(metadataProvider.getFunctionManager())));
- builder.setHashFunctionFactoryProvider(format.getBinaryHashFunctionFactoryProvider());
- builder.setHashFunctionFamilyProvider(format.getBinaryHashFunctionFamilyProvider());
- builder.setMissingWriterFactory(format.getMissingWriterFactory());
- builder.setNullWriterFactory(format.getNullWriterFactory());
- builder.setUnnestingPositionWriterFactory(format.getUnnestingPositionWriterFactory());
- builder.setPredicateEvaluatorFactoryProvider(format.getPredicateEvaluatorFactoryProvider());
- builder.setPrinterProvider(getPrinterFactoryProvider(format, conf.fmt()));
- builder.setSerializerDeserializerProvider(format.getSerdeProvider());
- builder.setTypeTraitProvider(format.getTypeTraitProvider());
- builder.setNormalizedKeyComputerFactoryProvider(format.getNormalizedKeyComputerFactoryProvider());
-
JobEventListenerFactory jobEventListenerFactory =
new JobEventListenerFactory(txnId, metadataProvider.isWriteTransaction());
- JobSpecification spec = compiler.createJob(metadataProvider.getApplicationContext(), jobEventListenerFactory);
+ JobSpecification spec = compiler.createJob(ccAppContext, jobEventListenerFactory);
if (isQuery) {
if (requestParameters == null || !requestParameters.isSkipAdmissionPolicy()) {
// Sets a required capacity, only for read-only queries.
// DDLs and DMLs are considered not that frequent.
// limit the computation locations to the locations that will be used in the query
- final INodeJobTracker nodeJobTracker = metadataProvider.getApplicationContext().getNodeJobTracker();
+ final INodeJobTracker nodeJobTracker = ccAppContext.getNodeJobTracker();
final AlgebricksAbsolutePartitionConstraint jobLocations =
getJobLocations(spec, nodeJobTracker, computationLocations);
final IClusterCapacity jobRequiredCapacity =
@@ -502,10 +483,10 @@
}
// Validates if the query contains unsupported query parameters.
- private static Map<String, Object> validateConfig(Map<String, Object> config, SourceLocation sourceLoc)
+ private Map<String, Object> validateConfig(Map<String, Object> config, SourceLocation sourceLoc)
throws AlgebricksException {
for (String parameterName : config.keySet()) {
- if (!CONFIGURABLE_PARAMETER_NAMES.contains(parameterName)
+ if (!configurableParameterNames.contains(parameterName)
&& !parameterName.startsWith(PREFIX_INTERNAL_PARAMETERS)) {
throw AsterixException.create(ErrorCode.COMPILATION_UNSUPPORTED_QUERY_PARAMETER, sourceLoc,
parameterName);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/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/QueryResultApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryResultApiServlet.java
index 981cdc9..bfebfd6 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryResultApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryResultApiServlet.java
@@ -21,7 +21,6 @@
import java.io.IOException;
import java.util.concurrent.ConcurrentMap;
-import org.apache.asterix.api.common.ResultMetadata;
import org.apache.asterix.app.result.ResponseMetrics;
import org.apache.asterix.app.result.ResponsePrinter;
import org.apache.asterix.app.result.ResultHandle;
@@ -31,6 +30,7 @@
import org.apache.asterix.app.result.fields.ResultsPrinter;
import org.apache.asterix.common.api.IApplicationContext;
import org.apache.asterix.translator.IStatementExecutor.Stats;
+import org.apache.asterix.translator.ResultMetadata;
import org.apache.asterix.translator.SessionConfig;
import org.apache.asterix.translator.SessionOutput;
import org.apache.hyracks.api.exceptions.ErrorCode;
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/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-runtime/src/main/java/org/apache/asterix/runtime/utils/CcApplicationContext.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CcApplicationContext.java
similarity index 87%
rename from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/CcApplicationContext.java
rename to asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CcApplicationContext.java
index 66f0e73..880880e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/CcApplicationContext.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CcApplicationContext.java
@@ -16,12 +16,13 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.asterix.runtime.utils;
+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;
@@ -56,13 +57,21 @@
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
@@ -87,7 +96,8 @@
private MessagingProperties messagingProperties;
private NodeProperties nodeProperties;
private Supplier<IMetadataBootstrap> metadataBootstrapSupplier;
- private IHyracksClientConnection hcc;
+ private volatile HyracksConnection hcc;
+ private volatile ResultSet resultSet;
private Object extensionManager;
private INcLifecycleCoordinator ftStrategy;
private IJobLifecycleListener activeLifeCycleListener;
@@ -103,7 +113,7 @@
private final IAdapterFactoryService adapterFactoryService;
private final ReentrantReadWriteLock compilationLock = new ReentrantReadWriteLock(true);
- public CcApplicationContext(ICCServiceContext ccServiceCtx, IHyracksClientConnection hcc,
+ public CcApplicationContext(ICCServiceContext ccServiceCtx, HyracksConnection hcc,
Supplier<IMetadataBootstrap> metadataBootstrapSupplier, IGlobalRecoveryManager globalRecoveryManager,
INcLifecycleCoordinator ftStrategy, IJobLifecycleListener activeLifeCycleListener,
IStorageComponentProvider storageComponentProvider, IMetadataLockManager mdLockManager,
@@ -188,18 +198,44 @@
@Override
public IHyracksClientConnection getHcc() throws HyracksDataException {
- if (!hcc.isConnected()) {
+ HyracksConnection hc = hcc;
+ if (!hc.isConnected()) {
synchronized (this) {
- if (!hcc.isConnected()) {
+ hc = hcc;
+ if (!hc.isConnected()) {
try {
- hcc = new HyracksConnection(hcc.getHost(), hcc.getPort());
+ 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 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(), ccServiceCtx.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/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/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/result/JobResultCallback.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/JobResultCallback.java
index 05073f9..201a470 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/JobResultCallback.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/JobResultCallback.java
@@ -23,8 +23,8 @@
import java.util.Iterator;
import java.util.Set;
-import org.apache.asterix.api.common.ResultMetadata;
import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.asterix.translator.ResultMetadata;
import org.apache.hyracks.api.exceptions.Warning;
import org.apache.hyracks.api.job.JobFlag;
import org.apache.hyracks.api.job.JobId;
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultReader.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultReader.java
index 1acae87..56dfd5e 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultReader.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultReader.java
@@ -18,6 +18,8 @@
*/
package org.apache.asterix.app.result;
+import org.apache.asterix.common.config.CompilerProperties;
+import org.apache.hyracks.api.client.IHyracksClientConnection;
import org.apache.hyracks.api.comm.IFrame;
import org.apache.hyracks.api.comm.IFrameTupleAccessor;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -27,12 +29,14 @@
import org.apache.hyracks.api.result.IResultSetReader;
import org.apache.hyracks.api.result.ResultJobRecord.Status;
import org.apache.hyracks.api.result.ResultSetId;
+import org.apache.hyracks.api.service.IControllerService;
+import org.apache.hyracks.client.result.ResultSet;
import org.apache.hyracks.dataflow.common.comm.io.ResultFrameTupleAccessor;
public class ResultReader {
- private IResultSetReader reader;
+ private final IResultSetReader reader;
- private IFrameTupleAccessor frameTupleAccessor;
+ private final IFrameTupleAccessor frameTupleAccessor;
// Number of parallel result reader buffers
public static final int NUM_READERS = 1;
@@ -57,4 +61,10 @@
public IResultMetadata getMetadata() {
return reader.getResultMetadata();
}
+
+ public static ResultSet createResultSet(IHyracksClientConnection hcc, IControllerService srv,
+ CompilerProperties compilerProperties) throws Exception {
+ return new ResultSet(hcc, srv.getNetworkSecurityManager().getSocketChannelFactory(),
+ compilerProperties.getFrameSize(), ResultReader.NUM_READERS);
+ }
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/SignaturePrinter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/SignaturePrinter.java
index 048584c..3f1e632 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/SignaturePrinter.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/SignaturePrinter.java
@@ -22,7 +22,6 @@
import java.util.LinkedHashSet;
import java.util.List;
-import org.apache.asterix.api.common.ResultMetadata;
import org.apache.asterix.api.http.server.ResultUtil;
import org.apache.asterix.common.annotations.IRecordTypeAnnotation;
import org.apache.asterix.common.annotations.RecordFieldOrderAnnotation;
@@ -34,6 +33,7 @@
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.IAType;
import org.apache.asterix.translator.ExecutionPlans;
+import org.apache.asterix.translator.ResultMetadata;
import org.apache.hyracks.algebricks.common.utils.Pair;
import org.apache.hyracks.util.JSONUtil;
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
index 41d9282..3b41beb 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,9 @@
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.AnalyzeStatement;
import org.apache.asterix.lang.common.statement.CompactStatement;
import org.apache.asterix.lang.common.statement.ConnectFeedStatement;
import org.apache.asterix.lang.common.statement.CreateAdapterStatement;
@@ -195,6 +197,7 @@
import org.apache.asterix.metadata.utils.KeyFieldTypeUtil;
import org.apache.asterix.metadata.utils.MetadataConstants;
import org.apache.asterix.metadata.utils.MetadataUtil;
+import org.apache.asterix.metadata.utils.SampleOperationsHelper;
import org.apache.asterix.metadata.utils.TypeUtil;
import org.apache.asterix.om.base.ANull;
import org.apache.asterix.om.base.IAObject;
@@ -208,6 +211,7 @@
import org.apache.asterix.runtime.fulltext.FullTextConfigDescriptor;
import org.apache.asterix.runtime.fulltext.IFullTextFilterDescriptor;
import org.apache.asterix.runtime.fulltext.StopwordsFullTextFilterDescriptor;
+import org.apache.asterix.runtime.operators.StreamStats;
import org.apache.asterix.transaction.management.service.transaction.DatasetIdFactory;
import org.apache.asterix.translator.AbstractLangTranslator;
import org.apache.asterix.translator.ClientRequest;
@@ -235,13 +239,11 @@
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.common.utils.Pair;
import org.apache.hyracks.algebricks.common.utils.Triple;
+import org.apache.hyracks.algebricks.core.algebra.base.Counter;
import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression.FunctionKind;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
import org.apache.hyracks.algebricks.data.IAWriterFactory;
-import org.apache.hyracks.algebricks.data.IResultSerializerFactoryProvider;
-import org.apache.hyracks.algebricks.runtime.serializer.ResultSerializerFactoryProvider;
-import org.apache.hyracks.algebricks.runtime.writers.PrinterBasedWriterFactory;
import org.apache.hyracks.api.client.IClusterInfoCollector;
import org.apache.hyracks.api.client.IHyracksClientConnection;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -253,6 +255,7 @@
import org.apache.hyracks.api.job.JobFlag;
import org.apache.hyracks.api.job.JobId;
import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
import org.apache.hyracks.api.result.IResultSet;
import org.apache.hyracks.api.result.ResultSetId;
import org.apache.hyracks.control.cc.ClusterControllerService;
@@ -330,10 +333,8 @@
public void compileAndExecute(IHyracksClientConnection hcc, IRequestParameters requestParameters) throws Exception {
validateStatements(requestParameters);
trackRequest(requestParameters);
- int resultSetIdCounter = 0;
+ Counter resultSetIdCounter = new Counter(0);
FileSplit outputFile = null;
- IAWriterFactory writerFactory = PrinterBasedWriterFactory.INSTANCE;
- IResultSerializerFactoryProvider resultSerializerFactoryProvider = ResultSerializerFactoryProvider.INSTANCE;
String threadName = Thread.currentThread().getName();
Thread.currentThread().setName(
QueryTranslator.class.getSimpleName() + ":" + requestParameters.getRequestReference().getUuid());
@@ -353,8 +354,8 @@
}
validateOperation(appCtx, activeDataverse, stmt);
MetadataProvider metadataProvider = MetadataProvider.create(appCtx, activeDataverse);
- configureMetadataProvider(metadataProvider, config, resultSerializerFactoryProvider, writerFactory,
- outputFile, requestParameters, stmt);
+ configureMetadataProvider(metadataProvider, config, resultSetIdCounter, outputFile, requestParameters,
+ stmt);
IStatementRewriter stmtRewriter = rewriterFactory.createStatementRewriter();
rewriteStatement(stmt, stmtRewriter, metadataProvider); // Rewrite the statement's AST.
Statement.Kind kind = stmt.getKind();
@@ -444,7 +445,7 @@
case INSERT:
case UPSERT:
if (((InsertStatement) stmt).getReturnExpression() != null) {
- metadataProvider.setResultSetId(new ResultSetId(resultSetIdCounter++));
+ metadataProvider.setResultSetId(new ResultSetId(resultSetIdCounter.getAndInc()));
metadataProvider.setResultAsyncMode(resultDelivery == ResultDelivery.ASYNC
|| resultDelivery == ResultDelivery.DEFERRED);
metadataProvider.setMaxResultReads(maxResultReads);
@@ -480,7 +481,7 @@
handleCreateFeedPolicyStatement(metadataProvider, stmt);
break;
case QUERY:
- metadataProvider.setResultSetId(new ResultSetId(resultSetIdCounter++));
+ metadataProvider.setResultSetId(new ResultSetId(resultSetIdCounter.getAndInc()));
metadataProvider.setResultAsyncMode(
resultDelivery == ResultDelivery.ASYNC || resultDelivery == ResultDelivery.DEFERRED);
metadataProvider.setMaxResultReads(maxResultReads);
@@ -490,6 +491,9 @@
handleQuery(metadataProvider, (Query) stmt, hcc, resultSet, resultDelivery, outMetadata, stats,
requestParameters, stmtParams, stmtRewriter);
break;
+ case ANALYZE:
+ handleAnalyzeStatement(metadataProvider, stmt, hcc, requestParameters);
+ break;
case COMPACT:
handleCompactStatement(metadataProvider, stmt, hcc);
break;
@@ -505,7 +509,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 +529,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 +558,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 +623,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 +729,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 +1080,9 @@
metadataProvider.validateDatabaseObjectName(stmtCreateIndex.getDataverseName(), indexName,
stmt.getSourceLocation());
DataverseName dataverseName = getActiveDataverseName(stmtCreateIndex.getDataverseName());
+ if (isCompileOnly()) {
+ return;
+ }
lockUtil.createIndexBegin(lockManager, metadataProvider.getLocks(), dataverseName, datasetName,
fullTextConfigName);
try {
@@ -1409,6 +1431,9 @@
stmt.getSourceLocation());
DataverseName dataverseName = getActiveDataverseName(stmtCreateFilter.getDataverseName());
+ if (isCompileOnly()) {
+ return;
+ }
lockUtil.createFullTextFilterBegin(lockManager, metadataProvider.getLocks(), dataverseName, fullTextFilterName);
try {
doCreateFullTextFilter(metadataProvider, stmtCreateFilter, dataverseName);
@@ -1477,6 +1502,9 @@
DataverseName dataverseName = getActiveDataverseName(stmtCreateConfig.getDataverseName());
ImmutableList<String> filterNames = stmtCreateConfig.getFilterNames();
+ if (isCompileOnly()) {
+ return;
+ }
lockUtil.createFullTextConfigBegin(lockManager, metadataProvider.getLocks(), dataverseName, configName,
filterNames);
try {
@@ -1842,6 +1870,9 @@
String typeName = stmtCreateType.getIdent().getValue();
metadataProvider.validateDatabaseObjectName(stmtCreateType.getDataverseName(), typeName, sourceLoc);
DataverseName dataverseName = getActiveDataverseName(stmtCreateType.getDataverseName());
+ if (isCompileOnly()) {
+ return;
+ }
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
metadataProvider.setMetadataTxnContext(mdTxnCtx);
lockUtil.createTypeBegin(lockManager, metadataProvider.getLocks(), dataverseName, typeName);
@@ -1895,6 +1926,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);
@@ -2091,6 +2125,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,
@@ -2198,6 +2235,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);
@@ -2380,6 +2420,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);
@@ -2420,6 +2463,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);
@@ -2469,6 +2515,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);
@@ -2509,6 +2558,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);
@@ -2553,6 +2605,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);
@@ -2707,8 +2762,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);
@@ -2744,6 +2801,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);
@@ -2826,6 +2886,9 @@
}
}
+ if (isCompileOnly()) {
+ return;
+ }
lockUtil.createFunctionBegin(lockManager, metadataProvider.getLocks(), dataverseName, signature.getName(),
libraryDataverseName, libraryName);
try {
@@ -2976,8 +3039,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);
@@ -3075,6 +3140,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);
@@ -3133,6 +3201,9 @@
libraryDataverseName = dataverseName;
}
String libraryName = cas.getLibraryName();
+ if (isCompileOnly()) {
+ return;
+ }
lockUtil.createAdapterBegin(lockManager, metadataProvider.getLocks(), dataverseName, adapterName,
libraryDataverseName, libraryName);
try {
@@ -3203,6 +3274,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);
@@ -3253,6 +3327,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);
@@ -3381,6 +3458,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);
@@ -3475,6 +3555,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);
@@ -3522,6 +3605,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);
@@ -3575,7 +3661,7 @@
afterCompile();
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
bActiveTxn = false;
- if (spec != null && sessionConfig.isExecuteQuery()) {
+ if (spec != null && !isCompileOnly()) {
runJob(hcc, spec);
}
} catch (Exception e) {
@@ -3620,7 +3706,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);
@@ -3671,7 +3757,7 @@
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
bActiveTxn = false;
- if (jobSpec != null && sessionConfig.isExecuteQuery()) {
+ if (jobSpec != null && !isCompileOnly()) {
runJob(hcc, jobSpec);
}
return jobSpec;
@@ -3694,8 +3780,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,
@@ -3703,7 +3791,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();
@@ -3711,8 +3799,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());
@@ -3748,6 +3838,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);
@@ -3787,6 +3880,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();
@@ -3848,6 +3944,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);
@@ -3901,6 +4000,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);
@@ -3930,6 +4032,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);
@@ -3992,6 +4097,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);
@@ -4009,6 +4117,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
@@ -4064,6 +4175,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);
@@ -4103,12 +4217,236 @@
}
}
+ 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);
+ 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);
+ }
+ StreamStats stats = new StreamStats(opStats.get(0));
+
+ Index.SampleIndexDetails newIndexDetailsFinal = new Index.SampleIndexDetails(dsDetails.getPrimaryKey(),
+ dsDetails.getKeySourceIndicator(), dsDetails.getPrimaryKeyType(), sampleCardinalityTarget,
+ stats.getCardinality(), stats.getAvgTupleSize(), sampleSeed);
+ 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 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);
@@ -4212,7 +4550,7 @@
afterCompile();
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
bActiveTxn = false;
- return query.isExplain() || !sessionConfig.isExecuteQuery() ? null : jobSpec;
+ return query.isExplain() || isCompileOnly() ? null : jobSpec;
} catch (Exception e) {
LOGGER.log(Level.INFO, e.getMessage(), e);
if (bActiveTxn) {
@@ -4270,8 +4608,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)) {
@@ -4329,6 +4667,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,
@@ -4382,6 +4726,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);
@@ -4416,9 +4763,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;
@@ -4429,6 +4773,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);
@@ -4763,7 +5113,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();
}
@@ -4788,6 +5138,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/utils/RedactionUtil.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RedactionUtil.java
index 156b78a..48cf511 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RedactionUtil.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RedactionUtil.java
@@ -20,7 +20,7 @@
import static java.util.regex.Pattern.CASE_INSENSITIVE;
import static java.util.regex.Pattern.DOTALL;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.SECRET_ACCESS_KEY_FIELD_NAME;
+import static org.apache.asterix.external.util.aws.s3.S3Constants.SECRET_ACCESS_KEY_FIELD_NAME;
import java.util.regex.Pattern;
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/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/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/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..f02b4d6 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,6 +24,7 @@
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;
@@ -51,12 +52,14 @@
@RunWith(Parameterized.class)
public class OptimizerTest extends AbstractOptimizerTest {
+ protected static final String PATH_EXPECTED = PATH_BASE + "results" + SEPARATOR;
private static final String PATTERN_VAR_ID_PREFIX = "\\$\\$";
private static final Pattern PATTERN_VAR_ID = Pattern.compile(PATTERN_VAR_ID_PREFIX + "(\\d+)");
+ protected String expectedFilePath;
static {
EXTENSION_RESULT = "plan";
- PATH_ACTUAL = "target" + File.separator + "opttest" + SEPARATOR;
+ PATH_ACTUAL = "target" + SEPARATOR + "opttest" + SEPARATOR;
}
@Parameters(name = "OptimizerTest {index}: {0}")
@@ -64,8 +67,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 +79,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(
@@ -89,6 +93,7 @@
List<String> linesActual = Files.readAllLines(actualFile.toPath(), StandardCharsets.UTF_8);
+ List<String> linesExpected = getExpectedLines();
int varBaseExpected = findBaseVarId(linesExpected);
int varBaseActual = findBaseVarId(linesActual);
@@ -116,9 +121,8 @@
}
}
- @Override
protected List<String> getExpectedLines() throws IOException {
- return Files.readAllLines(expectedFile.toPath(), StandardCharsets.UTF_8);
+ return Files.readAllLines(Path.of(PATH_EXPECTED, expectedFilePath), StandardCharsets.UTF_8);
}
private boolean planLineEquals(String lineExpected, int varIdBaseExpected, String lineActual, int varIdBaseActual) {
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppBatchPointLookupExecutionTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppBatchPointLookupExecutionTest.java
new file mode 100644
index 0000000..2608447
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppBatchPointLookupExecutionTest.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.test.runtime;
+
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.test.common.TestExecutor;
+import org.apache.asterix.testframework.context.TestCaseContext;
+import org.apache.hyracks.control.nc.NodeControllerService;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+/**
+ * Runs the SQL++ with Batched point-lookups enabled.
+ */
+@RunWith(Parameterized.class)
+public class SqlppBatchPointLookupExecutionTest {
+ protected static final String TEST_CONFIG_FILE_NAME = "src/test/resources/cc-batch-lookup.conf";
+
+ @BeforeClass
+ public static void setUp() throws Exception {
+ final TestExecutor testExecutor = new TestExecutor();
+ LangExecutionUtil.setUp(TEST_CONFIG_FILE_NAME, testExecutor);
+ setNcEndpoints(testExecutor);
+ }
+
+ @AfterClass
+ public static void tearDown() throws Exception {
+ LangExecutionUtil.tearDown();
+ }
+
+ @Parameters(name = "SqlppBatchPointLookupExecutionTest {index}: {0}")
+ public static Collection<Object[]> tests() throws Exception {
+ return LangExecutionUtil.tests("only_batch_lookup.xml", "testsuite_sqlpp_batch_lookup.xml");
+ }
+
+ protected TestCaseContext tcCtx;
+
+ public SqlppBatchPointLookupExecutionTest(TestCaseContext tcCtx) {
+ this.tcCtx = tcCtx;
+ }
+
+ @Test
+ public void test() throws Exception {
+ LangExecutionUtil.test(tcCtx);
+ }
+
+ private static void setNcEndpoints(TestExecutor testExecutor) {
+ final NodeControllerService[] ncs = ExecutionTestUtil.integrationUtil.ncs;
+ final Map<String, InetSocketAddress> ncEndPoints = new HashMap<>();
+ final String ip = InetAddress.getLoopbackAddress().getHostAddress();
+ for (NodeControllerService nc : ncs) {
+ final String nodeId = nc.getId();
+ final INcApplicationContext appCtx = (INcApplicationContext) nc.getApplicationContext();
+ int apiPort = appCtx.getExternalProperties().getNcApiPort();
+ ncEndPoints.put(nodeId, InetSocketAddress.createUnresolved(ip, apiPort));
+ }
+ testExecutor.setNcEndPoints(ncEndPoints);
+ }
+}
diff --git a/asterixdb/asterix-app/src/test/resources/cc-batch-lookup.conf b/asterixdb/asterix-app/src/test/resources/cc-batch-lookup.conf
new file mode 100644
index 0000000..6e10481
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/cc-batch-lookup.conf
@@ -0,0 +1,65 @@
+; Licensed to the Apache Software Foundation (ASF) under one
+; or more contributor license agreements. See the NOTICE file
+; distributed with this work for additional information
+; regarding copyright ownership. The ASF licenses this file
+; to you under the Apache License, Version 2.0 (the
+; "License"); you may not use this file except in compliance
+; with the License. You may obtain a copy of the License at
+;
+; http://www.apache.org/licenses/LICENSE-2.0
+;
+; Unless required by applicable law or agreed to in writing,
+; software distributed under the License is distributed on an
+; "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+; KIND, either express or implied. See the License for the
+; specific language governing permissions and limitations
+; under the License.
+
+[nc/asterix_nc1]
+txn.log.dir=target/tmp/asterix_nc1/txnlog
+core.dump.dir=target/tmp/asterix_nc1/coredump
+iodevices=target/tmp/asterix_nc1/iodevice1,
+iodevices=../asterix-server/target/tmp/asterix_nc1/iodevice2
+nc.api.port=19004
+#jvm.args=-agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=5006
+
+[nc/asterix_nc2]
+ncservice.port=9091
+txn.log.dir=target/tmp/asterix_nc2/txnlog
+core.dump.dir=target/tmp/asterix_nc2/coredump
+iodevices=target/tmp/asterix_nc2/iodevice1,../asterix-server/target/tmp/asterix_nc2/iodevice2
+nc.api.port=19005
+#jvm.args=-agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=5007
+
+[nc]
+credential.file=src/test/resources/security/passwd
+python.cmd.autolocate=true
+python.env=FOO=BAR=BAZ,BAR=BAZ
+address=127.0.0.1
+command=asterixnc
+app.class=org.apache.asterix.hyracks.bootstrap.NCApplication
+jvm.args=-Xmx4096m -Dnode.Resolver="org.apache.asterix.external.util.IdentitiyResolverFactory"
+storage.buffercache.pagesize=32KB
+storage.buffercache.size=128MB
+storage.memorycomponent.globalbudget=512MB
+
+[cc]
+address = 127.0.0.1
+app.class=org.apache.asterix.hyracks.bootstrap.CCApplication
+heartbeat.period=2000
+heartbeat.max.misses=25
+credential.file=src/test/resources/security/passwd
+
+[common]
+log.dir = logs/
+log.level = INFO
+compiler.framesize=32KB
+compiler.sortmemory=320KB
+compiler.groupmemory=160KB
+compiler.joinmemory=256KB
+compiler.textsearchmemory=160KB
+compiler.windowmemory=192KB
+compiler.batch.lookup=true
+messaging.frame.size=4096
+messaging.frame.count=512
+
diff --git a/asterixdb/asterix-app/src/test/resources/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/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/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/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.10.query.sqlpp
similarity index 84%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.10.query.sqlpp
index 6095b26..53097a4 100644
--- 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-2/compileonly-2.10.query.sqlpp
@@ -17,8 +17,10 @@
* under the License.
*/
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Check that objects were not created
+ */
-select value v from range(1,2) v where v > ?;
+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/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.2.query.sqlpp
similarity index 84%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.2.query.sqlpp
index 6095b26..1e3fb34 100644
--- 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-2/compileonly-2.2.query.sqlpp
@@ -17,8 +17,7 @@
* 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 > ?;
+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/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.3.ddl.sqlpp
similarity index 77%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.3.ddl.sqlpp
index 6095b26..b913179 100644
--- 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-2/compileonly-2.3.ddl.sqlpp
@@ -17,8 +17,29 @@
* under the License.
*/
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Test that DROP DDL statements are not executed in compile-only mode
+ */
-select value v from range(1,2) v where v > ?;
+-- 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/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.4.query.sqlpp
similarity index 84%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.4.query.sqlpp
index 6095b26..5760b8e 100644
--- 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-2/compileonly-2.4.query.sqlpp
@@ -17,8 +17,11 @@
* under the License.
*/
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Check that objects were not dropped
+ */
-select value v from range(1,2) v where v > ?;
+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/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.5.ddl.sqlpp
similarity index 87%
rename from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
rename to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.5.ddl.sqlpp
index 6095b26..37146d7 100644
--- 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-2/compileonly-2.5.ddl.sqlpp
@@ -17,8 +17,10 @@
* under the License.
*/
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Test that DROP DDL statements are not executed in compile-only mode
+ */
-select value v from range(1,2) v where v > ?;
+-- param compile-only:string=true
+
+drop dataverse test1;
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-2/compileonly-2.6.query.sqlpp
similarity index 84%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.6.query.sqlpp
index 6095b26..5760b8e 100644
--- 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-2/compileonly-2.6.query.sqlpp
@@ -17,8 +17,11 @@
* under the License.
*/
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Check that objects were not dropped
+ */
-select value v from range(1,2) v where v > ?;
+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/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.7.ddl.sqlpp
similarity index 70%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.7.ddl.sqlpp
index 6095b26..9b41631 100644
--- 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-2/compileonly-2.7.ddl.sqlpp
@@ -17,8 +17,27 @@
* under the License.
*/
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Test that CREATE DDL statements are not executed in compile-only mode
+ */
-select value v from range(1,2) v where v > ?;
+-- 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/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.8.query.sqlpp
similarity index 84%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.8.query.sqlpp
index 6095b26..c5afa49 100644
--- 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-2/compileonly-2.8.query.sqlpp
@@ -17,8 +17,11 @@
* under the License.
*/
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Check that objects were not created
+ */
-select value v from range(1,2) v where v > ?;
+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/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.9.ddl.sqlpp
similarity index 85%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.9.ddl.sqlpp
index 6095b26..7fb26db 100644
--- 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-2/compileonly-2.9.ddl.sqlpp
@@ -17,8 +17,11 @@
* under the License.
*/
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Test that CREATE DDL statements are not executed in compile-only mode
+ */
-select value v from range(1,2) v where v > ?;
+-- 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.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.ddl.sqlpp
similarity index 84%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.ddl.sqlpp
index 6095b26..82e9f1c 100644
--- 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.ddl.sqlpp
@@ -17,8 +17,7 @@
* under the License.
*/
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+drop dataverse test1 if exists;
+create dataverse test1;
-select value v from range(1,2) v where v > ?;
+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.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.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.5.plans.sqlpp
similarity index 81%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.5.plans.sqlpp
index 6095b26..5947c5d 100644
--- 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.5.plans.sqlpp
@@ -17,8 +17,11 @@
* under the License.
*/
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Test additional information returned when client-type=jdbc (update statement)
+ */
-select value v from range(1,2) v where v > ?;
+-- 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/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/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.10.ddl.sqlpp
similarity index 78%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.10.ddl.sqlpp
index 6095b26..da5fe13 100644
--- 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/ddl/analyze-dataset-1/analyze-dataset-1.10.ddl.sqlpp
@@ -17,8 +17,11 @@
* under the License.
*/
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: analyze dataset with sample=medium.
+ * Note, there are fewer tuples in the dataset that the target sample size
+ */
-select value v from range(1,2) v where v > ?;
+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/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.11.query.sqlpp
similarity index 78%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.11.query.sqlpp
index 6095b26..38ded0a 100644
--- 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/ddl/analyze-dataset-1/analyze-dataset-1.11.query.sqlpp
@@ -17,8 +17,14 @@
* under the License.
*/
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: check that the sample was re-created
+ */
-select value v from range(1,2) v where v > ?;
+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/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.12.update.sqlpp
similarity index 84%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.12.update.sqlpp
index 6095b26..0e3886d 100644
--- 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/ddl/analyze-dataset-1/analyze-dataset-1.12.update.sqlpp
@@ -17,8 +17,12 @@
* under the License.
*/
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: Insert more data
+ */
-select value v from range(1,2) v where v > ?;
+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/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.13.ddl.sqlpp
similarity index 80%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.13.ddl.sqlpp
index 6095b26..b098016 100644
--- 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/ddl/analyze-dataset-1/analyze-dataset-1.13.ddl.sqlpp
@@ -17,8 +17,11 @@
* under the License.
*/
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: analyze dataset with sample=medium
+ * Note, there are more tuples in the dataset that the target sample size
+ */
-select value v from range(1,2) v where v > ?;
+use test;
+
+analyze dataset ds1 with { "sample": "medium" };
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/ddl/analyze-dataset-1/analyze-dataset-1.14.query.sqlpp
similarity index 77%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.14.query.sqlpp
index 6095b26..4cae202 100644
--- 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/ddl/analyze-dataset-1/analyze-dataset-1.14.query.sqlpp
@@ -17,8 +17,14 @@
* under the License.
*/
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: check that the sample was re-created
+ */
-select value v from range(1,2) v where v > ?;
+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/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.15.ddl.sqlpp
similarity index 78%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.15.ddl.sqlpp
index 6095b26..6ceb8141 100644
--- 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/ddl/analyze-dataset-1/analyze-dataset-1.15.ddl.sqlpp
@@ -17,8 +17,11 @@
* under the License.
*/
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: analyze dataset with sample=high.
+ * Note, there are fewer tuples in the dataset that the target sample size
+ */
-select value v from range(1,2) v where v > ?;
+use test;
+
+analyze dataset ds1 with { "sample": "high", "sample-seed": "345" };
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/ddl/analyze-dataset-1/analyze-dataset-1.16.query.sqlpp
similarity index 78%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.16.query.sqlpp
index 6095b26..38ded0a 100644
--- 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/ddl/analyze-dataset-1/analyze-dataset-1.16.query.sqlpp
@@ -17,8 +17,14 @@
* under the License.
*/
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: check that the sample was re-created
+ */
-select value v from range(1,2) v where v > ?;
+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/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.17.update.sqlpp
similarity index 84%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.17.update.sqlpp
index 6095b26..0c5f8ad 100644
--- 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/ddl/analyze-dataset-1/analyze-dataset-1.17.update.sqlpp
@@ -17,8 +17,12 @@
* under the License.
*/
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: Insert more data
+ */
-select value v from range(1,2) v where v > ?;
+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/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.18.ddl.sqlpp
similarity index 80%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.18.ddl.sqlpp
index 6095b26..4385f95 100644
--- 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/ddl/analyze-dataset-1/analyze-dataset-1.18.ddl.sqlpp
@@ -17,8 +17,11 @@
* under the License.
*/
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: analyze dataset with sample=high
+ * Note, there are more tuples in the dataset that the target sample size
+ */
-select value v from range(1,2) v where v > ?;
+use test;
+
+analyze dataset ds1 with { "sample": "high" };
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/ddl/analyze-dataset-1/analyze-dataset-1.19.query.sqlpp
similarity index 77%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.19.query.sqlpp
index 6095b26..4cae202 100644
--- 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/ddl/analyze-dataset-1/analyze-dataset-1.19.query.sqlpp
@@ -17,8 +17,14 @@
* under the License.
*/
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: check that the sample was re-created
+ */
-select value v from range(1,2) v where v > ?;
+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/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.2.query.sqlpp
similarity index 83%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.2.query.sqlpp
index 6095b26..e786e0e 100644
--- 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/ddl/analyze-dataset-1/analyze-dataset-1.2.query.sqlpp
@@ -17,8 +17,11 @@
* under the License.
*/
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: Check that the sample index was created even thought
+ * the source dataset is empty
+ */
-select value v from range(1,2) v where v > ?;
+use test;
+
+listMetadata(true, false);
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/ddl/analyze-dataset-1/analyze-dataset-1.20.ddl.sqlpp
similarity index 84%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.20.ddl.sqlpp
index 6095b26..ffa19b4a 100644
--- 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/ddl/analyze-dataset-1/analyze-dataset-1.20.ddl.sqlpp
@@ -17,8 +17,11 @@
* under the License.
*/
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: test that the sample index is dropped when
+ * its source dataset is dropped
+ */
-select value v from range(1,2) v where v > ?;
+use test;
+
+drop dataset ds1;
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/ddl/analyze-dataset-1/analyze-dataset-1.21.query.sqlpp
similarity index 84%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.21.query.sqlpp
index 6095b26..759fc3f 100644
--- 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/ddl/analyze-dataset-1/analyze-dataset-1.21.query.sqlpp
@@ -17,8 +17,11 @@
* under the License.
*/
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: check that the sample was dropped
+ */
-select value v from range(1,2) v where v > ?;
+use test;
+
+select count(*) cnt
+from listMetadata(true, false) v;
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/ddl/analyze-dataset-1/analyze-dataset-1.3.update.sqlpp
similarity index 84%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.3.update.sqlpp
index 6095b26..1b83b99 100644
--- 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/ddl/analyze-dataset-1/analyze-dataset-1.3.update.sqlpp
@@ -17,8 +17,8 @@
* under the License.
*/
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+use test;
-select value v from range(1,2) v where v > ?;
+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/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.4.ddl.sqlpp
similarity index 84%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.4.ddl.sqlpp
index 6095b26..ed97897 100644
--- 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/ddl/analyze-dataset-1/analyze-dataset-1.4.ddl.sqlpp
@@ -17,8 +17,8 @@
* under the License.
*/
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: Test sample size parameter
+ */
-select value v from range(1,2) v where v > ?;
+analyze dataset test.ds1 with { "sample": "low", "sample-seed": 123 };
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/ddl/analyze-dataset-1/analyze-dataset-1.5.query.sqlpp
similarity index 72%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.5.query.sqlpp
index 6095b26..e0cd6cc 100644
--- 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/ddl/analyze-dataset-1/analyze-dataset-1.5.query.sqlpp
@@ -17,8 +17,15 @@
* under the License.
*/
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: check that sample index was recreated with a new name.
+ * Also check that all 8 dataset tuples are in the sample
+ */
-select value v from range(1,2) v where v > ?;
+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/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.6.update.sqlpp
similarity index 84%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.6.update.sqlpp
index 6095b26..340fbb3 100644
--- 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/ddl/analyze-dataset-1/analyze-dataset-1.6.update.sqlpp
@@ -17,8 +17,12 @@
* under the License.
*/
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: Insert more data
+ */
-select value v from range(1,2) v where v > ?;
+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/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.7.query.sqlpp
similarity index 72%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.7.query.sqlpp
index 6095b26..c4930b0 100644
--- 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/ddl/analyze-dataset-1/analyze-dataset-1.7.query.sqlpp
@@ -17,8 +17,15 @@
* under the License.
*/
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: check that the sample content remains the same
+ * after INSERT because we did not run ANALYZE DATASET after that
+ */
-select value v from range(1,2) v where v > ?;
+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/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.8.ddl.sqlpp
similarity index 80%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.8.ddl.sqlpp
index 6095b26..2f452f0 100644
--- 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/ddl/analyze-dataset-1/analyze-dataset-1.8.ddl.sqlpp
@@ -17,8 +17,11 @@
* under the License.
*/
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: analyze dataset with sample=low.
+ * Note, there are more tuples in the dataset that the target sample size
+ */
-select value v from range(1,2) v where v > ?;
+use test;
+
+analyze dataset ds1 with { "sample": "low" };
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/ddl/analyze-dataset-1/analyze-dataset-1.9.query.sqlpp
similarity index 77%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.9.query.sqlpp
index 6095b26..4cae202 100644
--- 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/ddl/analyze-dataset-1/analyze-dataset-1.9.query.sqlpp
@@ -17,8 +17,14 @@
* under the License.
*/
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: check that the sample was re-created
+ */
-select value v from range(1,2) v where v > ?;
+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/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q18/q18.4.query.sqlpp
similarity index 72%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q18/q18.4.query.sqlpp
index 6095b26..1f7c7d5 100644
--- 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/tpcds/q18/q18.4.query.sqlpp
@@ -17,8 +17,14 @@
* under the License.
*/
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+// see ASTERIXDB-3043
+SET `compiler.joinmemory` "160KB";
-select value v from range(1,2) v where v > ?;
+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/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q18/q18.5.query.sqlpp
similarity index 74%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q18/q18.5.query.sqlpp
index 6095b26..3a7ae0b 100644
--- 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/tpcds/q18/q18.5.query.sqlpp
@@ -17,8 +17,14 @@
* under the License.
*/
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+// see ASTERIXDB-3043
+SET `compiler.joinmemory` "160KB";
-select value v from range(1,2) v where v > ?;
+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/results/api/cluster_state_1/cluster_state_1.1.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm
index c55c0bc..54793dc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm
@@ -10,6 +10,7 @@
"active\.suspend\.timeout" : 3600,
"azure.request.timeout" : 120,
"compiler\.arrayindex" : true,
+ "compiler.batch.lookup" : false,
"compiler\.external\.field\.pushdown" : true,
"compiler\.framesize" : 32768,
"compiler\.groupmemory" : 163840,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm
index 661daf3..06da0ad 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm
@@ -10,6 +10,7 @@
"active\.suspend\.timeout" : 3600,
"azure.request.timeout" : 120,
"compiler\.arrayindex" : true,
+ "compiler.batch.lookup" : false,
"compiler\.external\.field\.pushdown" : true,
"compiler\.framesize" : 32768,
"compiler\.groupmemory" : 163840,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm
index 1f0e865..bef14c1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm
@@ -10,6 +10,7 @@
"active\.suspend\.timeout" : 3600,
"azure.request.timeout" : 120,
"compiler\.arrayindex" : true,
+ "compiler.batch.lookup" : false,
"compiler\.external\.field\.pushdown" : true,
"compiler\.framesize" : 32768,
"compiler\.groupmemory" : 163840,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/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.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.1.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.2.regexjson
similarity index 100%
rename from asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.1.regexjson
rename to asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.2.regexjson
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/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.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/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
index 9a536a6..e89d96a 100644
--- 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
@@ -2,7 +2,7 @@
-- DISTRIBUTE_RESULT |UNPARTITIONED|
exchange
-- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- assign [$$2] <- [{ a: "1", b: 1 }]
+ 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-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
index c637d87..2a8a531 100644
--- 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
@@ -2,7 +2,7 @@
-- DISTRIBUTE_RESULT |UNPARTITIONED|
exchange
-- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- assign [$$2] <- [{ a: "1" }]
+ assign [$$2] <- [{ "a": "1" }]
-- ASSIGN |UNPARTITIONED|
empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
\ 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.15.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.15.adm
index de214d0..02a5277 100644
--- 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
@@ -15,7 +15,7 @@
subplan {
aggregate [$$39] <- [empty-stream()]
-- AGGREGATE |LOCAL|
- select (not(if-missing-or-null(eq($$48, "string"), FALSE)))
+ select (not(if-missing-or-null(eq($$48, "string"), false)))
-- STREAM_SELECT |LOCAL|
assign [$$48] <- [$$ht.getField("display_url")]
-- ASSIGN |LOCAL|
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
index 9f122c6..f0a3dcb 100644
--- 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
@@ -15,7 +15,7 @@
subplan {
aggregate [$$39] <- [empty-stream()]
-- AGGREGATE |LOCAL|
- select (not(if-missing-or-null(eq($$48, "string"), FALSE)))
+ select (not(if-missing-or-null(eq($$48, "string"), false)))
-- STREAM_SELECT |LOCAL|
assign [$$48] <- [$$ht.getField("display_url")]
-- ASSIGN |LOCAL|
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
index 4a602ef..6114e6d 100644
--- 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
@@ -4,7 +4,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
project ([$$20])
-- STREAM_PROJECT |PARTITIONED|
- assign [$$20] <- [switch-case(TRUE, is-array($$22), $$24, $$25)]
+ assign [$$20] <- [switch-case(true, is-array($$22), $$24, $$25)]
-- ASSIGN |PARTITIONED|
project ([$$22, $$24, $$25])
-- STREAM_PROJECT |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
index 56dd2ee..420abf4 100644
--- 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
@@ -4,7 +4,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
project ([$$20])
-- STREAM_PROJECT |PARTITIONED|
- assign [$$20] <- [switch-case(TRUE, is-array($$22), $$24, $$25)]
+ assign [$$20] <- [switch-case(true, is-array($$22), $$24, $$25)]
-- ASSIGN |PARTITIONED|
project ([$$22, $$24, $$25])
-- STREAM_PROJECT |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
index 30d5ac3..cfe8897 100644
--- 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
@@ -93,7 +93,7 @@
-- HASH_PARTITION_EXCHANGE [$$81] |PARTITIONED|
project ([$$104, $$81])
-- STREAM_PROJECT |PARTITIONED|
- assign [$$104, $$81] <- [TRUE, $$ht2.getField("text")]
+ assign [$$104, $$81] <- [true, $$ht2.getField("text")]
-- ASSIGN |PARTITIONED|
project ([$$ht2])
-- STREAM_PROJECT |PARTITIONED|
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/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
index 3543f5d..607ed1a 100644
--- 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
@@ -12,7 +12,7 @@
-- 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
+ 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|
@@ -24,11 +24,11 @@
-- 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)
+ 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|
+ -- 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-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
index 1f4ef66..cad5cf0 100644
--- 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
@@ -18,7 +18,7 @@
-- 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
+ 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|
@@ -30,11 +30,11 @@
-- 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)
+ 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|
+ -- 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-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
index a0a3c84..60dedd5 100644
--- 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
@@ -12,7 +12,7 @@
-- 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
+ 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|
@@ -24,11 +24,11 @@
-- 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)
+ 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|
+ -- 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-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
index 44507f4..717f424 100644
--- 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
@@ -12,7 +12,7 @@
-- 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
+ 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|
@@ -24,11 +24,11 @@
-- 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)
+ 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|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
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
index f0da628..01a297a 100644
--- 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
@@ -6,7 +6,7 @@
-- 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 $$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/misc/constant_folding/constant_folding.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.1.adm
index 379603f..c757014 100644
--- 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
@@ -2,7 +2,7 @@
-- DISTRIBUTE_RESULT |UNPARTITIONED|
exchange
-- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- assign [$$5] <- [{ f1: 5, f2: 6, f3: 7 }]
+ 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.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.3.adm
index aafd2d0..7a73900 100644
--- 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
@@ -2,7 +2,7 @@
-- DISTRIBUTE_RESULT |UNPARTITIONED|
exchange
-- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- assign [$$14] <- [TRUE]
+ assign [$$14] <- [true]
-- ASSIGN |UNPARTITIONED|
project ([])
-- STREAM_PROJECT |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
index d2f3187..775cc2a 100644
--- 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
@@ -4,7 +4,7 @@
-- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
project ([$$16])
-- STREAM_PROJECT |UNPARTITIONED|
- assign [$$16] <- [or(TRUE, lt(get-year(current-date()), $$x))]
+ assign [$$16] <- [or(true, lt(get-year(current-date()), $$x))]
-- ASSIGN |UNPARTITIONED|
exchange
-- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
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
index e14f391..6faed2f 100644
--- 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
@@ -10,7 +10,7 @@
-- 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)
+ 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|
@@ -22,7 +22,7 @@
-- 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)
+ 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|
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
index e14f391..6faed2f 100644
--- 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
@@ -10,7 +10,7 @@
-- 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)
+ 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|
@@ -22,7 +22,7 @@
-- 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)
+ 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|
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/testsuite_sqlpp.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
index c0784c1..7aef277 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>
@@ -4120,6 +4125,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>
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-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..9599435 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
@@ -103,7 +103,11 @@
COMPILER_ARRAYINDEX(
BOOLEAN,
AlgebricksConfig.ARRAY_INDEX_DEFAULT,
- "Enable/disable using array-indexes in queries");
+ "Enable/disable using array-indexes in queries"),
+ COMPILER_BATCH_LOOKUP(
+ BOOLEAN,
+ AlgebricksConfig.BATCH_LOOKUP_DEFAULT,
+ "Enable/disable batch point-lookups when running queries with secondary indexes");
private final IOptionType type;
private final Object defaultValue;
@@ -173,6 +177,8 @@
public static final String COMPILER_EXTERNALSCANMEMORY_KEY = Option.COMPILER_EXTERNALSCANMEMORY.ini();
+ public static final String COMPILER_BATCHED_LOOKUP_KEY = Option.COMPILER_BATCH_LOOKUP.ini();
+
public static final int COMPILER_PARALLELISM_AS_STORAGE = 0;
public CompilerProperties(PropertiesAccessor accessor) {
@@ -246,4 +252,8 @@
public int getExternalScanMemorySize() {
return accessor.getInt(Option.COMPILER_EXTERNALSCANMEMORY);
}
+
+ public boolean isBatchLookup() {
+ return accessor.getBoolean(Option.COMPILER_BATCH_LOOKUP);
+ }
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/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..ab3a925 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/OptimizationConfUtil.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/OptimizationConfUtil.java
@@ -77,6 +77,8 @@
int externalScanBufferSize = getExternalScanBufferSize(
(String) querySpecificConfig.get(CompilerProperties.COMPILER_EXTERNALSCANMEMORY_KEY),
compilerProperties.getExternalScanMemorySize(), sourceLoc);
+ boolean batchLookup = getBoolean(querySpecificConfig, CompilerProperties.COMPILER_BATCHED_LOOKUP_KEY,
+ compilerProperties.isBatchLookup());
PhysicalOptimizationConfig physOptConf = new PhysicalOptimizationConfig();
physOptConf.setFrameSize(frameSize);
@@ -95,6 +97,7 @@
physOptConf.setMinMemoryAllocation(minMemoryAllocation);
physOptConf.setArrayIndexEnabled(arrayIndex);
physOptConf.setExternalScanBufferSize(externalScanBufferSize);
+ physOptConf.setBatchLookup(batchLookup);
return physOptConf;
}
@@ -117,6 +120,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,
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 fc234ef..2235005 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
@@ -265,6 +265,7 @@
S3_REGION_NOT_SUPPORTED(1170),
COMPILATION_SET_OPERATION_ERROR(1171),
INVALID_TIMEZONE(1172),
+ INVALID_PARAM_VALUE_ALLOWED_VALUE(1173),
// 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/metadata/IMetadataLockUtil.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/IMetadataLockUtil.java
index 28eb553..684ddae 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,9 @@
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 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/resources/asx_errormsg/en.properties b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
index b98dc43..7455a3b 100644
--- a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
+++ b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
@@ -267,6 +267,7 @@
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
# 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-external-data/pom.xml b/asterixdb/asterix-external-data/pom.xml
index dbb2382..abc8c6c 100644
--- a/asterixdb/asterix-external-data/pom.xml
+++ b/asterixdb/asterix-external-data/pom.xml
@@ -518,6 +518,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>
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStream.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStream.java
index f14af53..bbcf9cd 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStream.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStream.java
@@ -18,7 +18,6 @@
*/
package org.apache.asterix.external.input.record.reader.aws;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3;
import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
import java.io.IOException;
@@ -32,7 +31,7 @@
import org.apache.asterix.common.exceptions.RuntimeDataException;
import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStream;
import org.apache.asterix.external.util.ExternalDataConstants;
-import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.asterix.external.util.aws.s3.S3Utils;
import org.apache.commons.lang3.StringUtils;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.util.CleanupUtils;
@@ -109,7 +108,7 @@
}
private boolean shouldRetry(String errorCode, int currentRetry) {
- return currentRetry < MAX_RETRIES && AwsS3.isRetryableError(errorCode);
+ return currentRetry < MAX_RETRIES && S3Utils.isRetryableError(errorCode);
}
@Override
@@ -134,7 +133,7 @@
private S3Client buildAwsS3Client(Map<String, String> configuration) throws HyracksDataException {
try {
- return ExternalDataUtils.AwsS3.buildAwsS3Client(configuration);
+ return S3Utils.buildAwsS3Client(configuration);
} catch (CompilationException ex) {
throw HyracksDataException.create(ex);
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStreamFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStreamFactory.java
index 89ea39e..a241354 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStreamFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStreamFactory.java
@@ -26,6 +26,7 @@
import org.apache.asterix.external.api.AsterixInputStream;
import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStreamFactory;
import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.asterix.external.util.aws.s3.S3Utils;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.api.application.IServiceContext;
import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -53,8 +54,7 @@
IncludeExcludeMatcher includeExcludeMatcher = ExternalDataUtils.getIncludeExcludeMatchers(configuration);
//Get a list of S3 objects
- List<S3Object> filesOnly =
- ExternalDataUtils.AwsS3.listS3Objects(configuration, includeExcludeMatcher, warningCollector);
+ List<S3Object> filesOnly = S3Utils.listS3Objects(configuration, includeExcludeMatcher, warningCollector);
// Distribute work load amongst the partitions
distributeWorkLoad(filesOnly, getPartitionsCount());
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/parquet/AwsS3ParquetReaderFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/parquet/AwsS3ParquetReaderFactory.java
index 803e657..ff93a46 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/parquet/AwsS3ParquetReaderFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/parquet/AwsS3ParquetReaderFactory.java
@@ -28,6 +28,8 @@
import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStreamFactory.IncludeExcludeMatcher;
import org.apache.asterix.external.util.ExternalDataConstants;
import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.asterix.external.util.aws.s3.S3Constants;
+import org.apache.asterix.external.util.aws.s3.S3Utils;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.api.application.IServiceContext;
@@ -52,7 +54,7 @@
//Configure Hadoop S3 input splits
JobConf conf = createHdfsConf(serviceCtx, configuration);
int numberOfPartitions = getPartitionConstraint().getLocations().length;
- ExternalDataUtils.AwsS3.configureAwsS3HdfsJobConf(conf, configuration, numberOfPartitions);
+ S3Utils.configureAwsS3HdfsJobConf(conf, configuration, numberOfPartitions);
configureHdfsConf(conf, configuration);
}
@@ -89,8 +91,7 @@
throws CompilationException {
String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
IncludeExcludeMatcher includeExcludeMatcher = ExternalDataUtils.getIncludeExcludeMatchers(configuration);
- List<S3Object> filesOnly =
- ExternalDataUtils.AwsS3.listS3Objects(configuration, includeExcludeMatcher, warningCollector);
+ List<S3Object> filesOnly = S3Utils.listS3Objects(configuration, includeExcludeMatcher, warningCollector);
StringBuilder builder = new StringBuilder();
if (!filesOnly.isEmpty()) {
@@ -105,7 +106,7 @@
}
private static void appendFileURI(StringBuilder builder, String container, S3Object file) {
- builder.append(ExternalDataConstants.AwsS3.HADOOP_S3_PROTOCOL);
+ builder.append(S3Constants.HADOOP_S3_PROTOCOL);
builder.append("://");
builder.append(container);
builder.append('/');
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/blob/AzureBlobInputStream.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/blob/AzureBlobInputStream.java
index cdb3834..bbfece2 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/blob/AzureBlobInputStream.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/blob/AzureBlobInputStream.java
@@ -18,6 +18,7 @@
*/
package org.apache.asterix.external.input.record.reader.azure.blob;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.buildAzureBlobClient;
import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
import java.io.IOException;
@@ -31,7 +32,6 @@
import org.apache.asterix.common.exceptions.RuntimeDataException;
import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStream;
import org.apache.asterix.external.util.ExternalDataConstants;
-import org.apache.asterix.external.util.ExternalDataUtils;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.util.LogRedactionUtil;
@@ -86,7 +86,7 @@
private BlobServiceClient buildAzureClient(IApplicationContext appCtx, Map<String, String> configuration)
throws HyracksDataException {
try {
- return ExternalDataUtils.Azure.buildAzureBlobClient(appCtx, configuration);
+ return buildAzureBlobClient(appCtx, configuration);
} catch (CompilationException ex) {
throw HyracksDataException.create(ex);
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/blob/AzureBlobInputStreamFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/blob/AzureBlobInputStreamFactory.java
index 064b319..55c0521 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/blob/AzureBlobInputStreamFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/blob/AzureBlobInputStreamFactory.java
@@ -18,6 +18,9 @@
*/
package org.apache.asterix.external.input.record.reader.azure.blob;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.buildAzureBlobClient;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.listBlobItems;
+
import java.util.Comparator;
import java.util.List;
import java.util.Map;
@@ -57,9 +60,9 @@
// Ensure the validity of include/exclude
ExternalDataUtils.validateIncludeExclude(configuration);
IncludeExcludeMatcher includeExcludeMatcher = ExternalDataUtils.getIncludeExcludeMatchers(configuration);
- BlobServiceClient blobServiceClient = ExternalDataUtils.Azure.buildAzureBlobClient(appCtx, configuration);
- List<BlobItem> filesOnly = ExternalDataUtils.Azure.listBlobItems(blobServiceClient, configuration,
- includeExcludeMatcher, warningCollector);
+ BlobServiceClient blobServiceClient = buildAzureBlobClient(appCtx, configuration);
+ List<BlobItem> filesOnly =
+ listBlobItems(blobServiceClient, configuration, includeExcludeMatcher, warningCollector);
// Distribute work load amongst the partitions
distributeWorkLoad(filesOnly, getPartitionsCount());
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/datalake/AzureDataLakeInputStream.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/datalake/AzureDataLakeInputStream.java
index e34d188..7a95222 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/datalake/AzureDataLakeInputStream.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/datalake/AzureDataLakeInputStream.java
@@ -18,6 +18,7 @@
*/
package org.apache.asterix.external.input.record.reader.azure.datalake;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.buildAzureDatalakeClient;
import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
import java.io.IOException;
@@ -31,7 +32,6 @@
import org.apache.asterix.common.exceptions.RuntimeDataException;
import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStream;
import org.apache.asterix.external.util.ExternalDataConstants;
-import org.apache.asterix.external.util.ExternalDataUtils;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.util.LogRedactionUtil;
@@ -86,7 +86,7 @@
private DataLakeServiceClient buildAzureClient(IApplicationContext appCtx, Map<String, String> configuration)
throws HyracksDataException {
try {
- return ExternalDataUtils.Azure.buildAzureDatalakeClient(appCtx, configuration);
+ return buildAzureDatalakeClient(appCtx, configuration);
} catch (CompilationException ex) {
throw HyracksDataException.create(ex);
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/datalake/AzureDataLakeInputStreamFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/datalake/AzureDataLakeInputStreamFactory.java
index e9f8d4c..929cb6e 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/datalake/AzureDataLakeInputStreamFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/datalake/AzureDataLakeInputStreamFactory.java
@@ -18,6 +18,9 @@
*/
package org.apache.asterix.external.input.record.reader.azure.datalake;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.buildAzureDatalakeClient;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.listDatalakePathItems;
+
import java.util.Comparator;
import java.util.List;
import java.util.Map;
@@ -57,9 +60,9 @@
// Ensure the validity of include/exclude
ExternalDataUtils.validateIncludeExclude(configuration);
IncludeExcludeMatcher includeExcludeMatcher = ExternalDataUtils.getIncludeExcludeMatchers(configuration);
- DataLakeServiceClient client = ExternalDataUtils.Azure.buildAzureDatalakeClient(appCtx, configuration);
- List<PathItem> filesOnly = ExternalDataUtils.Azure.listDatalakePathItems(client, configuration,
- includeExcludeMatcher, warningCollector);
+ DataLakeServiceClient client = buildAzureDatalakeClient(appCtx, configuration);
+ List<PathItem> filesOnly =
+ listDatalakePathItems(client, configuration, includeExcludeMatcher, warningCollector);
// Distribute work load amongst the partitions
distributeWorkLoad(filesOnly, getPartitionsCount());
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/parquet/AzureBlobParquetReaderFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/parquet/AzureBlobParquetReaderFactory.java
index c2251df..e08013c 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/parquet/AzureBlobParquetReaderFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/parquet/AzureBlobParquetReaderFactory.java
@@ -18,6 +18,11 @@
*/
package org.apache.asterix.external.input.record.reader.azure.parquet;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.HADOOP_AZURE_BLOB_PROTOCOL;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.buildAzureBlobClient;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.configureAzureHdfsJobConf;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.listBlobItems;
+
import java.util.Collections;
import java.util.List;
import java.util.Map;
@@ -47,7 +52,7 @@
public void configure(IServiceContext serviceCtx, Map<String, String> configuration,
IWarningCollector warningCollector) throws AlgebricksException, HyracksDataException {
IApplicationContext appCtx = (IApplicationContext) serviceCtx.getApplicationContext();
- BlobServiceClient blobServiceClient = ExternalDataUtils.Azure.buildAzureBlobClient(appCtx, configuration);
+ BlobServiceClient blobServiceClient = buildAzureBlobClient(appCtx, configuration);
//Get endpoint
String endPoint = extractEndPoint(blobServiceClient.getAccountUrl());
//Get path
@@ -57,7 +62,7 @@
//Configure Hadoop Azure input splits
JobConf conf = createHdfsConf(serviceCtx, configuration);
- ExternalDataUtils.Azure.configureAzureHdfsJobConf(conf, configuration, endPoint);
+ configureAzureHdfsJobConf(conf, configuration, endPoint);
configureHdfsConf(conf, configuration);
}
@@ -94,8 +99,8 @@
private static String buildPathURIs(Map<String, String> configuration, IWarningCollector warningCollector,
BlobServiceClient blobServiceClient, String endPoint) throws CompilationException {
IncludeExcludeMatcher includeExcludeMatcher = ExternalDataUtils.getIncludeExcludeMatchers(configuration);
- List<BlobItem> filesOnly = ExternalDataUtils.Azure.listBlobItems(blobServiceClient, configuration,
- includeExcludeMatcher, warningCollector);
+ List<BlobItem> filesOnly =
+ listBlobItems(blobServiceClient, configuration, includeExcludeMatcher, warningCollector);
StringBuilder builder = new StringBuilder();
String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
@@ -118,7 +123,7 @@
}
private static void appendFileURI(StringBuilder builder, String container, String endPoint, BlobItem file) {
- builder.append(ExternalDataConstants.Azure.HADOOP_AZURE_BLOB_PROTOCOL);
+ builder.append(HADOOP_AZURE_BLOB_PROTOCOL);
builder.append("://");
builder.append(container);
builder.append('@');
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/parquet/AzureDataLakeParquetReaderFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/parquet/AzureDataLakeParquetReaderFactory.java
index db87868..c98fc8b 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/parquet/AzureDataLakeParquetReaderFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/parquet/AzureDataLakeParquetReaderFactory.java
@@ -18,6 +18,11 @@
*/
package org.apache.asterix.external.input.record.reader.azure.parquet;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.HADOOP_AZURE_DATALAKE_PROTOCOL;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.buildAzureDatalakeClient;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.configureAzureHdfsJobConf;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.listDatalakePathItems;
+
import java.util.Collections;
import java.util.List;
import java.util.Map;
@@ -47,8 +52,7 @@
public void configure(IServiceContext serviceCtx, Map<String, String> configuration,
IWarningCollector warningCollector) throws AlgebricksException, HyracksDataException {
IApplicationContext appCtx = (IApplicationContext) serviceCtx.getApplicationContext();
- DataLakeServiceClient dataLakeServiceClient =
- ExternalDataUtils.Azure.buildAzureDatalakeClient(appCtx, configuration);
+ DataLakeServiceClient dataLakeServiceClient = buildAzureDatalakeClient(appCtx, configuration);
//Get endpoint
String endPoint = extractEndPoint(dataLakeServiceClient.getAccountUrl());
@@ -61,7 +65,7 @@
//Configure Hadoop Azure input splits
JobConf conf = createHdfsConf(serviceCtx, configuration);
- ExternalDataUtils.Azure.configureAzureHdfsJobConf(conf, configuration, endPoint);
+ configureAzureHdfsJobConf(conf, configuration, endPoint);
configureHdfsConf(conf, configuration);
}
@@ -98,8 +102,8 @@
private static String buildPathURIs(Map<String, String> configuration, IWarningCollector warningCollector,
DataLakeServiceClient dataLakeServiceClient, String endPoint) throws CompilationException {
IncludeExcludeMatcher includeExcludeMatcher = ExternalDataUtils.getIncludeExcludeMatchers(configuration);
- List<PathItem> filesOnly = ExternalDataUtils.Azure.listDatalakePathItems(dataLakeServiceClient, configuration,
- includeExcludeMatcher, warningCollector);
+ List<PathItem> filesOnly =
+ listDatalakePathItems(dataLakeServiceClient, configuration, includeExcludeMatcher, warningCollector);
StringBuilder builder = new StringBuilder();
String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
@@ -122,7 +126,7 @@
}
private static void appendFileURI(StringBuilder builder, String container, String endPoint, PathItem file) {
- builder.append(ExternalDataConstants.Azure.HADOOP_AZURE_DATALAKE_PROTOCOL);
+ builder.append(HADOOP_AZURE_DATALAKE_PROTOCOL);
builder.append("://");
builder.append(container);
builder.append('@');
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/GCSInputStream.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/GCSInputStream.java
index 652fa3e..007e8be 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/GCSInputStream.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/GCSInputStream.java
@@ -32,7 +32,7 @@
import org.apache.asterix.common.exceptions.RuntimeDataException;
import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStream;
import org.apache.asterix.external.util.ExternalDataConstants;
-import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.asterix.external.util.google.gcs.GCSUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.util.CleanupUtils;
@@ -134,7 +134,7 @@
private Storage buildClient(Map<String, String> configuration) throws HyracksDataException {
try {
- return ExternalDataUtils.GCS.buildClient(configuration);
+ return GCSUtils.buildClient(configuration);
} catch (CompilationException ex) {
throw HyracksDataException.create(ex);
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/GCSInputStreamFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/GCSInputStreamFactory.java
index 0e7ea90..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/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/util/ExternalDataConstants.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java
index f0b9c90..429706e 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java
@@ -333,107 +333,4 @@
*/
public static final Set<String> VALID_TIME_ZONES = Set.of(TimeZone.getAvailableIDs());
}
-
- public static class AwsS3 {
- private AwsS3() {
- throw new AssertionError("do not instantiate");
- }
-
- public static final String REGION_FIELD_NAME = "region";
- public static final String ACCESS_KEY_ID_FIELD_NAME = "accessKeyId";
- public static final String SECRET_ACCESS_KEY_FIELD_NAME = "secretAccessKey";
- public static final String SESSION_TOKEN_FIELD_NAME = "sessionToken";
- public static final String SERVICE_END_POINT_FIELD_NAME = "serviceEndpoint";
-
- // AWS S3 specific error codes
- public static final String ERROR_INTERNAL_ERROR = "InternalError";
- public static final String ERROR_SLOW_DOWN = "SlowDown";
- public static final String ERROR_METHOD_NOT_IMPLEMENTED = "NotImplemented";
-
- public static boolean isRetryableError(String errorCode) {
- return errorCode.equals(ERROR_INTERNAL_ERROR) || errorCode.equals(ERROR_SLOW_DOWN);
- }
-
- /*
- * Hadoop-AWS
- * AWS connectors for s3 and s3n are deprecated.
- */
- public static final String HADOOP_ACCESS_KEY_ID = "fs.s3a.access.key";
- public static final String HADOOP_SECRET_ACCESS_KEY = "fs.s3a.secret.key";
- public static final String HADOOP_SESSION_TOKEN = "fs.s3a.session.token";
- public static final String HADOOP_REGION = "fs.s3a.region";
- public static final String HADOOP_SERVICE_END_POINT = "fs.s3a.endpoint";
-
- /*
- * Internal configurations
- */
- //Allows accessing directories as file system path
- public static final String HADOOP_PATH_STYLE_ACCESS = "fs.s3a.path.style.access";
- //The number of maximum HTTP connections in connection pool
- public static final String HADOOP_S3_CONNECTION_POOL_SIZE = "fs.s3a.connection.maximum";
- //S3 used protocol
- public static final String HADOOP_S3_PROTOCOL = "s3a";
-
- //Hadoop credentials provider key
- public static final String HADOOP_CREDENTIAL_PROVIDER_KEY = "fs.s3a.aws.credentials.provider";
- //Anonymous credential provider
- public static final String HADOOP_ANONYMOUS_ACCESS = "org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider";
- //Temporary credential provider
- public static final String HADOOP_TEMP_ACCESS = "org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider";
-
- }
-
- /*
- * Note: Azure Blob and Azure Datalake use identical authentication, so they are using the same properties.
- * If they end up diverging, then properties for AzureBlob and AzureDataLake need to be created.
- */
- public static class Azure {
- private Azure() {
- throw new AssertionError("do not instantiate");
- }
-
- /*
- * Asterix Configuration Keys
- */
- public static final String MANAGED_IDENTITY_ID_FIELD_NAME = "managedIdentityId";
- public static final String ACCOUNT_NAME_FIELD_NAME = "accountName";
- public static final String ACCOUNT_KEY_FIELD_NAME = "accountKey";
- public static final String SHARED_ACCESS_SIGNATURE_FIELD_NAME = "sharedAccessSignature";
- public static final String TENANT_ID_FIELD_NAME = "tenantId";
- public static final String CLIENT_ID_FIELD_NAME = "clientId";
- public static final String CLIENT_SECRET_FIELD_NAME = "clientSecret";
- public static final String CLIENT_CERTIFICATE_FIELD_NAME = "clientCertificate";
- public static final String CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME = "clientCertificatePassword";
- public static final String ENDPOINT_FIELD_NAME = "endpoint";
-
- // Specific Azure data lake property
- /*
- The behavior of Data Lake (true file system) is to read the files of the specified prefix only, example:
- storage/myData/personal/file1.json
- storage/myData/personal/file2.json
- storage/myData/file3.json
- If the prefix used is "myData", then only the file file3.json is read. However, if the property "recursive"
- is set to "true" when creating the external dataset, then it goes recursively overall the paths, and the result
- is file1.json, file2.json and file3.json.
- */
- public static final String RECURSIVE_FIELD_NAME = "recursive";
-
- /*
- * Hadoop-Azure
- */
- //Used when accountName and accessKey are provided
- public static final String HADOOP_AZURE_FS_ACCOUNT_KEY = "fs.azure.account.key";
- //Used when a connectionString is provided
- public static final String HADOOP_AZURE_FS_SAS = "fs.azure.sas";
- public static final String HADOOP_AZURE_BLOB_PROTOCOL = "wasbs";
- public static final String HADOOP_AZURE_DATALAKE_PROTOCOL = "abfss";
- }
-
- public static class GCS {
- private GCS() {
- throw new AssertionError("do not instantiate");
- }
-
- public static final String JSON_CREDENTIALS_FIELD_NAME = "jsonCredentials";
- }
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
index 8e38eed..62dc074 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
@@ -18,43 +18,9 @@
*/
package org.apache.asterix.external.util;
-import static com.google.cloud.storage.Storage.BlobListOption;
-import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.apache.asterix.common.exceptions.ErrorCode.EXTERNAL_SOURCE_ERROR;
import static org.apache.asterix.common.exceptions.ErrorCode.INVALID_REQ_PARAM_VAL;
import static org.apache.asterix.common.exceptions.ErrorCode.PARAMETERS_NOT_ALLOWED_AT_SAME_TIME;
import static org.apache.asterix.common.exceptions.ErrorCode.PARAMETERS_REQUIRED;
-import static org.apache.asterix.common.exceptions.ErrorCode.PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT;
-import static org.apache.asterix.common.exceptions.ErrorCode.REQUIRED_PARAM_IF_PARAM_IS_PRESENT;
-import static org.apache.asterix.common.exceptions.ErrorCode.REQUIRED_PARAM_OR_PARAM_IF_PARAM_IS_PRESENT;
-import static org.apache.asterix.common.exceptions.ErrorCode.S3_REGION_NOT_SUPPORTED;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.ACCESS_KEY_ID_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.ERROR_METHOD_NOT_IMPLEMENTED;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_ACCESS_KEY_ID;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_ANONYMOUS_ACCESS;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_CREDENTIAL_PROVIDER_KEY;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_PATH_STYLE_ACCESS;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_S3_CONNECTION_POOL_SIZE;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_S3_PROTOCOL;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_SECRET_ACCESS_KEY;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_SESSION_TOKEN;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_TEMP_ACCESS;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.SECRET_ACCESS_KEY_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.ACCOUNT_KEY_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.ACCOUNT_NAME_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.CLIENT_CERTIFICATE_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.CLIENT_ID_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.CLIENT_SECRET_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.ENDPOINT_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.HADOOP_AZURE_BLOB_PROTOCOL;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.HADOOP_AZURE_FS_ACCOUNT_KEY;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.HADOOP_AZURE_FS_SAS;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.MANAGED_IDENTITY_ID_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.RECURSIVE_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.SHARED_ACCESS_SIGNATURE_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.TENANT_ID_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.GCS.JSON_CREDENTIALS_FIELD_NAME;
import static org.apache.asterix.external.util.ExternalDataConstants.KEY_ADAPTER_NAME_GCS;
import static org.apache.asterix.external.util.ExternalDataConstants.KEY_DELIMITER;
import static org.apache.asterix.external.util.ExternalDataConstants.KEY_ESCAPE;
@@ -65,18 +31,14 @@
import static org.apache.asterix.external.util.ExternalDataConstants.KEY_QUOTE;
import static org.apache.asterix.external.util.ExternalDataConstants.KEY_RECORD_END;
import static org.apache.asterix.external.util.ExternalDataConstants.KEY_RECORD_START;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.validateAzureBlobProperties;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.validateAzureDataLakeProperties;
+import static org.apache.asterix.external.util.google.gcs.GCSUtils.validateProperties;
import static org.apache.asterix.runtime.evaluators.functions.StringEvaluatorUtils.RESERVED_REGEX_CHARS;
-import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
-import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.DataOutputStream;
import java.io.IOException;
-import java.io.InputStream;
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
-import java.net.URI;
-import java.net.URISyntaxException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Base64;
@@ -85,7 +47,6 @@
import java.util.List;
import java.util.Map;
import java.util.Optional;
-import java.util.function.BiPredicate;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import java.util.regex.PatternSyntaxException;
@@ -106,20 +67,17 @@
import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStreamFactory.IncludeExcludeMatcher;
import org.apache.asterix.external.library.JavaLibrary;
import org.apache.asterix.external.util.ExternalDataConstants.ParquetOptions;
+import org.apache.asterix.external.util.aws.s3.S3Utils;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.AUnionType;
import org.apache.asterix.runtime.evaluators.common.NumberUtils;
import org.apache.asterix.runtime.projection.DataProjectionInfo;
import org.apache.asterix.runtime.projection.FunctionCallInformation;
-import org.apache.hadoop.fs.s3a.Constants;
-import org.apache.hadoop.mapred.JobConf;
import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.exceptions.IWarningCollector;
import org.apache.hyracks.api.exceptions.SourceLocation;
-import org.apache.hyracks.api.exceptions.Warning;
-import org.apache.hyracks.api.util.CleanupUtils;
import org.apache.hyracks.dataflow.common.data.parsers.BooleanParserFactory;
import org.apache.hyracks.dataflow.common.data.parsers.DoubleParserFactory;
import org.apache.hyracks.dataflow.common.data.parsers.FloatParserFactory;
@@ -129,46 +87,6 @@
import org.apache.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
import org.apache.hyracks.util.StorageUtil;
-import com.azure.core.credential.AzureSasCredential;
-import com.azure.core.http.rest.PagedIterable;
-import com.azure.identity.ClientCertificateCredentialBuilder;
-import com.azure.identity.ClientSecretCredentialBuilder;
-import com.azure.identity.ManagedIdentityCredentialBuilder;
-import com.azure.storage.blob.BlobContainerClient;
-import com.azure.storage.blob.BlobServiceClient;
-import com.azure.storage.blob.BlobServiceClientBuilder;
-import com.azure.storage.blob.models.BlobItem;
-import com.azure.storage.blob.models.ListBlobsOptions;
-import com.azure.storage.common.StorageSharedKeyCredential;
-import com.azure.storage.common.policy.RequestRetryOptions;
-import com.azure.storage.file.datalake.DataLakeFileSystemClient;
-import com.azure.storage.file.datalake.DataLakeServiceClient;
-import com.azure.storage.file.datalake.DataLakeServiceClientBuilder;
-import com.azure.storage.file.datalake.models.ListPathsOptions;
-import com.azure.storage.file.datalake.models.PathItem;
-import com.google.api.gax.paging.Page;
-import com.google.auth.oauth2.ServiceAccountCredentials;
-import com.google.cloud.storage.Blob;
-import com.google.cloud.storage.Storage;
-import com.google.cloud.storage.StorageOptions;
-
-import software.amazon.awssdk.auth.credentials.AnonymousCredentialsProvider;
-import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
-import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
-import software.amazon.awssdk.auth.credentials.AwsSessionCredentials;
-import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
-import software.amazon.awssdk.core.exception.SdkException;
-import software.amazon.awssdk.regions.Region;
-import software.amazon.awssdk.services.s3.S3Client;
-import software.amazon.awssdk.services.s3.S3ClientBuilder;
-import software.amazon.awssdk.services.s3.model.ListObjectsRequest;
-import software.amazon.awssdk.services.s3.model.ListObjectsResponse;
-import software.amazon.awssdk.services.s3.model.ListObjectsV2Request;
-import software.amazon.awssdk.services.s3.model.ListObjectsV2Response;
-import software.amazon.awssdk.services.s3.model.S3Exception;
-import software.amazon.awssdk.services.s3.model.S3Object;
-import software.amazon.awssdk.services.s3.model.S3Response;
-
public class ExternalDataUtils {
private static final Map<ATypeTag, IValueParserFactory> valueParserFactoryMap = new EnumMap<>(ATypeTag.class);
private static final int DEFAULT_MAX_ARGUMENT_SZ = 1024 * 1024;
@@ -604,16 +522,16 @@
switch (type) {
case ExternalDataConstants.KEY_ADAPTER_NAME_AWS_S3:
- AwsS3.validateProperties(configuration, srcLoc, collector);
+ S3Utils.validateProperties(configuration, srcLoc, collector);
break;
case ExternalDataConstants.KEY_ADAPTER_NAME_AZURE_BLOB:
- Azure.validateAzureBlobProperties(configuration, srcLoc, collector, appCtx);
+ validateAzureBlobProperties(configuration, srcLoc, collector, appCtx);
break;
case ExternalDataConstants.KEY_ADAPTER_NAME_AZURE_DATA_LAKE:
- Azure.validateAzureDataLakeProperties(configuration, srcLoc, collector, appCtx);
+ validateAzureDataLakeProperties(configuration, srcLoc, collector, appCtx);
break;
- case KEY_ADAPTER_NAME_GCS:
- GCS.validateProperties(configuration, srcLoc, collector);
+ case ExternalDataConstants.KEY_ADAPTER_NAME_GCS:
+ validateProperties(configuration, srcLoc, collector);
break;
default:
// Nothing needs to be done
@@ -844,7 +762,7 @@
}
}
- private static boolean isParquetFormat(Map<String, String> properties) {
+ public static boolean isParquetFormat(Map<String, String> properties) {
String inputFormat = properties.get(ExternalDataConstants.KEY_INPUT_FORMAT);
return ExternalDataConstants.CLASS_NAME_PARQUET_INPUT_FORMAT.equals(inputFormat)
|| ExternalDataConstants.INPUT_FORMAT_PARQUET.equals(inputFormat)
@@ -893,1008 +811,6 @@
return encoder.encodeToString(byteArrayOutputStream.toByteArray());
}
- public static class AwsS3 {
- private AwsS3() {
- throw new AssertionError("do not instantiate");
- }
-
- /**
- * Builds the S3 client using the provided configuration
- *
- * @param configuration properties
- * @return S3 client
- * @throws CompilationException CompilationException
- */
- public static S3Client buildAwsS3Client(Map<String, String> configuration) throws CompilationException {
- // TODO(Hussain): Need to ensure that all required parameters are present in a previous step
- String accessKeyId = configuration.get(ACCESS_KEY_ID_FIELD_NAME);
- String secretAccessKey = configuration.get(SECRET_ACCESS_KEY_FIELD_NAME);
- String sessionToken = configuration.get(ExternalDataConstants.AwsS3.SESSION_TOKEN_FIELD_NAME);
- String regionId = configuration.get(ExternalDataConstants.AwsS3.REGION_FIELD_NAME);
- String serviceEndpoint = configuration.get(ExternalDataConstants.AwsS3.SERVICE_END_POINT_FIELD_NAME);
-
- S3ClientBuilder builder = S3Client.builder();
-
- // Credentials
- AwsCredentialsProvider credentialsProvider;
-
- // No auth required
- if (accessKeyId == null) {
- credentialsProvider = AnonymousCredentialsProvider.create();
- } else {
- // auth required, check for temporary or permanent credentials
- if (sessionToken != null) {
- credentialsProvider = StaticCredentialsProvider
- .create(AwsSessionCredentials.create(accessKeyId, secretAccessKey, sessionToken));
- } else {
- credentialsProvider =
- StaticCredentialsProvider.create(AwsBasicCredentials.create(accessKeyId, secretAccessKey));
- }
- }
-
- builder.credentialsProvider(credentialsProvider);
-
- // Validate the region
- List<Region> regions = S3Client.serviceMetadata().regions();
- Optional<Region> selectedRegion =
- regions.stream().filter(region -> region.id().equals(regionId)).findFirst();
-
- if (selectedRegion.isEmpty()) {
- throw new CompilationException(S3_REGION_NOT_SUPPORTED, regionId);
- }
- builder.region(selectedRegion.get());
-
- // Validate the service endpoint if present
- if (serviceEndpoint != null) {
- try {
- URI uri = new URI(serviceEndpoint);
- try {
- builder.endpointOverride(uri);
- } catch (NullPointerException ex) {
- throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
- }
- } catch (URISyntaxException ex) {
- throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR,
- String.format("Invalid service endpoint %s", serviceEndpoint));
- }
- }
-
- return builder.build();
- }
-
- /**
- * Builds the S3 client using the provided configuration
- *
- * @param configuration properties
- * @param numberOfPartitions number of partitions in the cluster
- */
- public static void configureAwsS3HdfsJobConf(JobConf conf, Map<String, String> configuration,
- int numberOfPartitions) {
- String accessKeyId = configuration.get(ExternalDataConstants.AwsS3.ACCESS_KEY_ID_FIELD_NAME);
- String secretAccessKey = configuration.get(ExternalDataConstants.AwsS3.SECRET_ACCESS_KEY_FIELD_NAME);
- String sessionToken = configuration.get(ExternalDataConstants.AwsS3.SESSION_TOKEN_FIELD_NAME);
- String serviceEndpoint = configuration.get(ExternalDataConstants.AwsS3.SERVICE_END_POINT_FIELD_NAME);
-
- //Disable caching S3 FileSystem
- HDFSUtils.disableHadoopFileSystemCache(conf, HADOOP_S3_PROTOCOL);
-
- /*
- * Authentication Methods:
- * 1- Anonymous: no accessKeyId and no secretAccessKey
- * 2- Temporary: has to provide accessKeyId, secretAccessKey and sessionToken
- * 3- Private: has to provide accessKeyId and secretAccessKey
- */
- if (accessKeyId == null) {
- //Tells hadoop-aws it is an anonymous access
- conf.set(HADOOP_CREDENTIAL_PROVIDER_KEY, HADOOP_ANONYMOUS_ACCESS);
- } else {
- conf.set(HADOOP_ACCESS_KEY_ID, accessKeyId);
- conf.set(HADOOP_SECRET_ACCESS_KEY, secretAccessKey);
- if (sessionToken != null) {
- conf.set(HADOOP_SESSION_TOKEN, sessionToken);
- //Tells hadoop-aws it is a temporary access
- conf.set(HADOOP_CREDENTIAL_PROVIDER_KEY, HADOOP_TEMP_ACCESS);
- }
- }
-
- /*
- * This is to allow S3 definition to have path-style form. Should always be true to match the current
- * way we access files in S3
- */
- conf.set(HADOOP_PATH_STYLE_ACCESS, ExternalDataConstants.TRUE);
-
- /*
- * Set the size of S3 connection pool to be the number of partitions
- */
- conf.set(HADOOP_S3_CONNECTION_POOL_SIZE, String.valueOf(numberOfPartitions));
-
- if (serviceEndpoint != null) {
- // Validation of the URL should be done at hadoop-aws level
- conf.set(ExternalDataConstants.AwsS3.HADOOP_SERVICE_END_POINT, serviceEndpoint);
- } else {
- //Region is ignored and buckets could be found by the central endpoint
- conf.set(ExternalDataConstants.AwsS3.HADOOP_SERVICE_END_POINT, Constants.CENTRAL_ENDPOINT);
- }
- }
-
- /**
- * Validate external dataset properties
- *
- * @param configuration properties
- * @throws CompilationException Compilation exception
- */
- public static void validateProperties(Map<String, String> configuration, SourceLocation srcLoc,
- IWarningCollector collector) throws CompilationException {
-
- // check if the format property is present
- if (configuration.get(ExternalDataConstants.KEY_FORMAT) == null) {
- throw new CompilationException(ErrorCode.PARAMETERS_REQUIRED, srcLoc, ExternalDataConstants.KEY_FORMAT);
- }
-
- // Both parameters should be passed, or neither should be passed (for anonymous/no auth)
- String accessKeyId = configuration.get(ACCESS_KEY_ID_FIELD_NAME);
- String secretAccessKey = configuration.get(SECRET_ACCESS_KEY_FIELD_NAME);
- if (accessKeyId == null || secretAccessKey == null) {
- // If one is passed, the other is required
- if (accessKeyId != null) {
- throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, SECRET_ACCESS_KEY_FIELD_NAME,
- ACCESS_KEY_ID_FIELD_NAME);
- } else if (secretAccessKey != null) {
- throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCESS_KEY_ID_FIELD_NAME,
- SECRET_ACCESS_KEY_FIELD_NAME);
- }
- }
-
- validateIncludeExclude(configuration);
-
- // Check if the bucket is present
- S3Client s3Client = buildAwsS3Client(configuration);
- S3Response response;
- boolean useOldApi = false;
- String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
- String prefix = getPrefix(configuration);
-
- try {
- response = isBucketEmpty(s3Client, container, prefix, false);
- } catch (S3Exception ex) {
- // Method not implemented, try falling back to old API
- try {
- // For error code, see https://docs.aws.amazon.com/AmazonS3/latest/API/ErrorResponses.html
- if (ex.awsErrorDetails().errorCode().equals(ERROR_METHOD_NOT_IMPLEMENTED)) {
- useOldApi = true;
- response = isBucketEmpty(s3Client, container, prefix, true);
- } else {
- throw ex;
- }
- } catch (SdkException ex2) {
- throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
- }
- } catch (SdkException ex) {
- throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
- } finally {
- if (s3Client != null) {
- CleanupUtils.close(s3Client, null);
- }
- }
-
- boolean isEmpty = useOldApi ? ((ListObjectsResponse) response).contents().isEmpty()
- : ((ListObjectsV2Response) response).contents().isEmpty();
- if (isEmpty && collector.shouldWarn()) {
- Warning warning = Warning.of(srcLoc, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
- collector.warn(warning);
- }
-
- // Returns 200 only in case the bucket exists, otherwise, throws an exception. However, to
- // ensure coverage, check if the result is successful as well and not only catch exceptions
- if (!response.sdkHttpResponse().isSuccessful()) {
- throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_CONTAINER_NOT_FOUND, container);
- }
- }
-
- /**
- * Checks for a single object in the specified bucket to determine if the bucket is empty or not.
- *
- * @param s3Client s3 client
- * @param container the container name
- * @param prefix Prefix to be used
- * @param useOldApi flag whether to use the old API or not
- * @return returns the S3 response
- */
- private static S3Response isBucketEmpty(S3Client s3Client, String container, String prefix, boolean useOldApi) {
- S3Response response;
- if (useOldApi) {
- ListObjectsRequest.Builder listObjectsBuilder = ListObjectsRequest.builder();
- listObjectsBuilder.prefix(prefix);
- response = s3Client.listObjects(listObjectsBuilder.bucket(container).maxKeys(1).build());
- } else {
- ListObjectsV2Request.Builder listObjectsBuilder = ListObjectsV2Request.builder();
- listObjectsBuilder.prefix(prefix);
- response = s3Client.listObjectsV2(listObjectsBuilder.bucket(container).maxKeys(1).build());
- }
- return response;
- }
-
- /**
- * Returns the lists of S3 objects.
- *
- * @param configuration properties
- * @param includeExcludeMatcher include/exclude matchers to apply
- */
- public static List<S3Object> listS3Objects(Map<String, String> configuration,
- IncludeExcludeMatcher includeExcludeMatcher, IWarningCollector warningCollector)
- throws CompilationException {
- // Prepare to retrieve the objects
- List<S3Object> filesOnly;
- String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
- S3Client s3Client = buildAwsS3Client(configuration);
- String prefix = getPrefix(configuration);
-
- try {
- filesOnly = listS3Objects(s3Client, container, prefix, includeExcludeMatcher);
- } catch (S3Exception ex) {
- // New API is not implemented, try falling back to old API
- try {
- // For error code, see https://docs.aws.amazon.com/AmazonS3/latest/API/ErrorResponses.html
- if (ex.awsErrorDetails().errorCode()
- .equals(ExternalDataConstants.AwsS3.ERROR_METHOD_NOT_IMPLEMENTED)) {
- filesOnly = oldApiListS3Objects(s3Client, container, prefix, includeExcludeMatcher);
- } else {
- throw ex;
- }
- } catch (SdkException ex2) {
- throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
- }
- } catch (SdkException ex) {
- throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
- } finally {
- if (s3Client != null) {
- CleanupUtils.close(s3Client, null);
- }
- }
-
- // Warn if no files are returned
- if (filesOnly.isEmpty() && warningCollector.shouldWarn()) {
- Warning warning = Warning.of(null, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
- warningCollector.warn(warning);
- }
-
- return filesOnly;
- }
-
- /**
- * Uses the latest API to retrieve the objects from the storage.
- *
- * @param s3Client S3 client
- * @param container container name
- * @param prefix definition prefix
- * @param includeExcludeMatcher include/exclude matchers to apply
- */
- private static List<S3Object> listS3Objects(S3Client s3Client, String container, String prefix,
- IncludeExcludeMatcher includeExcludeMatcher) {
- String newMarker = null;
- List<S3Object> filesOnly = new ArrayList<>();
-
- ListObjectsV2Response listObjectsResponse;
- ListObjectsV2Request.Builder listObjectsBuilder = ListObjectsV2Request.builder().bucket(container);
- listObjectsBuilder.prefix(prefix);
-
- while (true) {
- // List the objects from the start, or from the last marker in case of truncated result
- if (newMarker == null) {
- listObjectsResponse = s3Client.listObjectsV2(listObjectsBuilder.build());
- } else {
- listObjectsResponse =
- s3Client.listObjectsV2(listObjectsBuilder.continuationToken(newMarker).build());
- }
-
- // Collect the paths to files only
- collectAndFilterFiles(listObjectsResponse.contents(), includeExcludeMatcher.getPredicate(),
- includeExcludeMatcher.getMatchersList(), filesOnly);
-
- // Mark the flag as done if done, otherwise, get the marker of the previous response for the next request
- if (!listObjectsResponse.isTruncated()) {
- break;
- } else {
- newMarker = listObjectsResponse.nextContinuationToken();
- }
- }
-
- return filesOnly;
- }
-
- /**
- * Uses the old API (in case the new API is not implemented) to retrieve the objects from the storage
- *
- * @param s3Client S3 client
- * @param container container name
- * @param prefix definition prefix
- * @param includeExcludeMatcher include/exclude matchers to apply
- */
- private static List<S3Object> oldApiListS3Objects(S3Client s3Client, String container, String prefix,
- IncludeExcludeMatcher includeExcludeMatcher) {
- String newMarker = null;
- List<S3Object> filesOnly = new ArrayList<>();
-
- ListObjectsResponse listObjectsResponse;
- ListObjectsRequest.Builder listObjectsBuilder = ListObjectsRequest.builder().bucket(container);
- listObjectsBuilder.prefix(prefix);
-
- while (true) {
- // List the objects from the start, or from the last marker in case of truncated result
- if (newMarker == null) {
- listObjectsResponse = s3Client.listObjects(listObjectsBuilder.build());
- } else {
- listObjectsResponse = s3Client.listObjects(listObjectsBuilder.marker(newMarker).build());
- }
-
- // Collect the paths to files only
- collectAndFilterFiles(listObjectsResponse.contents(), includeExcludeMatcher.getPredicate(),
- includeExcludeMatcher.getMatchersList(), filesOnly);
-
- // Mark the flag as done if done, otherwise, get the marker of the previous response for the next request
- if (!listObjectsResponse.isTruncated()) {
- break;
- } else {
- newMarker = listObjectsResponse.nextMarker();
- }
- }
-
- return filesOnly;
- }
-
- /**
- * AWS S3 returns all the objects as paths, not differentiating between folder and files. The path is considered
- * a file if it does not end up with a "/" which is the separator in a folder structure.
- *
- * @param s3Objects List of returned objects
- */
- private static void collectAndFilterFiles(List<S3Object> s3Objects,
- BiPredicate<List<Matcher>, String> predicate, List<Matcher> matchers, List<S3Object> filesOnly) {
- for (S3Object object : s3Objects) {
- // skip folders
- if (object.key().endsWith("/")) {
- continue;
- }
-
- // No filter, add file
- if (predicate.test(matchers, object.key())) {
- filesOnly.add(object);
- }
- }
- }
- }
-
- /*
- * Note: Azure Blob and Azure Datalake use identical authentication, so they are using the same properties.
- * If they end up diverging, then properties for AzureBlob and AzureDataLake need to be created.
- */
- public static class Azure {
- private Azure() {
- throw new AssertionError("do not instantiate");
- }
-
- /**
- * Builds the Azure storage account using the provided configuration
- *
- * @param configuration properties
- * @return client
- */
- public static BlobServiceClient buildAzureBlobClient(IApplicationContext appCtx,
- Map<String, String> configuration) throws CompilationException {
- String managedIdentityId = configuration.get(MANAGED_IDENTITY_ID_FIELD_NAME);
- String accountName = configuration.get(ACCOUNT_NAME_FIELD_NAME);
- String accountKey = configuration.get(ACCOUNT_KEY_FIELD_NAME);
- String sharedAccessSignature = configuration.get(SHARED_ACCESS_SIGNATURE_FIELD_NAME);
- String tenantId = configuration.get(TENANT_ID_FIELD_NAME);
- String clientId = configuration.get(CLIENT_ID_FIELD_NAME);
- String clientSecret = configuration.get(CLIENT_SECRET_FIELD_NAME);
- String clientCertificate = configuration.get(CLIENT_CERTIFICATE_FIELD_NAME);
- String clientCertificatePassword = configuration.get(CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME);
- String endpoint = configuration.get(ENDPOINT_FIELD_NAME);
-
- // Client builder
- BlobServiceClientBuilder builder = new BlobServiceClientBuilder();
- int timeout = appCtx.getExternalProperties().getAzureRequestTimeout();
- RequestRetryOptions requestRetryOptions = new RequestRetryOptions(null, null, timeout, null, null, null);
- builder.retryOptions(requestRetryOptions);
-
- // Endpoint is required
- if (endpoint == null) {
- throw new CompilationException(PARAMETERS_REQUIRED, ENDPOINT_FIELD_NAME);
- }
- builder.endpoint(endpoint);
-
- // Shared Key
- if (accountName != null || accountKey != null) {
- if (accountName == null) {
- throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCOUNT_NAME_FIELD_NAME,
- ACCOUNT_KEY_FIELD_NAME);
- }
-
- if (accountKey == null) {
- throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCOUNT_KEY_FIELD_NAME,
- ACCOUNT_NAME_FIELD_NAME);
- }
-
- Optional<String> provided = getFirstNotNull(configuration, SHARED_ACCESS_SIGNATURE_FIELD_NAME,
- MANAGED_IDENTITY_ID_FIELD_NAME, CLIENT_ID_FIELD_NAME, CLIENT_SECRET_FIELD_NAME,
- CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
- if (provided.isPresent()) {
- throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
- ACCOUNT_KEY_FIELD_NAME);
- }
- StorageSharedKeyCredential credential = new StorageSharedKeyCredential(accountName, accountKey);
- builder.credential(credential);
- }
-
- // Shared access signature
- if (sharedAccessSignature != null) {
- Optional<String> provided = getFirstNotNull(configuration, MANAGED_IDENTITY_ID_FIELD_NAME,
- CLIENT_ID_FIELD_NAME, CLIENT_SECRET_FIELD_NAME, CLIENT_CERTIFICATE_FIELD_NAME,
- CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
- if (provided.isPresent()) {
- throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
- SHARED_ACCESS_SIGNATURE_FIELD_NAME);
- }
- AzureSasCredential credential = new AzureSasCredential(sharedAccessSignature);
- builder.credential(credential);
- }
-
- // Managed Identity auth
- if (managedIdentityId != null) {
- Optional<String> provided = getFirstNotNull(configuration, CLIENT_ID_FIELD_NAME,
- CLIENT_SECRET_FIELD_NAME, CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME,
- TENANT_ID_FIELD_NAME);
- if (provided.isPresent()) {
- throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
- MANAGED_IDENTITY_ID_FIELD_NAME);
- }
- builder.credential(new ManagedIdentityCredentialBuilder().clientId(managedIdentityId).build());
- }
-
- // Client secret & certificate auth
- if (clientId != null) {
- // Both (or neither) client secret and client secret were provided, only one is allowed
- if ((clientSecret == null) == (clientCertificate == null)) {
- if (clientSecret != null) {
- throw new CompilationException(PARAMETERS_NOT_ALLOWED_AT_SAME_TIME, CLIENT_SECRET_FIELD_NAME,
- CLIENT_CERTIFICATE_FIELD_NAME);
- } else {
- throw new CompilationException(REQUIRED_PARAM_OR_PARAM_IF_PARAM_IS_PRESENT,
- CLIENT_SECRET_FIELD_NAME, CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_ID_FIELD_NAME);
- }
- }
-
- // Tenant ID is required
- if (tenantId == null) {
- throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, TENANT_ID_FIELD_NAME,
- CLIENT_ID_FIELD_NAME);
- }
-
- // Client certificate password is not allowed if client secret is used
- if (clientCertificatePassword != null && clientSecret != null) {
- throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT,
- CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, CLIENT_SECRET_FIELD_NAME);
- }
-
- // Use AD authentication
- if (clientSecret != null) {
- ClientSecretCredentialBuilder secret = new ClientSecretCredentialBuilder();
- secret.clientId(clientId);
- secret.tenantId(tenantId);
- secret.clientSecret(clientSecret);
- builder.credential(secret.build());
- } else {
- // Certificate
- ClientCertificateCredentialBuilder certificate = new ClientCertificateCredentialBuilder();
- certificate.clientId(clientId);
- certificate.tenantId(tenantId);
- try {
- InputStream certificateContent = new ByteArrayInputStream(clientCertificate.getBytes(UTF_8));
- if (clientCertificatePassword == null) {
- Method pemCertificate = ClientCertificateCredentialBuilder.class
- .getDeclaredMethod("pemCertificate", InputStream.class);
- pemCertificate.setAccessible(true);
- pemCertificate.invoke(certificate, certificateContent);
- } else {
- Method pemCertificate = ClientCertificateCredentialBuilder.class
- .getDeclaredMethod("pfxCertificate", InputStream.class, String.class);
- pemCertificate.setAccessible(true);
- pemCertificate.invoke(certificate, certificateContent, clientCertificatePassword);
- }
- } catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException ex) {
- throw new CompilationException(EXTERNAL_SOURCE_ERROR, ex.getMessage());
- }
- builder.credential(certificate.build());
- }
- }
-
- // If client id is not present, ensure client secret, certificate, tenant id and client certificate
- // password are not present
- if (clientId == null) {
- Optional<String> provided = getFirstNotNull(configuration, CLIENT_SECRET_FIELD_NAME,
- CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
- if (provided.isPresent()) {
- throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
- SHARED_ACCESS_SIGNATURE_FIELD_NAME);
- }
- }
-
- try {
- return builder.buildClient();
- } catch (Exception ex) {
- throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
- }
- }
-
- /**
- * Builds the Azure data lake storage account using the provided configuration
- *
- * @param configuration properties
- * @return client
- */
- public static DataLakeServiceClient buildAzureDatalakeClient(IApplicationContext appCtx,
- Map<String, String> configuration) throws CompilationException {
- String managedIdentityId = configuration.get(MANAGED_IDENTITY_ID_FIELD_NAME);
- String accountName = configuration.get(ACCOUNT_NAME_FIELD_NAME);
- String accountKey = configuration.get(ACCOUNT_KEY_FIELD_NAME);
- String sharedAccessSignature = configuration.get(SHARED_ACCESS_SIGNATURE_FIELD_NAME);
- String tenantId = configuration.get(TENANT_ID_FIELD_NAME);
- String clientId = configuration.get(CLIENT_ID_FIELD_NAME);
- String clientSecret = configuration.get(CLIENT_SECRET_FIELD_NAME);
- String clientCertificate = configuration.get(CLIENT_CERTIFICATE_FIELD_NAME);
- String clientCertificatePassword = configuration.get(CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME);
- String endpoint = configuration.get(ENDPOINT_FIELD_NAME);
-
- // Client builder
- DataLakeServiceClientBuilder builder = new DataLakeServiceClientBuilder();
- int timeout = appCtx.getExternalProperties().getAzureRequestTimeout();
- RequestRetryOptions requestRetryOptions = new RequestRetryOptions(null, null, timeout, null, null, null);
- builder.retryOptions(requestRetryOptions);
-
- // Endpoint is required
- if (endpoint == null) {
- throw new CompilationException(PARAMETERS_REQUIRED, ENDPOINT_FIELD_NAME);
- }
- builder.endpoint(endpoint);
-
- // Shared Key
- if (accountName != null || accountKey != null) {
- if (accountName == null) {
- throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCOUNT_NAME_FIELD_NAME,
- ACCOUNT_KEY_FIELD_NAME);
- }
-
- if (accountKey == null) {
- throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCOUNT_KEY_FIELD_NAME,
- ACCOUNT_NAME_FIELD_NAME);
- }
-
- Optional<String> provided = getFirstNotNull(configuration, SHARED_ACCESS_SIGNATURE_FIELD_NAME,
- MANAGED_IDENTITY_ID_FIELD_NAME, CLIENT_ID_FIELD_NAME, CLIENT_SECRET_FIELD_NAME,
- CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
- if (provided.isPresent()) {
- throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
- ACCOUNT_KEY_FIELD_NAME);
- }
- StorageSharedKeyCredential credential = new StorageSharedKeyCredential(accountName, accountKey);
- builder.credential(credential);
- }
-
- // Shared access signature
- if (sharedAccessSignature != null) {
- Optional<String> provided = getFirstNotNull(configuration, MANAGED_IDENTITY_ID_FIELD_NAME,
- CLIENT_ID_FIELD_NAME, CLIENT_SECRET_FIELD_NAME, CLIENT_CERTIFICATE_FIELD_NAME,
- CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
- if (provided.isPresent()) {
- throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
- SHARED_ACCESS_SIGNATURE_FIELD_NAME);
- }
- AzureSasCredential credential = new AzureSasCredential(sharedAccessSignature);
- builder.credential(credential);
- }
-
- // Managed Identity auth
- if (managedIdentityId != null) {
- Optional<String> provided = getFirstNotNull(configuration, CLIENT_ID_FIELD_NAME,
- CLIENT_SECRET_FIELD_NAME, CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME,
- TENANT_ID_FIELD_NAME);
- if (provided.isPresent()) {
- throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
- MANAGED_IDENTITY_ID_FIELD_NAME);
- }
- builder.credential(new ManagedIdentityCredentialBuilder().clientId(managedIdentityId).build());
- }
-
- // Client secret & certificate auth
- if (clientId != null) {
- // Both (or neither) client secret and client secret were provided, only one is allowed
- if ((clientSecret == null) == (clientCertificate == null)) {
- if (clientSecret != null) {
- throw new CompilationException(PARAMETERS_NOT_ALLOWED_AT_SAME_TIME, CLIENT_SECRET_FIELD_NAME,
- CLIENT_CERTIFICATE_FIELD_NAME);
- } else {
- throw new CompilationException(REQUIRED_PARAM_OR_PARAM_IF_PARAM_IS_PRESENT,
- CLIENT_SECRET_FIELD_NAME, CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_ID_FIELD_NAME);
- }
- }
-
- // Tenant ID is required
- if (tenantId == null) {
- throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, TENANT_ID_FIELD_NAME,
- CLIENT_ID_FIELD_NAME);
- }
-
- // Client certificate password is not allowed if client secret is used
- if (clientCertificatePassword != null && clientSecret != null) {
- throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT,
- CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, CLIENT_SECRET_FIELD_NAME);
- }
-
- // Use AD authentication
- if (clientSecret != null) {
- ClientSecretCredentialBuilder secret = new ClientSecretCredentialBuilder();
- secret.clientId(clientId);
- secret.tenantId(tenantId);
- secret.clientSecret(clientSecret);
- builder.credential(secret.build());
- } else {
- // Certificate
- ClientCertificateCredentialBuilder certificate = new ClientCertificateCredentialBuilder();
- certificate.clientId(clientId);
- certificate.tenantId(tenantId);
- try {
- InputStream certificateContent = new ByteArrayInputStream(clientCertificate.getBytes(UTF_8));
- if (clientCertificatePassword == null) {
- Method pemCertificate = ClientCertificateCredentialBuilder.class
- .getDeclaredMethod("pemCertificate", InputStream.class);
- pemCertificate.setAccessible(true);
- pemCertificate.invoke(certificate, certificateContent);
- } else {
- Method pemCertificate = ClientCertificateCredentialBuilder.class
- .getDeclaredMethod("pfxCertificate", InputStream.class, String.class);
- pemCertificate.setAccessible(true);
- pemCertificate.invoke(certificate, certificateContent, clientCertificatePassword);
- }
- } catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException ex) {
- throw new CompilationException(EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
- }
- builder.credential(certificate.build());
- }
- }
-
- // If client id is not present, ensure client secret, certificate, tenant id and client certificate
- // password are not present
- if (clientId == null) {
- Optional<String> provided = getFirstNotNull(configuration, CLIENT_SECRET_FIELD_NAME,
- CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
- if (provided.isPresent()) {
- throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
- SHARED_ACCESS_SIGNATURE_FIELD_NAME);
- }
- }
-
- try {
- return builder.buildClient();
- } catch (Exception ex) {
- throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
- }
- }
-
- public static List<BlobItem> listBlobItems(BlobServiceClient blobServiceClient,
- Map<String, String> configuration, IncludeExcludeMatcher includeExcludeMatcher,
- IWarningCollector warningCollector) throws CompilationException {
- String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
-
- List<BlobItem> filesOnly = new ArrayList<>();
-
- // Ensure the validity of include/exclude
- ExternalDataUtils.validateIncludeExclude(configuration);
-
- BlobContainerClient blobContainer;
- try {
- blobContainer = blobServiceClient.getBlobContainerClient(container);
-
- // Get all objects in a container and extract the paths to files
- ListBlobsOptions listBlobsOptions = new ListBlobsOptions();
- listBlobsOptions.setPrefix(ExternalDataUtils.getPrefix(configuration));
- Iterable<BlobItem> blobItems = blobContainer.listBlobs(listBlobsOptions, null);
-
- // Collect the paths to files only
- collectAndFilterBlobFiles(blobItems, includeExcludeMatcher.getPredicate(),
- includeExcludeMatcher.getMatchersList(), filesOnly);
-
- // Warn if no files are returned
- if (filesOnly.isEmpty() && warningCollector.shouldWarn()) {
- Warning warning = Warning.of(null, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
- warningCollector.warn(warning);
- }
- } catch (Exception ex) {
- throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
- }
-
- return filesOnly;
- }
-
- /**
- * Collects and filters the files only, and excludes any folders
- *
- * @param items storage items
- * @param predicate predicate to test with for file filtration
- * @param matchers include/exclude matchers to test against
- * @param filesOnly List containing the files only (excluding folders)
- */
- private static void collectAndFilterBlobFiles(Iterable<BlobItem> items,
- BiPredicate<List<Matcher>, String> predicate, List<Matcher> matchers, List<BlobItem> filesOnly) {
- for (BlobItem item : items) {
- String uri = item.getName();
-
- // skip folders
- if (uri.endsWith("/")) {
- continue;
- }
-
- // No filter, add file
- if (predicate.test(matchers, uri)) {
- filesOnly.add(item);
- }
- }
- }
-
- public static List<PathItem> listDatalakePathItems(DataLakeServiceClient client,
- Map<String, String> configuration, IncludeExcludeMatcher includeExcludeMatcher,
- IWarningCollector warningCollector) throws CompilationException {
- String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
-
- List<PathItem> filesOnly = new ArrayList<>();
-
- // Ensure the validity of include/exclude
- ExternalDataUtils.validateIncludeExclude(configuration);
-
- DataLakeFileSystemClient fileSystemClient;
- try {
- fileSystemClient = client.getFileSystemClient(container);
-
- // Get all objects in a container and extract the paths to files
- ListPathsOptions listOptions = new ListPathsOptions();
- boolean recursive = Boolean.parseBoolean(configuration.get(RECURSIVE_FIELD_NAME));
- listOptions.setRecursive(recursive);
- listOptions.setPath(ExternalDataUtils.getPrefix(configuration, false));
- PagedIterable<PathItem> pathItems = fileSystemClient.listPaths(listOptions, null);
-
- // Collect the paths to files only
- collectAndFilterDatalakeFiles(pathItems, includeExcludeMatcher.getPredicate(),
- includeExcludeMatcher.getMatchersList(), filesOnly);
-
- // Warn if no files are returned
- if (filesOnly.isEmpty() && warningCollector.shouldWarn()) {
- Warning warning = Warning.of(null, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
- warningCollector.warn(warning);
- }
- } catch (Exception ex) {
- throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
- }
-
- return filesOnly;
- }
-
- /**
- * Collects and filters the files only, and excludes any folders
- *
- * @param items storage items
- * @param predicate predicate to test with for file filtration
- * @param matchers include/exclude matchers to test against
- * @param filesOnly List containing the files only (excluding folders)
- */
- private static void collectAndFilterDatalakeFiles(Iterable<PathItem> items,
- BiPredicate<List<Matcher>, String> predicate, List<Matcher> matchers, List<PathItem> filesOnly) {
- for (PathItem item : items) {
- String uri = item.getName();
-
- // skip folders
- if (uri.endsWith("/")) {
- continue;
- }
-
- // No filter, add file
- if (predicate.test(matchers, uri)) {
- filesOnly.add(item);
- }
- }
- }
-
- /**
- * Validate external dataset properties
- *
- * @param configuration properties
- * @throws CompilationException Compilation exception
- */
- public static void validateAzureBlobProperties(Map<String, String> configuration, SourceLocation srcLoc,
- IWarningCollector collector, IApplicationContext appCtx) throws CompilationException {
-
- // check if the format property is present
- if (configuration.get(ExternalDataConstants.KEY_FORMAT) == null) {
- throw new CompilationException(ErrorCode.PARAMETERS_REQUIRED, srcLoc, ExternalDataConstants.KEY_FORMAT);
- }
-
- validateIncludeExclude(configuration);
-
- // Check if the bucket is present
- BlobServiceClient blobServiceClient;
- try {
- String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
- blobServiceClient = buildAzureBlobClient(appCtx, configuration);
- BlobContainerClient blobContainer = blobServiceClient.getBlobContainerClient(container);
-
- // Get all objects in a container and extract the paths to files
- ListBlobsOptions listBlobsOptions = new ListBlobsOptions();
- listBlobsOptions.setPrefix(getPrefix(configuration));
- Iterable<BlobItem> blobItems = blobContainer.listBlobs(listBlobsOptions, null);
-
- if (!blobItems.iterator().hasNext() && collector.shouldWarn()) {
- Warning warning = Warning.of(srcLoc, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
- collector.warn(warning);
- }
- } catch (CompilationException ex) {
- throw ex;
- } catch (Exception ex) {
- throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
- }
- }
-
- /**
- * Validate external dataset properties
- *
- * @param configuration properties
- * @throws CompilationException Compilation exception
- */
- public static void validateAzureDataLakeProperties(Map<String, String> configuration, SourceLocation srcLoc,
- IWarningCollector collector, IApplicationContext appCtx) throws CompilationException {
-
- // check if the format property is present
- if (configuration.get(ExternalDataConstants.KEY_FORMAT) == null) {
- throw new CompilationException(ErrorCode.PARAMETERS_REQUIRED, srcLoc, ExternalDataConstants.KEY_FORMAT);
- }
-
- validateIncludeExclude(configuration);
-
- // Check if the bucket is present
- DataLakeServiceClient dataLakeServiceClient;
- try {
- String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
- dataLakeServiceClient = buildAzureDatalakeClient(appCtx, configuration);
- DataLakeFileSystemClient fileSystemClient = dataLakeServiceClient.getFileSystemClient(container);
-
- // Get all objects in a container and extract the paths to files
- ListPathsOptions listPathsOptions = new ListPathsOptions();
- listPathsOptions.setPath(getPrefix(configuration));
- Iterable<PathItem> blobItems = fileSystemClient.listPaths(listPathsOptions, null);
-
- if (!blobItems.iterator().hasNext() && collector.shouldWarn()) {
- Warning warning = Warning.of(srcLoc, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
- collector.warn(warning);
- }
- } catch (CompilationException ex) {
- throw ex;
- } catch (Exception ex) {
- throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
- }
- }
-
- /**
- * Builds the Azure Blob storage client using the provided configuration
- *
- * @param configuration properties
- * @see <a href="https://docs.microsoft.com/en-us/azure/databricks/data/data-sources/azure/azure-storage">Azure
- * Blob storage</a>
- */
- public static void configureAzureHdfsJobConf(JobConf conf, Map<String, String> configuration, String endPoint) {
- String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
- String accountKey = configuration.get(ACCOUNT_KEY_FIELD_NAME);
- String sharedAccessSignature = configuration.get(SHARED_ACCESS_SIGNATURE_FIELD_NAME);
-
- //Disable caching S3 FileSystem
- HDFSUtils.disableHadoopFileSystemCache(conf, HADOOP_AZURE_BLOB_PROTOCOL);
-
- //Key for Hadoop configuration
- StringBuilder hadoopKey = new StringBuilder();
- //Value for Hadoop configuration
- String hadoopValue;
- if (accountKey != null || sharedAccessSignature != null) {
- if (accountKey != null) {
- hadoopKey.append(HADOOP_AZURE_FS_ACCOUNT_KEY).append('.');
- //Set only the AccountKey
- hadoopValue = accountKey;
- } else {
- //Use SAS for Hadoop FS as connectionString is provided
- hadoopKey.append(HADOOP_AZURE_FS_SAS).append('.');
- //Setting the container is required for SAS
- hadoopKey.append(container).append('.');
- //Set the connection string for SAS
- hadoopValue = sharedAccessSignature;
- }
- //Set the endPoint, which includes the AccountName
- hadoopKey.append(endPoint);
- //Tells Hadoop we are reading from Blob Storage
- conf.set(hadoopKey.toString(), hadoopValue);
- }
- }
- }
-
- public static class GCS {
- private GCS() {
- throw new AssertionError("do not instantiate");
-
- }
-
- //TODO(htowaileb): Add validation step similar to other externals, which also checks if empty bucket
- //upon creating the external dataset
-
- /**
- * Builds the client using the provided configuration
- *
- * @param configuration properties
- * @return clientasterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
- * @throws CompilationException CompilationException
- */
- public static Storage buildClient(Map<String, String> configuration) throws CompilationException {
- String jsonCredentials = configuration.get(JSON_CREDENTIALS_FIELD_NAME);
-
- StorageOptions.Builder builder = StorageOptions.newBuilder();
-
- // Use credentials if available
- if (jsonCredentials != null) {
- try (InputStream credentialsStream = new ByteArrayInputStream(jsonCredentials.getBytes())) {
- builder.setCredentials(ServiceAccountCredentials.fromStream(credentialsStream));
- } catch (IOException ex) {
- throw new CompilationException(EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
- }
- }
-
- return builder.build().getService();
- }
-
- /**
- * Validate external dataset properties
- *
- * @param configuration properties
- * @throws CompilationException Compilation exception
- */
- public static void validateProperties(Map<String, String> configuration, SourceLocation srcLoc,
- IWarningCollector collector) throws CompilationException {
-
- // check if the format property is present
- if (configuration.get(ExternalDataConstants.KEY_FORMAT) == null) {
- throw new CompilationException(ErrorCode.PARAMETERS_REQUIRED, srcLoc, ExternalDataConstants.KEY_FORMAT);
- }
-
- // parquet is not supported for google cloud storage
- if (isParquetFormat(configuration)) {
- throw new CompilationException(INVALID_REQ_PARAM_VAL, srcLoc, KEY_FORMAT,
- configuration.get(KEY_FORMAT));
- }
-
- validateIncludeExclude(configuration);
- String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
-
- try {
- BlobListOption limitOption = BlobListOption.pageSize(1);
- BlobListOption prefixOption = BlobListOption.prefix(getPrefix(configuration));
- Storage storage = buildClient(configuration);
- Page<Blob> items = storage.list(container, limitOption, prefixOption);
-
- if (!items.iterateAll().iterator().hasNext() && collector.shouldWarn()) {
- Warning warning = Warning.of(srcLoc, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
- collector.warn(warning);
- }
- } catch (CompilationException ex) {
- throw ex;
- } catch (Exception ex) {
- throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
- }
- }
- }
-
public static int roundUpToNearestFrameSize(int size, int framesize) {
return ((size / framesize) + 1) * framesize;
}
@@ -1911,7 +827,7 @@
return maxArgSz;
}
- private static Optional<String> getFirstNotNull(Map<String, String> configuration, String... parameters) {
+ public static Optional<String> getFirstNotNull(Map<String, String> configuration, String... parameters) {
return Arrays.stream(parameters).filter(field -> configuration.get(field) != null).findFirst();
}
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/aws/s3/S3Constants.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/aws/s3/S3Constants.java
new file mode 100644
index 0000000..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..6bf2266
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/google/gcs/GCSConstants.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.external.util.google.gcs;
+
+public class GCSConstants {
+ private GCSConstants() {
+ throw new AssertionError("do not instantiate");
+ }
+
+ 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..93dc272
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/google/gcs/GCSUtils.java
@@ -0,0 +1,217 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF 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.external.util.ExternalDataUtils.getPrefix;
+import static org.apache.asterix.external.util.ExternalDataUtils.validateIncludeExclude;
+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.ServiceAccountCredentials;
+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 jsonCredentials = configuration.get(JSON_CREDENTIALS_FIELD_NAME);
+ String endpoint = configuration.get(ENDPOINT_FIELD_NAME);
+
+ StorageOptions.Builder builder = StorageOptions.newBuilder();
+
+ // Use credentials if available
+ if (jsonCredentials != null) {
+ try (InputStream credentialsStream = new ByteArrayInputStream(jsonCredentials.getBytes())) {
+ builder.setCredentials(ServiceAccountCredentials.fromStream(credentialsStream));
+ } catch (IOException ex) {
+ throw new CompilationException(EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+ }
+ }
+
+ 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-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-external-data/src/main/java/org/checkerframework/checker/nullness/compatqual/NullableDecl.java
similarity index 61%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-external-data/src/main/java/org/checkerframework/checker/nullness/compatqual/NullableDecl.java
index 6095b26..a4cb401 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-external-data/src/main/java/org/checkerframework/checker/nullness/compatqual/NullableDecl.java
@@ -16,9 +16,18 @@
* specific language governing permissions and limitations
* under the License.
*/
+package org.checkerframework.checker.nullness.compatqual;
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
-select value v from range(1,2) v where v > ?;
+/*
+ * 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-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..b1ad073 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,7 @@
CREATE_SYNONYM,
SYNONYM_DROP,
VIEW_DROP,
+ ANALYZE,
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/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/visitor/FormatPrintVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java
index 0ddbeb4..c6db0a3 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,7 @@
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.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 +930,14 @@
}
@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(CompactStatement del, Integer step) throws CompilationException {
return null;
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/AbstractQueryExpressionVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/AbstractQueryExpressionVisitor.java
index 91fe664..bc8a8d3 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,7 @@
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.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 +262,11 @@
}
@Override
+ public R visit(AnalyzeStatement 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..a5c71ef 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/ILangVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/ILangVisitor.java
@@ -42,6 +42,7 @@
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.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 +208,8 @@
R visit(SynonymDropStatement del, T arg) throws CompilationException;
+ R visit(AnalyzeStatement as, T arg) throws CompilationException;
+
R visit(CompactStatement del, T arg) throws CompilationException;
R visit(ListSliceExpression expression, T arg) throws CompilationException;
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppFunctionBodyRewriter.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppFunctionBodyRewriter.java
index 557f17d..4352210 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppFunctionBodyRewriter.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppFunctionBodyRewriter.java
@@ -61,7 +61,7 @@
/**
* This rewriter is used to rewrite body expression of user defined functions and views
*/
-class SqlppFunctionBodyRewriter extends SqlppQueryRewriter {
+public class SqlppFunctionBodyRewriter extends SqlppQueryRewriter {
public SqlppFunctionBodyRewriter(IParserFactory parserFactory) {
super(parserFactory);
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
index 4e64944..9edd07d 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
+++ b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
@@ -76,6 +76,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 +125,7 @@
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.CompactStatement;
import org.apache.asterix.lang.common.statement.ConnectFeedStatement;
import org.apache.asterix.lang.common.statement.StartFeedStatement;
@@ -205,7 +207,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;
@@ -836,6 +838,7 @@
| stmt = UpsertStatement()
| stmt = ConnectionStatement()
| stmt = CompactStatement()
+ | stmt = AnalyzeStatement()
| stmt = Query()
| stmt = RefreshExternalDatasetStatement()
)
@@ -1378,7 +1381,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 +2631,25 @@
}
}
+Statement AnalyzeStatement() throws ParseException:
+{
+ Token startToken = null;
+ Pair<DataverseName,Identifier> nameComponents = null;
+ RecordConstructor withRecord = null;
+}
+{
+ <ANALYZE> { startToken = token; } Dataset() nameComponents = QualifiedName()
+ ( <WITH> withRecord = RecordConstructor() )?
+ {
+ try {
+ AnalyzeStatement stmt = new AnalyzeStatement(nameComponents.first, nameComponents.second.getValue(), withRecord);
+ return addSourceLocation(stmt, startToken);
+ } catch (CompilationException e) {
+ throw new SqlppParseException(getSourceLocation(startToken), e.getMessage());
+ }
+ }
+}
+
Statement CompactStatement() throws ParseException:
{
Token startToken = null;
@@ -5318,6 +5340,7 @@
{
<ADAPTER: "adapter">
| <ALL : "all">
+ | <ANALYZE: "analyze">
| <AND : "and">
| <ANY : "any">
| <APPLY : "apply">
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..5daaaa0 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
@@ -53,6 +53,7 @@
import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
import org.apache.hyracks.api.job.JobSpecification;
import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
+import org.apache.hyracks.storage.am.common.impls.DefaultTupleProjectorFactory;
public class DatasetDataSource extends DataSource {
@@ -78,13 +79,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 +98,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
@@ -135,7 +147,7 @@
return metadataProvider.buildBtreeRuntime(jobSpec, opSchema, typeEnv, context, true, false, null,
((DatasetDataSource) dataSource).getDataset(), primaryIndex.getIndexName(), null, null, true,
true, false, null, minFilterFieldIndexes, maxFilterFieldIndexes, tupleFilterFactory,
- outputLimit, false, false);
+ outputLimit, false, false, DefaultTupleProjectorFactory.INSTANCE);
default:
throw new AlgebricksException("Unknown datasource type");
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
index 4cc83ab..87959c4 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
@@ -93,7 +93,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 +117,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 +172,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 +186,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 +265,6 @@
this.isWriteTransaction = writeTransaction;
}
- public void setWriterFactory(IAWriterFactory writerFactory) {
- this.writerFactory = writerFactory;
- }
-
public void setMetadataTxnContext(MetadataTransactionContext mdTxnCtx) {
this.mdTxnCtx = mdTxnCtx;
}
@@ -274,10 +273,6 @@
return mdTxnCtx;
}
- public IAWriterFactory getWriterFactory() {
- return this.writerFactory;
- }
-
public FileSplit getOutputFile() {
return outputFile;
}
@@ -310,12 +305,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 +432,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 +547,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 +567,12 @@
case BTREE:
numSecondaryKeys = ((Index.ValueIndexDetails) theIndex.getIndexDetails()).getKeyFieldNames().size();
break;
+ case SAMPLE:
+ if (isIndexOnlyPlan) {
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, "");
+ }
+ numSecondaryKeys = 0;
+ break;
default:
throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE,
theIndex.getIndexType().toString());
@@ -601,12 +613,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 +696,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 +705,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 +721,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 +874,7 @@
*
* @param dataset
* @return Number of elements that will be used to create a bloom filter per
- * dataset per partition
+ * dataset per partition
* @throws AlgebricksException
*/
public long getCardinalityPerPartitionHint(Dataset dataset) throws AlgebricksException {
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/SampleDataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/SampleDataSource.java
new file mode 100644
index 0000000..418cb8e
--- /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 });
+ }
+}
\ No newline at end of file
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..21d2aaa 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
@@ -275,6 +275,7 @@
switch (indexType) {
case ARRAY:
case BTREE:
+ case SAMPLE:
return ResourceType.LSM_BTREE;
case RTREE:
return ResourceType.LSM_RTREE;
@@ -296,7 +297,8 @@
public enum IndexCategory {
VALUE,
TEXT,
- ARRAY;
+ ARRAY,
+ SAMPLE;
public static IndexCategory of(IndexType indexType) {
switch (indexType) {
@@ -310,6 +312,8 @@
return TEXT;
case ARRAY:
return ARRAY;
+ case SAMPLE:
+ return SAMPLE;
default:
throw new IllegalArgumentException(String.valueOf(indexType));
}
@@ -533,6 +537,75 @@
}
}
+ 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;
+
+ public SampleIndexDetails(List<List<String>> keyFieldNames, List<Integer> keyFieldSourceIndicators,
+ List<IAType> keyFieldTypes, int sampleCardinalityTarget, long sourceCardinality, int sourceAvgItemSize,
+ long sampleSeed) {
+ this.keyFieldNames = keyFieldNames;
+ this.keyFieldSourceIndicators = keyFieldSourceIndicators;
+ this.keyFieldTypes = keyFieldTypes;
+ this.sampleCardinalityTarget = sampleCardinalityTarget;
+ this.sourceCardinality = sourceCardinality;
+ this.sourceAvgItemSize = sourceAvgItemSize;
+ this.sampleSeed = sampleSeed;
+ }
+
+ @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;
+ }
+ }
+
@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..9c742ed 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
@@ -47,6 +47,7 @@
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.AMutableInt8;
import org.apache.asterix.om.base.ANull;
@@ -90,6 +91,10 @@
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";
protected final TxnId txnId;
protected final MetadataNode metadataNode;
@@ -240,6 +245,9 @@
searchElements.add(searchElement);
}
break;
+ case SAMPLE:
+ searchElements = Collections.emptyList();
+ break;
default:
throw new AsterixException(ErrorCode.METADATA_ERROR, indexType.toString());
}
@@ -452,6 +460,39 @@
}
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();
+
+ indexDetails = new Index.SampleIndexDetails(keyFieldNames, keyFieldSourceIndicator, keyFieldTypes,
+ sampleCardinalityTarget, sourceCardinality, sourceAvgItemSize, sampleSeed);
+ break;
default:
throw new AsterixException(ErrorCode.METADATA_ERROR, indexType.toString());
}
@@ -527,6 +568,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 +640,7 @@
writeSearchKeySourceIndicator(index);
writeExcludeUnknownKey(index);
writeCast(index);
+ writeSampleDetails(index);
}
private void writeComplexSearchKeys(Index.ArrayIndexDetails indexDetails) throws HyracksDataException {
@@ -767,6 +812,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 +903,38 @@
}
}
}
+
+ 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);
+ }
+ }
}
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/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..afd1582 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,13 @@
}
@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 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..0d3e015
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SampleOperationsHelper.java
@@ -0,0 +1,349 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF 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.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.LSMIndexBulkLoadOperatorDescriptor;
+import org.apache.asterix.runtime.operators.StreamStatsOperatorDescriptor;
+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;
+
+/**
+ * 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;
+ }
+
+ JobSpecification spec = RuntimeUtils.createJobSpecification(metadataProvider.getApplicationContext());
+ IIndexDataflowHelperFactory dataflowHelperFactory = new IndexDataflowHelperFactory(
+ metadataProvider.getStorageComponentProvider().getStorageManager(), 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
+ targetOp = new StreamStatsOperatorDescriptor(spec, recordDesc, DATASET_STATS_OPERATOR_NAME);
+ 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/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/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-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/operators/StreamStats.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/StreamStats.java
new file mode 100644
index 0000000..268009f
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/StreamStats.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.operators;
+
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
+
+/**
+ * Helper method to access stats produced by {@link org.apache.asterix.runtime.operators.StreamStatsOperatorDescriptor}
+ */
+public final class StreamStats {
+
+ private final long cardinality;
+
+ private final int avgTupleSize;
+
+ public StreamStats(IOperatorStats opStats) {
+ this.cardinality = opStats.getTupleCounter().get();
+ long totalTupleSize = opStats.getDiskIoCounter().get();
+ this.avgTupleSize = cardinality > 0 ? (int) (totalTupleSize / cardinality) : 0;
+ }
+
+ static void update(IOperatorStats opStats, long tupleCount, long tupleSize) {
+ opStats.getTupleCounter().update(tupleCount);
+ opStats.getDiskIoCounter().update(tupleSize);
+ }
+
+ public long getCardinality() {
+ return cardinality;
+ }
+
+ public int getAvgTupleSize() {
+ return avgTupleSize;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/StreamStatsOperatorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/StreamStatsOperatorDescriptor.java
new file mode 100644
index 0000000..353401a
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/StreamStatsOperatorDescriptor.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.runtime.operators;
+
+import java.nio.ByteBuffer;
+
+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.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;
+
+/**
+ * Computes total tuple count and total tuple length for all input tuples,
+ * and emits these values as operator stats.
+ */
+public final class StreamStatsOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+
+ private static final long serialVersionUID = 1L;
+
+ private final String operatorName;
+
+ public StreamStatsOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor rDesc,
+ String operatorName) {
+ super(spec, 1, 1);
+ outRecDescs[0] = rDesc;
+ this.operatorName = operatorName;
+ }
+
+ @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();
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ fta.reset(buffer);
+ computeStats();
+ FrameUtils.flushFrame(buffer, writer);
+ }
+
+ private void computeStats() {
+ int n = fta.getTupleCount();
+ totalTupleCount += n;
+ for (int i = 0; i < n; i++) {
+ totalTupleLength += fta.getTupleLength(i);
+ }
+ }
+
+ @Override
+ public void fail() throws HyracksDataException {
+ writer.fail();
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ IStatsCollector statsCollector = ctx.getStatsCollector();
+ if (statsCollector != null) {
+ IOperatorStats stats = statsCollector.getOrAddOperatorStats(operatorName);
+ StreamStats.update(stats, totalTupleCount, totalTupleLength);
+ }
+ 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/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-server/pom.xml b/asterixdb/asterix-server/pom.xml
index c5ba4cc..3963e03 100644
--- a/asterixdb/asterix-server/pom.xml
+++ b/asterixdb/asterix-server/pom.xml
@@ -353,12 +353,30 @@
</override>
<override>
<gavs>
+ <gav>io.opencensus:opencensus-proto:0.2.0</gav>
+ </gavs>
+ <url>https://raw.githubusercontent.com/census-instrumentation/opencensus-proto/v0.2.0/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>
</gavs>
<url>https://raw.githubusercontent.com/census-instrumentation/opencensus-java/v0.28.0/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:1.33.0</gav>
<url>https://raw.githubusercontent.com/googleapis/google-api-java-client/v1.33.0/LICENSE</url>
</override>
@@ -402,7 +420,20 @@
<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-context: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>
@@ -418,6 +449,68 @@
<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>
</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>
<license>
@@ -466,6 +559,18 @@
<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>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-proto/v0.2.0/LICENSE</aliasUrl>
+ <alisUrl>https://raw.githubusercontent.com/google/conscrypt/2.5.1/LICENSE</alisUrl>
+ <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/googleapis/master/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>
+ <alisUrl>https://raw.githubusercontent.com/LMAX-Exchange/disruptor/3.4.2/LICENCE.txt</alisUrl>
+ <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>
</aliasUrls>
<metric>1</metric>
</license>
diff --git a/asterixdb/pom.xml b/asterixdb/pom.xml
index 617d075..108765c 100644
--- a/asterixdb/pom.xml
+++ b/asterixdb/pom.xml
@@ -94,6 +94,7 @@
<azuredatalakejavasdk.version>12.7.2</azuredatalakejavasdk.version>
<gcsjavasdk.version>2.3.0</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>
@@ -1941,6 +1942,19 @@
<version>${hadoop.version}</version>
</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 -->
</dependencies>
</dependencyManagement>
diff --git a/asterixdb/src/main/appended-resources/supplemental-models.xml b/asterixdb/src/main/appended-resources/supplemental-models.xml
index 2a8e88a..b3231b9 100644
--- a/asterixdb/src/main/appended-resources/supplemental-models.xml
+++ b/asterixdb/src/main/appended-resources/supplemental-models.xml
@@ -1161,6 +1161,31 @@
</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 non-fixed ALv2 with no NOTICE file -->
<supplement>
<project>
@@ -1174,6 +1199,32 @@
</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 non-fixed ALv2 with no NOTICE file -->
<supplement>
<project>
@@ -1187,6 +1238,316 @@
</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.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.2.2-alpha</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>2.2.2-alpha</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>2.2.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.2.2-alpha</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>2.2.2-alpha</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>2.2.2-alpha</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-core-grpc</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.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>
+
+ <!-- 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.1</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>2.5.1</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>2.5.1</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>
+
+ <!-- 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.19</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.19</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.7.1</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>2.7.1</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>2.7.1</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
<!-- com.google.cloud:google-cloud-core is non-fixed ALv2 with no NOTICE file -->
<supplement>
<project>
@@ -1239,7 +1600,59 @@
</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.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-api</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-auth</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-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>
@@ -1252,6 +1665,97 @@
</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>
+ </properties>
+ </project>
+ </supplement>
+
<!-- com.google.protobuf has no NOTICE file -->
<supplement>
<project>
@@ -1388,7 +1892,7 @@
</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>
@@ -1713,4 +2217,28 @@
</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>
</supplementalDataModels>
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_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_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_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_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_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/hyracks-fullstack/algebricks/algebricks-compiler/pom.xml b/hyracks-fullstack/algebricks/algebricks-compiler/pom.xml
index fbe9dbc..f557d7a 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-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/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/expressions/ConstantExpression.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/ConstantExpression.java
index e2a68da..6f0cbba 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
@@ -55,7 +55,7 @@
@Override
public String toString() {
- return "TRUE";
+ return "true";
}
@Override
@@ -92,7 +92,7 @@
@Override
public String toString() {
- return "FALSE";
+ return "false";
}
@Override
@@ -129,7 +129,7 @@
@Override
public String toString() {
- return "NULL";
+ return "null";
}
@Override
@@ -166,7 +166,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/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/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/AbstractUnnestMapOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractUnnestMapOperator.java
index 5fc69b2..56e2e22 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractUnnestMapOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractUnnestMapOperator.java
@@ -25,10 +25,10 @@
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
import org.apache.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
+import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
public abstract class AbstractUnnestMapOperator extends AbstractUnnestOperator {
- protected final Mutable<ILogicalExpression> expression;
protected final List<Object> variableTypes;
protected boolean propagateInput;
protected List<Mutable<ILogicalExpression>> additionalFilteringExpressions;
@@ -42,7 +42,6 @@
public AbstractUnnestMapOperator(List<LogicalVariable> variables, Mutable<ILogicalExpression> expression,
List<Object> variableTypes, boolean propagateInput) {
super(variables, expression);
- this.expression = expression;
this.variableTypes = variableTypes;
this.propagateInput = propagateInput;
this.propagateIndexFilter = false;
@@ -90,6 +89,18 @@
};
}
+ @Override
+ public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform visitor) throws AlgebricksException {
+ boolean changed = super.acceptExpressionTransform(visitor);
+
+ if (additionalFilteringExpressions != null) {
+ for (Mutable<ILogicalExpression> filterExpr : additionalFilteringExpressions) {
+ changed |= visitor.transform(filterExpr);
+ }
+ }
+ return changed;
+ }
+
public boolean propagatesInput() {
return propagateInput;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/DataSourceScanOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/DataSourceScanOperator.java
index bb18014..9f73113 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/DataSourceScanOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/DataSourceScanOperator.java
@@ -57,7 +57,7 @@
}
public DataSourceScanOperator(List<LogicalVariable> variables, IDataSource<?> dataSource,
- Mutable<ILogicalExpression> selectCondition, long outputLimit, IProjectionInfo projectionInfo) {
+ Mutable<ILogicalExpression> selectCondition, long outputLimit, IProjectionInfo<?> projectionInfo) {
super(variables, dataSource);
projectVars = new ArrayList<>();
this.selectCondition = selectCondition;
@@ -77,7 +77,13 @@
@Override
public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform visitor) throws AlgebricksException {
- return false;
+ boolean changed = selectCondition != null && visitor.transform(selectCondition);
+ if (additionalFilteringExpressions != null) {
+ for (Mutable<ILogicalExpression> filterExpr : additionalFilteringExpressions) {
+ changed |= visitor.transform(filterExpr);
+ }
+ }
+ return changed;
}
@Override
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/UnnestMapOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/UnnestMapOperator.java
index c4bcc52..f8d07b8 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/UnnestMapOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/UnnestMapOperator.java
@@ -28,6 +28,7 @@
import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
import org.apache.hyracks.algebricks.core.algebra.typing.ITypingContext;
import org.apache.hyracks.algebricks.core.algebra.typing.NonPropagatingTypeEnvironment;
+import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
public class UnnestMapOperator extends AbstractUnnestMapOperator {
@@ -61,6 +62,12 @@
return visitor.visitUnnestMapOperator(this, arg);
}
+ @Override
+ public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform visitor) throws AlgebricksException {
+ boolean changed = super.acceptExpressionTransform(visitor);
+ return selectCondition != null && visitor.transform(selectCondition) || changed;
+ }
+
// When propagateInput is true,
// this operator propagates all input variables.
@Override
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/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/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/config/AlgebricksConfig.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java
index 86c6066..b626ae6 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java
@@ -37,4 +37,5 @@
public static final boolean ARRAY_INDEX_DEFAULT = true;
public static final int EXTERNAL_SCAN_BUFFER_SIZE =
StorageUtil.getIntSizeInBytes(8, StorageUtil.StorageUnit.KILOBYTE);
+ public static final boolean BATCH_LOOKUP_DEFAULT = false;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobGenContext.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobGenContext.java
index 7c7d5a8..471380c 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobGenContext.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobGenContext.java
@@ -35,6 +35,7 @@
import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
import org.apache.hyracks.algebricks.core.algebra.typing.ITypingContext;
import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
+import org.apache.hyracks.algebricks.data.IAWriterFactory;
import org.apache.hyracks.algebricks.data.IBinaryBooleanInspectorFactory;
import org.apache.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
import org.apache.hyracks.algebricks.data.IBinaryHashFunctionFactoryProvider;
@@ -42,6 +43,7 @@
import org.apache.hyracks.algebricks.data.IBinaryIntegerInspectorFactory;
import org.apache.hyracks.algebricks.data.INormalizedKeyComputerFactoryProvider;
import org.apache.hyracks.algebricks.data.IPrinterFactoryProvider;
+import org.apache.hyracks.algebricks.data.IResultSerializerFactoryProvider;
import org.apache.hyracks.algebricks.data.ISerializerDeserializerProvider;
import org.apache.hyracks.algebricks.data.ITypeTraitProvider;
import org.apache.hyracks.algebricks.data.IUnnestingPositionWriterFactory;
@@ -57,6 +59,8 @@
private final IBinaryHashFunctionFamilyProvider hashFunctionFamilyProvider;
private final IBinaryComparatorFactoryProvider comparatorFactoryProvider;
private final IPrinterFactoryProvider printerFactoryProvider;
+ private final IAWriterFactory writerFactory;
+ private final IResultSerializerFactoryProvider resultSerializerFactoryProvider;
private final ITypeTraitProvider typeTraitProvider;
private final IMetadataProvider<?, ?> metadataProvider;
private final IMissingWriterFactory missingWriterFactory;
@@ -86,6 +90,7 @@
IBinaryComparatorFactoryProvider comparatorFactoryProvider, ITypeTraitProvider typeTraitProvider,
IBinaryBooleanInspectorFactory booleanInspectorFactory,
IBinaryIntegerInspectorFactory integerInspectorFactory, IPrinterFactoryProvider printerFactoryProvider,
+ IAWriterFactory writerFactory, IResultSerializerFactoryProvider resultSerializerFactoryProvider,
IMissingWriterFactory missingWriterFactory, IMissingWriterFactory nullWriterFactory,
IUnnestingPositionWriterFactory unnestingPositionWriterFactory,
INormalizedKeyComputerFactoryProvider normalizedKeyComputerFactoryProvider,
@@ -106,6 +111,8 @@
this.booleanInspectorFactory = booleanInspectorFactory;
this.integerInspectorFactory = integerInspectorFactory;
this.printerFactoryProvider = printerFactoryProvider;
+ this.writerFactory = writerFactory;
+ this.resultSerializerFactoryProvider = resultSerializerFactoryProvider;
this.clusterLocations = clusterLocations;
this.normalizedKeyComputerFactoryProvider = normalizedKeyComputerFactoryProvider;
this.missingWriterFactory = missingWriterFactory;
@@ -172,6 +179,14 @@
return printerFactoryProvider;
}
+ public IAWriterFactory getWriterFactory() {
+ return writerFactory;
+ }
+
+ public IResultSerializerFactoryProvider getResultSerializerFactoryProvider() {
+ return resultSerializerFactoryProvider;
+ }
+
public IPredicateEvaluatorFactoryProvider getPredicateEvaluatorFactoryProvider() {
return predEvaluatorFactoryProvider;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/AlgebricksOptimizationContext.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/AlgebricksOptimizationContext.java
index d1fd247..bee842c 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/AlgebricksOptimizationContext.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/AlgebricksOptimizationContext.java
@@ -46,45 +46,29 @@
import org.apache.hyracks.algebricks.core.algebra.properties.ILogicalPropertiesVector;
import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.NoOpWarningCollector;
/**
* The Algebricks default implementation for IOptimizationContext.
*/
-@SuppressWarnings({ "unchecked", "rawtypes" })
+@SuppressWarnings({ "rawtypes" })
public class AlgebricksOptimizationContext implements IOptimizationContext {
- private int varCounter;
+ private final IOptimizationContextFactory optContextFactory;
private final IExpressionEvalSizeComputer expressionEvalSizeComputer;
private final IMergeAggregationExpressionFactory mergeAggregationExpressionFactory;
private final PhysicalOptimizationConfig physicalOptimizationConfig;
- private final IVariableEvalSizeEnvironment varEvalSizeEnv = new IVariableEvalSizeEnvironment() {
- Map<LogicalVariable, Integer> varSizeMap = new HashMap<>();
-
- @Override
- public void setVariableEvalSize(LogicalVariable var, int size) {
- varSizeMap.put(var, size);
- }
-
- @Override
- public int getVariableEvalSize(LogicalVariable var) {
- return varSizeMap.get(var);
- }
- };
-
- private Map<ILogicalOperator, IVariableTypeEnvironment> typeEnvMap = new HashMap<>();
-
- private Map<ILogicalOperator, HashSet<ILogicalOperator>> alreadyCompared = new HashMap<>();
- private Map<IAlgebraicRewriteRule, HashSet<ILogicalOperator>> dontApply = new HashMap<>();
- private Map<LogicalVariable, FunctionalDependency> varToPrimaryKey = new HashMap<>();
-
- private IMetadataProvider metadataProvider;
- private HashSet<LogicalVariable> notToBeInlinedVars = new HashSet<>();
-
+ private final VariableEvalSizeEnvironmentImpl varEvalSizeEnv = new VariableEvalSizeEnvironmentImpl();
+ private final Map<ILogicalOperator, IVariableTypeEnvironment> typeEnvMap = new HashMap<>();
+ private final Map<ILogicalOperator, HashSet<ILogicalOperator>> alreadyCompared = new HashMap<>();
+ private final Map<IAlgebraicRewriteRule, HashSet<ILogicalOperator>> dontApply = new HashMap<>();
+ private final Map<LogicalVariable, FunctionalDependency> varToPrimaryKey = new HashMap<>();
+ private final HashSet<LogicalVariable> notToBeInlinedVars = new HashSet<>();
protected final Map<ILogicalOperator, List<FunctionalDependency>> fdGlobalMap = new HashMap<>();
protected final Map<ILogicalOperator, Map<LogicalVariable, EquivalenceClass>> eqClassGlobalMap = new HashMap<>();
-
protected final Map<ILogicalOperator, ILogicalPropertiesVector> logicalProps = new HashMap<>();
+
private final IExpressionTypeComputer expressionTypeComputer;
private final IMissableTypeComputer nullableTypeComputer;
private final INodeDomain defaultNodeDomain;
@@ -94,12 +78,18 @@
private final PlanStructureVerifier planStructureVerifier;
private final PlanStabilityVerifier planStabilityVerifier;
- public AlgebricksOptimizationContext(int varCounter, IExpressionEvalSizeComputer expressionEvalSizeComputer,
+ private int varCounter;
+ private IMetadataProvider metadataProvider;
+ private Object compilerFactory;
+
+ public AlgebricksOptimizationContext(IOptimizationContextFactory optContextFactory, int varCounter,
+ IExpressionEvalSizeComputer expressionEvalSizeComputer,
IMergeAggregationExpressionFactory mergeAggregationExpressionFactory,
IExpressionTypeComputer expressionTypeComputer, IMissableTypeComputer nullableTypeComputer,
IConflictingTypeResolver conflictingTypeResovler, PhysicalOptimizationConfig physicalOptimizationConfig,
AlgebricksPartitionConstraint clusterLocations, IPlanPrettyPrinter prettyPrinter,
IWarningCollector warningCollector) {
+ this.optContextFactory = optContextFactory;
this.varCounter = varCounter;
this.expressionEvalSizeComputer = expressionEvalSizeComputer;
this.mergeAggregationExpressionFactory = mergeAggregationExpressionFactory;
@@ -115,6 +105,35 @@
this.planStabilityVerifier = isSanityCheckEnabled ? new PlanStabilityVerifier(prettyPrinter) : null;
}
+ public AlgebricksOptimizationContext(AlgebricksOptimizationContext from) {
+ optContextFactory = from.optContextFactory;
+ varCounter = from.varCounter;
+ expressionEvalSizeComputer = from.expressionEvalSizeComputer;
+ mergeAggregationExpressionFactory = from.mergeAggregationExpressionFactory;
+ expressionTypeComputer = from.expressionTypeComputer;
+ nullableTypeComputer = from.nullableTypeComputer;
+ physicalOptimizationConfig = from.physicalOptimizationConfig;
+ defaultNodeDomain = from.defaultNodeDomain;
+ prettyPrinter = from.prettyPrinter;
+ conflictingTypeResovler = from.conflictingTypeResovler;
+ warningCollector = NoOpWarningCollector.INSTANCE;
+ boolean isSanityCheckEnabled = physicalOptimizationConfig.isSanityCheckEnabled();
+ planStructureVerifier = isSanityCheckEnabled ? new PlanStructureVerifier(from.prettyPrinter, this) : null;
+ planStabilityVerifier = isSanityCheckEnabled ? new PlanStabilityVerifier(from.prettyPrinter) : null;
+ metadataProvider = from.metadataProvider;
+ compilerFactory = from.compilerFactory;
+
+ varEvalSizeEnv.varSizeMap.putAll(from.varEvalSizeEnv.varSizeMap);
+ typeEnvMap.putAll(from.typeEnvMap);
+ alreadyCompared.putAll(from.alreadyCompared);
+ dontApply.putAll(from.dontApply);
+ varToPrimaryKey.putAll(from.varToPrimaryKey);
+ notToBeInlinedVars.addAll(from.notToBeInlinedVars);
+ fdGlobalMap.putAll(from.fdGlobalMap);
+ eqClassGlobalMap.putAll(from.eqClassGlobalMap);
+ logicalProps.putAll(from.logicalProps);
+ }
+
@Override
public int getVarCounter() {
return varCounter;
@@ -354,4 +373,34 @@
public PlanStabilityVerifier getPlanStabilityVerifier() {
return planStabilityVerifier;
}
+
+ @Override
+ public IOptimizationContextFactory getOptimizationContextFactory() {
+ return optContextFactory;
+ }
+
+ @Override
+ public void setCompilerFactory(Object compilerFactory) {
+ this.compilerFactory = compilerFactory;
+ }
+
+ @Override
+ public Object getCompilerFactory() {
+ return compilerFactory;
+ }
+
+ protected static class VariableEvalSizeEnvironmentImpl implements IVariableEvalSizeEnvironment {
+
+ protected final Map<LogicalVariable, Integer> varSizeMap = new HashMap<>();
+
+ @Override
+ public void setVariableEvalSize(LogicalVariable var, int size) {
+ varSizeMap.put(var, size);
+ }
+
+ @Override
+ public int getVariableEvalSize(LogicalVariable var) {
+ return varSizeMap.get(var);
+ }
+ }
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/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/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/IRuleSetKind.java
similarity index 83%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/IRuleSetKind.java
index 6095b26..5ce61ec 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/IRuleSetKind.java
@@ -17,8 +17,8 @@
* under the License.
*/
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+package org.apache.hyracks.algebricks.core.rewriter.base;
-select value v from range(1,2) v where v > ?;
+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..4d54a10 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java
@@ -47,6 +47,7 @@
private static final String MIN_MEMORY_ALLOCATION = "MIN_MEMORY_ALLOCATION";
private static final String ARRAY_INDEX = "ARRAY_INDEX";
private static final String EXTERNAL_SCAN_BUFFER_SIZE = "EXTERNAL_SCAN_BUFFER_SIZE";
+ private static final String BATCH_LOOKUP = "BATCH_LOOKUP";
private final Properties properties = new Properties();
@@ -239,6 +240,14 @@
return getInt(EXTERNAL_SCAN_BUFFER_SIZE, AlgebricksConfig.EXTERNAL_SCAN_BUFFER_SIZE);
}
+ public boolean isBatchLookupEnabled() {
+ return getBoolean(BATCH_LOOKUP, AlgebricksConfig.BATCH_LOOKUP_DEFAULT);
+ }
+
+ public void setBatchLookup(boolean batchedLookup) {
+ setBoolean(BATCH_LOOKUP, batchedLookup);
+ }
+
public void setExternalScanBufferSize(int bufferSize) {
setInt(EXTERNAL_SCAN_BUFFER_SIZE, bufferSize);
}
diff --git a/hyracks-fullstack/algebricks/algebricks-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/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-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/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/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/NoOpWarningCollector.java
similarity index 62%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/NoOpWarningCollector.java
index 6095b26..caf2464 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/NoOpWarningCollector.java
@@ -17,8 +17,27 @@
* under the License.
*/
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+package org.apache.hyracks.api.exceptions;
-select value v from range(1,2) v where v > ?;
+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/profiling/IStatsCollector.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/profiling/IStatsCollector.java
index 8930d34..c903c68 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/profiling/IStatsCollector.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/profiling/IStatsCollector.java
@@ -43,6 +43,13 @@
IOperatorStats getOrAddOperatorStats(String operatorName);
/**
+ * @param operatorName
+ * @return {@link IOperatorStats} for the operator with name <code>operatorName</code>
+ * if it already exists, or {@code null} if the operator does not exist
+ */
+ IOperatorStats getOperatorStats(String operatorName);
+
+ /**
* Get every registered operator stats object
* @return All registered operators, and their collected stats, with the names as keys and stats as values
*/
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/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 409da1f8..66b71fa 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/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/job/profiling/StatsCollector.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/StatsCollector.java
index 878d7c3..580a560 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/StatsCollector.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/StatsCollector.java
@@ -52,6 +52,11 @@
}
@Override
+ public IOperatorStats getOperatorStats(String operatorName) {
+ return operatorStatsMap.get(operatorName);
+ }
+
+ @Override
public Map<String, IOperatorStats> getAllOperatorStats() {
return Collections.unmodifiableMap(operatorStatsMap);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/om/JobProfile.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/om/JobProfile.java
index c4eff85..0026af4 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,11 +21,18 @@
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
+import java.util.Arrays;
import java.util.HashMap;
+import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
+import org.apache.hyracks.api.dataflow.TaskAttemptId;
+import org.apache.hyracks.api.dataflow.TaskId;
import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
+import org.apache.hyracks.api.job.profiling.IStatsCollector;
+import org.apache.hyracks.api.job.profiling.OperatorStats;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.node.ArrayNode;
@@ -109,4 +116,44 @@
entry.getValue().writeFields(output);
}
}
+
+ public List<IOperatorStats> getAggregatedStats(List<String> operatorNames) {
+ if (jobletProfiles == null || operatorNames == null || operatorNames.isEmpty()) {
+ return null;
+ }
+ // gather final task attempts for each task
+ Map<TaskId, TaskProfile> taskProfileMap = new HashMap<>();
+ for (JobletProfile jobletProfile : jobletProfiles.values()) {
+ for (TaskProfile taskProfile : jobletProfile.getTaskProfiles().values()) {
+ TaskAttemptId taskAttemptId = taskProfile.getTaskId();
+ TaskId taskId = taskAttemptId.getTaskId();
+ TaskProfile existingProfile = taskProfileMap.get(taskId);
+ if (existingProfile == null || taskAttemptId.getAttempt() > existingProfile.getTaskId().getAttempt()) {
+ taskProfileMap.put(taskId, taskProfile);
+ }
+ }
+ }
+ // compute aggregated counts
+ int n = operatorNames.size();
+ IOperatorStats[] outStats = new IOperatorStats[n];
+ for (TaskProfile taskProfile : taskProfileMap.values()) {
+ IStatsCollector statsCollector = taskProfile.getStatsCollector();
+ for (int i = 0; i < n; i++) {
+ String operatorName = operatorNames.get(i);
+ IOperatorStats opTaskStats = statsCollector.getOperatorStats(operatorName);
+ if (opTaskStats == null) {
+ continue;
+ }
+ IOperatorStats opOutStats = outStats[i];
+ if (opOutStats == null) {
+ opOutStats = new OperatorStats(operatorName);
+ outStats[i] = opOutStats;
+ }
+ opOutStats.getTupleCounter().update(opTaskStats.getTupleCounter().get());
+ opOutStats.getTimeCounter().update(opTaskStats.getTimeCounter().get());
+ opOutStats.getDiskIoCounter().update(opTaskStats.getDiskIoCounter().get());
+ }
+ }
+ return Arrays.asList(outStats);
+ }
}
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-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-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-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-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/api/IDiskBTreeStatefulPointSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/api/IDiskBTreeStatefulPointSearchCursor.java
new file mode 100644
index 0000000..f436636
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/api/IDiskBTreeStatefulPointSearchCursor.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.btree.api;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.btree.impls.DiskBTreePointSearchCursor;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
+import org.apache.hyracks.storage.common.ISearchPredicate;
+
+/**
+ * Allows stateful {@link DiskBTreePointSearchCursor} to resume the search without closing and reopening the cursor
+ * Note: upon closing a cursor, ensure the search state is cleared
+ */
+public interface IDiskBTreeStatefulPointSearchCursor {
+ int getLastPageId();
+
+ void setCursorToNextKey(ISearchPredicate searchPred) throws HyracksDataException;
+
+ ITreeIndexFrame getFrame();
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
index 2455625..0ab88a5 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
@@ -29,10 +29,12 @@
import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import org.apache.hyracks.storage.am.common.impls.DefaultTupleProjectorFactory;
+import org.apache.hyracks.storage.common.projection.ITupleProjectorFactory;
public class BTreeSearchOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
- private static final long serialVersionUID = 1L;
+ private static final long serialVersionUID = 2L;
protected final int[] lowKeyFields;
protected final int[] highKeyFields;
@@ -52,6 +54,7 @@
protected byte[] searchCallbackProceedResultTrueValue;
protected final ITupleFilterFactory tupleFilterFactory;
protected final long outputLimit;
+ protected final ITupleProjectorFactory tupleProjectorFactory;
public BTreeSearchOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor outRecDesc,
int[] lowKeyFields, int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
@@ -61,7 +64,8 @@
IMissingWriterFactory nonFilterWriterFactory) {
this(spec, outRecDesc, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory,
retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, minFilterFieldIndexes,
- maxFilterFieldIndexes, appendIndexFilter, nonFilterWriterFactory, null, -1, false, null, null);
+ maxFilterFieldIndexes, appendIndexFilter, nonFilterWriterFactory, null, -1, false, null, null,
+ DefaultTupleProjectorFactory.INSTANCE);
}
public BTreeSearchOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor outRecDesc,
@@ -71,7 +75,7 @@
int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, boolean appendIndexFilter,
IMissingWriterFactory nonFilterWriterFactory, ITupleFilterFactory tupleFilterFactory, long outputLimit,
boolean appendOpCallbackProceedResult, byte[] searchCallbackProceedResultFalseValue,
- byte[] searchCallbackProceedResultTrueValue) {
+ byte[] searchCallbackProceedResultTrueValue, ITupleProjectorFactory tupleProjectorFactory) {
super(spec, 1, 1);
this.indexHelperFactory = indexHelperFactory;
this.retainInput = retainInput;
@@ -92,6 +96,7 @@
this.appendOpCallbackProceedResult = appendOpCallbackProceedResult;
this.searchCallbackProceedResultFalseValue = searchCallbackProceedResultFalseValue;
this.searchCallbackProceedResultTrueValue = searchCallbackProceedResultTrueValue;
+ this.tupleProjectorFactory = tupleProjectorFactory;
}
@Override
@@ -102,7 +107,7 @@
lowKeyInclusive, highKeyInclusive, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter,
nonFilterWriterFactory, tupleFilterFactory, outputLimit, appendOpCallbackProceedResult,
- searchCallbackProceedResultFalseValue, searchCallbackProceedResultTrueValue);
+ searchCallbackProceedResultFalseValue, searchCallbackProceedResultTrueValue, tupleProjectorFactory);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
index fb331bf..a56e305 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
@@ -22,6 +22,7 @@
import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.util.HyracksConstants;
import org.apache.hyracks.dataflow.common.data.accessors.PermutingFrameTupleReference;
import org.apache.hyracks.storage.am.btree.impls.RangePredicate;
import org.apache.hyracks.storage.am.btree.util.BTreeUtils;
@@ -30,9 +31,11 @@
import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
import org.apache.hyracks.storage.am.common.dataflow.IndexSearchOperatorNodePushable;
+import org.apache.hyracks.storage.am.common.impls.DefaultTupleProjectorFactory;
import org.apache.hyracks.storage.common.IIndexAccessParameters;
import org.apache.hyracks.storage.common.ISearchPredicate;
import org.apache.hyracks.storage.common.MultiComparator;
+import org.apache.hyracks.storage.common.projection.ITupleProjectorFactory;
public class BTreeSearchOperatorNodePushable extends IndexSearchOperatorNodePushable {
protected final boolean lowKeyInclusive;
@@ -52,7 +55,7 @@
this(ctx, partition, inputRecDesc, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive,
minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory, retainInput, retainMissing,
nonMatchWriterFactory, searchCallbackFactory, appendIndexFilter, nonFilterWriterFactory, null, -1,
- false, null, null);
+ false, null, null, DefaultTupleProjectorFactory.INSTANCE);
}
public BTreeSearchOperatorNodePushable(IHyracksTaskContext ctx, int partition, RecordDescriptor inputRecDesc,
@@ -62,11 +65,12 @@
ISearchOperationCallbackFactory searchCallbackFactory, boolean appendIndexFilter,
IMissingWriterFactory nonFilterWriterFactory, ITupleFilterFactory tupleFilterFactory, long outputLimit,
boolean appendOpCallbackProceedResult, byte[] searchCallbackProceedResultFalseValue,
- byte[] searchCallbackProceedResultTrueValue) throws HyracksDataException {
+ byte[] searchCallbackProceedResultTrueValue, ITupleProjectorFactory projectorFactory)
+ throws HyracksDataException {
super(ctx, inputRecDesc, partition, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
retainInput, retainMissing, nonMatchWriterFactory, searchCallbackFactory, appendIndexFilter,
nonFilterWriterFactory, tupleFilterFactory, outputLimit, appendOpCallbackProceedResult,
- searchCallbackProceedResultFalseValue, searchCallbackProceedResultTrueValue);
+ searchCallbackProceedResultFalseValue, searchCallbackProceedResultTrueValue, projectorFactory);
this.lowKeyInclusive = lowKeyInclusive;
this.highKeyInclusive = highKeyInclusive;
if (lowKeyFields != null && lowKeyFields.length > 0) {
@@ -111,7 +115,8 @@
@Override
protected void addAdditionalIndexAccessorParams(IIndexAccessParameters iap) throws HyracksDataException {
- // No additional parameters are required for the B+Tree search case
+ //Set tuple projector to get the information about the pushed down value accesses (if supported by the index)
+ iap.getParameters().put(HyracksConstants.TUPLE_PROJECTOR, tupleProjector);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java
index 78faaff..40b2f5c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java
@@ -42,7 +42,6 @@
import org.apache.hyracks.storage.am.btree.impls.BTreeOpContext.PageValidationInfo;
import org.apache.hyracks.storage.am.common.api.IBTreeIndexTupleReference;
import org.apache.hyracks.storage.am.common.api.IPageManager;
-import org.apache.hyracks.storage.am.common.api.ISplitKey;
import org.apache.hyracks.storage.am.common.api.ITreeIndexAccessor;
import org.apache.hyracks.storage.am.common.api.ITreeIndexCursor;
import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
@@ -51,7 +50,6 @@
import org.apache.hyracks.storage.am.common.frames.FrameOpSpaceStatus;
import org.apache.hyracks.storage.am.common.impls.AbstractTreeIndex;
import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
-import org.apache.hyracks.storage.am.common.impls.NodeFrontier;
import org.apache.hyracks.storage.am.common.impls.TreeIndexDiskOrderScanCursor;
import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
import org.apache.hyracks.storage.common.IIndexAccessParameters;
@@ -64,17 +62,13 @@
import org.apache.hyracks.storage.common.ISearchPredicate;
import org.apache.hyracks.storage.common.MultiComparator;
import org.apache.hyracks.storage.common.NoOpIndexCursorStats;
-import org.apache.hyracks.storage.common.buffercache.BufferCache;
import org.apache.hyracks.storage.common.buffercache.IBufferCache;
import org.apache.hyracks.storage.common.buffercache.ICachedPage;
import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
import org.apache.hyracks.storage.common.file.BufferedFileHandle;
-import org.apache.hyracks.util.JSONUtil;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
-import com.fasterxml.jackson.databind.node.ObjectNode;
-
public class BTree extends AbstractTreeIndex {
public static final float DEFAULT_FILL_FACTOR = 0.7f;
@@ -85,7 +79,7 @@
private final AtomicInteger smoCounter;
private final ReadWriteLock treeLatch;
- private final int maxTupleSize;
+ protected final int maxTupleSize;
public BTree(IBufferCache bufferCache, IPageManager freePageManager, ITreeIndexFrameFactory interiorFrameFactory,
ITreeIndexFrameFactory leafFrameFactory, IBinaryComparatorFactory[] cmpFactories, int fieldCount,
@@ -886,13 +880,13 @@
}
@Override
- public BTreeRangeSearchCursor createSearchCursor(boolean exclusive) {
+ public ITreeIndexCursor createSearchCursor(boolean exclusive) {
IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) btree.getLeafFrameFactory().createFrame();
return new BTreeRangeSearchCursor(leafFrame, exclusive, (IIndexCursorStats) iap.getParameters()
.getOrDefault(HyracksConstants.INDEX_CURSOR_STATS, NoOpIndexCursorStats.INSTANCE));
}
- public BTreeRangeSearchCursor createPointCursor(boolean exclusive, boolean stateful) {
+ public ITreeIndexCursor createPointCursor(boolean exclusive, boolean stateful) {
return createSearchCursor(exclusive);
}
@@ -1004,220 +998,7 @@
@Override
public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
boolean checkIfEmptyIndex, IPageWriteCallback callback) throws HyracksDataException {
- return new BTreeBulkLoader(fillFactor, verifyInput, callback);
- }
-
- public class BTreeBulkLoader extends AbstractTreeIndex.AbstractTreeIndexBulkLoader {
- protected final ISplitKey splitKey;
- protected final boolean verifyInput;
-
- public BTreeBulkLoader(float fillFactor, boolean verifyInput, IPageWriteCallback callback)
- throws HyracksDataException {
- super(fillFactor, callback);
- this.verifyInput = verifyInput;
- splitKey = new BTreeSplitKey(leafFrame.getTupleWriter().createTupleReference());
- splitKey.getTuple().setFieldCount(cmp.getKeyFieldCount());
- }
-
- @Override
- public void add(ITupleReference tuple) throws HyracksDataException {
- try {
- int tupleSize = Math.max(leafFrame.getBytesRequiredToWriteTuple(tuple),
- interiorFrame.getBytesRequiredToWriteTuple(tuple));
- NodeFrontier leafFrontier = nodeFrontiers.get(0);
- int spaceNeeded = tupleWriter.bytesRequired(tuple) + slotSize;
- int spaceUsed = leafFrame.getBuffer().capacity() - leafFrame.getTotalFreeSpace();
-
- // try to free space by compression
- if (spaceUsed + spaceNeeded > leafMaxBytes) {
- leafFrame.compress();
- spaceUsed = leafFrame.getBuffer().capacity() - leafFrame.getTotalFreeSpace();
- }
- //full, allocate new page
- if (spaceUsed + spaceNeeded > leafMaxBytes) {
- if (leafFrame.getTupleCount() == 0) {
- bufferCache.returnPage(leafFrontier.page, false);
- } else {
- leafFrontier.lastTuple.resetByTupleIndex(leafFrame, leafFrame.getTupleCount() - 1);
- if (verifyInput) {
- verifyInputTuple(tuple, leafFrontier.lastTuple);
- }
- int splitKeySize = tupleWriter.bytesRequired(leafFrontier.lastTuple, 0, cmp.getKeyFieldCount());
- splitKey.initData(splitKeySize);
- tupleWriter.writeTupleFields(leafFrontier.lastTuple, 0, cmp.getKeyFieldCount(),
- splitKey.getBuffer().array(), 0);
- splitKey.getTuple().resetByTupleOffset(splitKey.getBuffer().array(), 0);
- splitKey.setLeftPage(leafFrontier.pageId);
-
- propagateBulk(1, pagesToWrite);
-
- leafFrontier.pageId = freePageManager.takePage(metaFrame);
-
- ((IBTreeLeafFrame) leafFrame).setNextLeaf(leafFrontier.pageId);
-
- write(leafFrontier.page);
- for (ICachedPage c : pagesToWrite) {
- write(c);
- }
- pagesToWrite.clear();
- splitKey.setRightPage(leafFrontier.pageId);
- }
- if (tupleSize > maxTupleSize) {
- final long dpid = BufferedFileHandle.getDiskPageId(getFileId(), leafFrontier.pageId);
- // calculate required number of pages.
- int headerSize = Math.max(leafFrame.getPageHeaderSize(), interiorFrame.getPageHeaderSize());
- final int multiplier =
- (int) Math.ceil((double) tupleSize / (bufferCache.getPageSize() - headerSize));
- if (multiplier > 1) {
- leafFrontier.page = bufferCache.confiscateLargePage(dpid, multiplier,
- freePageManager.takeBlock(metaFrame, multiplier - 1));
- } else {
- leafFrontier.page = bufferCache.confiscatePage(dpid);
- }
- leafFrame.setPage(leafFrontier.page);
- leafFrame.initBuffer((byte) 0);
- ((IBTreeLeafFrame) leafFrame).setLargeFlag(true);
- } else {
- final long dpid = BufferedFileHandle.getDiskPageId(getFileId(), leafFrontier.pageId);
- leafFrontier.page = bufferCache.confiscatePage(dpid);
- leafFrame.setPage(leafFrontier.page);
- leafFrame.initBuffer((byte) 0);
- }
- } else {
- if (verifyInput && leafFrame.getTupleCount() > 0) {
- leafFrontier.lastTuple.resetByTupleIndex(leafFrame, leafFrame.getTupleCount() - 1);
- verifyInputTuple(tuple, leafFrontier.lastTuple);
- }
- }
- ((IBTreeLeafFrame) leafFrame).insertSorted(tuple);
- } catch (HyracksDataException | RuntimeException e) {
- logState(tuple, e);
- handleException();
- throw e;
- }
- }
-
- protected void verifyInputTuple(ITupleReference tuple, ITupleReference prevTuple) throws HyracksDataException {
- // New tuple should be strictly greater than last tuple.
- int cmpResult = cmp.compare(tuple, prevTuple);
- if (cmpResult < 0) {
- throw HyracksDataException.create(ErrorCode.UNSORTED_LOAD_INPUT);
- }
- if (cmpResult == 0) {
- throw HyracksDataException.create(ErrorCode.DUPLICATE_LOAD_INPUT);
- }
- }
-
- protected void propagateBulk(int level, List<ICachedPage> pagesToWrite) throws HyracksDataException {
- if (splitKey.getBuffer() == null) {
- return;
- }
-
- if (level >= nodeFrontiers.size()) {
- addLevel();
- }
-
- NodeFrontier frontier = nodeFrontiers.get(level);
- interiorFrame.setPage(frontier.page);
-
- ITupleReference tuple = splitKey.getTuple();
- int tupleBytes = tupleWriter.bytesRequired(tuple, 0, cmp.getKeyFieldCount());
- int spaceNeeded = tupleBytes + slotSize + 4;
- if (tupleBytes > interiorFrame.getMaxTupleSize(BTree.this.bufferCache.getPageSize())) {
- throw HyracksDataException.create(ErrorCode.RECORD_IS_TOO_LARGE, tupleBytes,
- interiorFrame.getMaxTupleSize(BTree.this.bufferCache.getPageSize()));
- }
-
- int spaceUsed = interiorFrame.getBuffer().capacity() - interiorFrame.getTotalFreeSpace();
- if (spaceUsed + spaceNeeded > interiorMaxBytes) {
-
- ISplitKey copyKey = splitKey.duplicate(leafFrame.getTupleWriter().createTupleReference());
- tuple = copyKey.getTuple();
-
- frontier.lastTuple.resetByTupleIndex(interiorFrame, interiorFrame.getTupleCount() - 1);
- int splitKeySize = tupleWriter.bytesRequired(frontier.lastTuple, 0, cmp.getKeyFieldCount());
- splitKey.initData(splitKeySize);
- tupleWriter.writeTupleFields(frontier.lastTuple, 0, cmp.getKeyFieldCount(),
- splitKey.getBuffer().array(), 0);
- splitKey.getTuple().resetByTupleOffset(splitKey.getBuffer().array(), 0);
-
- ((IBTreeInteriorFrame) interiorFrame).deleteGreatest();
- int finalPageId = freePageManager.takePage(metaFrame);
- frontier.page.setDiskPageId(BufferedFileHandle.getDiskPageId(getFileId(), finalPageId));
- pagesToWrite.add(frontier.page);
- splitKey.setLeftPage(finalPageId);
-
- propagateBulk(level + 1, pagesToWrite);
- frontier.page = bufferCache.confiscatePage(BufferCache.INVALID_DPID);
- interiorFrame.setPage(frontier.page);
- interiorFrame.initBuffer((byte) level);
- }
- ((IBTreeInteriorFrame) interiorFrame).insertSorted(tuple);
- }
-
- private void persistFrontiers(int level, int rightPage) throws HyracksDataException {
- if (level >= nodeFrontiers.size()) {
- rootPage = nodeFrontiers.get(level - 1).pageId;
- releasedLatches = true;
- return;
- }
- if (level < 1) {
- ICachedPage lastLeaf = nodeFrontiers.get(level).page;
- int lastLeafPage = nodeFrontiers.get(level).pageId;
- lastLeaf.setDiskPageId(BufferedFileHandle.getDiskPageId(getFileId(), nodeFrontiers.get(level).pageId));
- write(lastLeaf);
- nodeFrontiers.get(level).page = null;
- persistFrontiers(level + 1, lastLeafPage);
- return;
- }
- NodeFrontier frontier = nodeFrontiers.get(level);
- interiorFrame.setPage(frontier.page);
- //just finalize = the layer right above the leaves has correct righthand pointers already
- if (rightPage < 0) {
- throw new HyracksDataException(
- "Error in index creation. Internal node appears to have no rightmost guide");
- }
- ((IBTreeInteriorFrame) interiorFrame).setRightmostChildPageId(rightPage);
- int finalPageId = freePageManager.takePage(metaFrame);
- frontier.page.setDiskPageId(BufferedFileHandle.getDiskPageId(getFileId(), finalPageId));
- write(frontier.page);
- frontier.pageId = finalPageId;
- persistFrontiers(level + 1, finalPageId);
- }
-
- @Override
- public void end() throws HyracksDataException {
- try {
- persistFrontiers(0, -1);
- super.end();
- } catch (HyracksDataException | RuntimeException e) {
- handleException();
- throw e;
- }
- }
-
- @Override
- public void abort() throws HyracksDataException {
- super.handleException();
- }
-
- private void logState(ITupleReference tuple, Exception e) {
- try {
- ObjectNode state = JSONUtil.createObject();
- state.set("leafFrame", leafFrame.getState());
- state.set("interiorFrame", interiorFrame.getState());
- int tupleSize = Math.max(leafFrame.getBytesRequiredToWriteTuple(tuple),
- interiorFrame.getBytesRequiredToWriteTuple(tuple));
- state.put("tupleSize", tupleSize);
- state.put("spaceNeeded", tupleWriter.bytesRequired(tuple) + slotSize);
- state.put("spaceUsed", leafFrame.getBuffer().capacity() - leafFrame.getTotalFreeSpace());
- state.put("leafMaxBytes", leafMaxBytes);
- state.put("maxTupleSize", maxTupleSize);
- LOGGER.error("failed to add tuple {}", state, e);
- } catch (Throwable t) {
- e.addSuppressed(t);
- }
- }
+ return new BTreeNSMBulkLoader(fillFactor, verifyInput, callback, this);
}
@SuppressWarnings("rawtypes")
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeNSMBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeNSMBulkLoader.java
new file mode 100644
index 0000000..04c84e1
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeNSMBulkLoader.java
@@ -0,0 +1,279 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.btree.impls;
+
+import java.util.List;
+
+import org.apache.hyracks.api.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.btree.api.IBTreeInteriorFrame;
+import org.apache.hyracks.storage.am.btree.api.IBTreeLeafFrame;
+import org.apache.hyracks.storage.am.common.api.ISplitKey;
+import org.apache.hyracks.storage.am.common.api.ITreeIndex;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
+import org.apache.hyracks.storage.am.common.impls.AbstractTreeIndexBulkLoader;
+import org.apache.hyracks.storage.am.common.impls.NodeFrontier;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.storage.common.buffercache.ICachedPage;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
+import org.apache.hyracks.storage.common.file.BufferedFileHandle;
+import org.apache.hyracks.util.JSONUtil;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
+public class BTreeNSMBulkLoader extends AbstractTreeIndexBulkLoader {
+ private static final Logger LOGGER = LogManager.getLogger();
+ protected final ISplitKey splitKey;
+ protected final boolean verifyInput;
+ private final int maxTupleSize;
+
+ public BTreeNSMBulkLoader(float fillFactor, boolean verifyInput, IPageWriteCallback callback, ITreeIndex index)
+ throws HyracksDataException {
+ this(fillFactor, verifyInput, callback, index, index.getLeafFrameFactory().createFrame());
+ }
+
+ protected BTreeNSMBulkLoader(float fillFactor, boolean verifyInput, IPageWriteCallback callback, ITreeIndex index,
+ ITreeIndexFrame leafFrame) throws HyracksDataException {
+ super(fillFactor, callback, index, leafFrame);
+ this.verifyInput = verifyInput;
+ splitKey = new BTreeSplitKey(tupleWriter.createTupleReference());
+ splitKey.getTuple().setFieldCount(cmp.getKeyFieldCount());
+ maxTupleSize = ((BTree) index).maxTupleSize;
+ }
+
+ @Override
+ public void add(ITupleReference tuple) throws HyracksDataException {
+ try {
+ int tupleSize = Math.max(leafFrame.getBytesRequiredToWriteTuple(tuple),
+ interiorFrame.getBytesRequiredToWriteTuple(tuple));
+ NodeFrontier leafFrontier = nodeFrontiers.get(0);
+ int spaceNeeded = tupleWriter.bytesRequired(tuple) + slotSize;
+ int spaceUsed = leafFrame.getBuffer().capacity() - leafFrame.getTotalFreeSpace();
+
+ // try to free space by compression
+ if (spaceUsed + spaceNeeded > leafMaxBytes) {
+ leafFrame.compress();
+ spaceUsed = leafFrame.getBuffer().capacity() - leafFrame.getTotalFreeSpace();
+ }
+ //full, allocate new page
+ if (spaceUsed + spaceNeeded > leafMaxBytes) {
+ if (leafFrame.getTupleCount() == 0) {
+ //The current page is empty. Return it.
+ bufferCache.returnPage(leafFrontier.page, false);
+ } else {
+ leafFrontier.lastTuple.resetByTupleIndex(leafFrame, leafFrame.getTupleCount() - 1);
+ if (verifyInput) {
+ verifyInputTuple(tuple, leafFrontier.lastTuple);
+ }
+ //The current page is not empty. Write it.
+ writeFullLeafPage();
+ }
+ if (tupleSize > maxTupleSize) {
+ //We need a large page
+ final long dpid = BufferedFileHandle.getDiskPageId(fileId, leafFrontier.pageId);
+ // calculate required number of pages.
+ int headerSize = Math.max(leafFrame.getPageHeaderSize(), interiorFrame.getPageHeaderSize());
+ final int multiplier =
+ (int) Math.ceil((double) tupleSize / (bufferCache.getPageSize() - headerSize));
+ if (multiplier > 1) {
+ leafFrontier.page = bufferCache.confiscateLargePage(dpid, multiplier,
+ freePageManager.takeBlock(metaFrame, multiplier - 1));
+ } else {
+ leafFrontier.page = bufferCache.confiscatePage(dpid);
+ }
+ leafFrame.setPage(leafFrontier.page);
+ leafFrame.initBuffer((byte) 0);
+ ((IBTreeLeafFrame) leafFrame).setLargeFlag(true);
+ } else {
+ //allocate a new page
+ confiscateNewLeafPage();
+ }
+ } else {
+ if (verifyInput && leafFrame.getTupleCount() > 0) {
+ leafFrontier.lastTuple.resetByTupleIndex(leafFrame, leafFrame.getTupleCount() - 1);
+ verifyInputTuple(tuple, leafFrontier.lastTuple);
+ }
+ }
+ ((IBTreeLeafFrame) leafFrame).insertSorted(tuple);
+ } catch (HyracksDataException | RuntimeException e) {
+ logState(tuple, e);
+ handleException();
+ throw e;
+ }
+ }
+
+ protected void verifyInputTuple(ITupleReference tuple, ITupleReference prevTuple) throws HyracksDataException {
+ // New tuple should be strictly greater than last tuple.
+ int cmpResult = cmp.compare(tuple, prevTuple);
+ if (cmpResult < 0) {
+ throw HyracksDataException.create(ErrorCode.UNSORTED_LOAD_INPUT);
+ }
+ if (cmpResult == 0) {
+ throw HyracksDataException.create(ErrorCode.DUPLICATE_LOAD_INPUT);
+ }
+ }
+
+ protected void propagateBulk(int level, List<ICachedPage> pagesToWrite) throws HyracksDataException {
+ if (splitKey.getBuffer() == null) {
+ return;
+ }
+
+ if (level >= nodeFrontiers.size()) {
+ addLevel();
+ }
+
+ NodeFrontier frontier = nodeFrontiers.get(level);
+ interiorFrame.setPage(frontier.page);
+
+ ITupleReference tuple = splitKey.getTuple();
+ int tupleBytes = tupleWriter.bytesRequired(tuple, 0, cmp.getKeyFieldCount());
+ int spaceNeeded = tupleBytes + slotSize + 4;
+ if (tupleBytes > interiorFrame.getMaxTupleSize(bufferCache.getPageSize())) {
+ throw HyracksDataException.create(ErrorCode.RECORD_IS_TOO_LARGE, tupleBytes,
+ interiorFrame.getMaxTupleSize(bufferCache.getPageSize()));
+ }
+
+ int spaceUsed = interiorFrame.getBuffer().capacity() - interiorFrame.getTotalFreeSpace();
+ if (spaceUsed + spaceNeeded > interiorMaxBytes) {
+ ISplitKey copyKey = splitKey.duplicate(tupleWriter.createTupleReference());
+ tuple = copyKey.getTuple();
+
+ frontier.lastTuple.resetByTupleIndex(interiorFrame, interiorFrame.getTupleCount() - 1);
+ int splitKeySize = tupleWriter.bytesRequired(frontier.lastTuple, 0, cmp.getKeyFieldCount());
+ splitKey.initData(splitKeySize);
+ tupleWriter.writeTupleFields(frontier.lastTuple, 0, cmp.getKeyFieldCount(), splitKey.getBuffer().array(),
+ 0);
+ splitKey.getTuple().resetByTupleOffset(splitKey.getBuffer().array(), 0);
+
+ ((IBTreeInteriorFrame) interiorFrame).deleteGreatest();
+ int finalPageId = freePageManager.takePage(metaFrame);
+ frontier.page.setDiskPageId(BufferedFileHandle.getDiskPageId(fileId, finalPageId));
+ pagesToWrite.add(frontier.page);
+ splitKey.setLeftPage(finalPageId);
+
+ propagateBulk(level + 1, pagesToWrite);
+ frontier.page = bufferCache.confiscatePage(IBufferCache.INVALID_DPID);
+ interiorFrame.setPage(frontier.page);
+ interiorFrame.initBuffer((byte) level);
+ }
+ ((IBTreeInteriorFrame) interiorFrame).insertSorted(tuple);
+ }
+
+ private void persistFrontiers(int level, int rightPage) throws HyracksDataException {
+ if (level >= nodeFrontiers.size()) {
+ setRootPageId(nodeFrontiers.get(level - 1).pageId);
+ releasedLatches = true;
+ return;
+ }
+ if (level < 1) {
+ ICachedPage lastLeaf = nodeFrontiers.get(level).page;
+ int lastLeafPage = nodeFrontiers.get(level).pageId;
+ lastLeaf.setDiskPageId(BufferedFileHandle.getDiskPageId(fileId, nodeFrontiers.get(level).pageId));
+ writeLastLeaf(lastLeaf);
+ nodeFrontiers.get(level).page = null;
+ persistFrontiers(level + 1, lastLeafPage);
+ return;
+ }
+ NodeFrontier frontier = nodeFrontiers.get(level);
+ interiorFrame.setPage(frontier.page);
+ //just finalize = the layer right above the leaves has correct righthand pointers already
+ if (rightPage < 0) {
+ throw new HyracksDataException("Error in index creation. Internal node appears to have no rightmost guide");
+ }
+ ((IBTreeInteriorFrame) interiorFrame).setRightmostChildPageId(rightPage);
+ int finalPageId = freePageManager.takePage(metaFrame);
+ frontier.page.setDiskPageId(BufferedFileHandle.getDiskPageId(fileId, finalPageId));
+ write(frontier.page);
+ frontier.pageId = finalPageId;
+ persistFrontiers(level + 1, finalPageId);
+ }
+
+ @Override
+ public void end() throws HyracksDataException {
+ try {
+ persistFrontiers(0, -1);
+ super.end();
+ } catch (HyracksDataException | RuntimeException e) {
+ handleException();
+ throw e;
+ }
+ }
+
+ @Override
+ public void abort() throws HyracksDataException {
+ super.handleException();
+ }
+
+ protected void writeFullLeafPage() throws HyracksDataException {
+ final NodeFrontier leafFrontier = nodeFrontiers.get(0);
+ leafFrontier.lastTuple.resetByTupleIndex(leafFrame, leafFrame.getTupleCount() - 1);
+ final int splitKeySize = tupleWriter.bytesRequired(leafFrontier.lastTuple, 0, cmp.getKeyFieldCount());
+ splitKey.initData(splitKeySize);
+ tupleWriter.writeTupleFields(leafFrontier.lastTuple, 0, cmp.getKeyFieldCount(), splitKey.getBuffer().array(),
+ 0);
+ splitKey.getTuple().resetByTupleOffset(splitKey.getBuffer().array(), 0);
+ splitKey.setLeftPage(leafFrontier.pageId);
+
+ propagateBulk(1, pagesToWrite);
+
+ leafFrontier.pageId = freePageManager.takePage(metaFrame);
+
+ ((IBTreeLeafFrame) leafFrame).setNextLeaf(leafFrontier.pageId);
+
+ write(leafFrontier.page);
+ for (ICachedPage c : pagesToWrite) {
+ write(c);
+ }
+ pagesToWrite.clear();
+ splitKey.setRightPage(leafFrontier.pageId);
+ }
+
+ protected void writeLastLeaf(ICachedPage page) throws HyracksDataException {
+ write(page);
+ }
+
+ protected final void confiscateNewLeafPage() throws HyracksDataException {
+ final NodeFrontier leafFrontier = nodeFrontiers.get(0);
+ final long dpid = BufferedFileHandle.getDiskPageId(fileId, leafFrontier.pageId);
+ leafFrontier.page = bufferCache.confiscatePage(dpid);
+ leafFrame.setPage(leafFrontier.page);
+ leafFrame.initBuffer((byte) 0);
+ }
+
+ private void logState(ITupleReference tuple, Exception e) {
+ try {
+ ObjectNode state = JSONUtil.createObject();
+ state.set("leafFrame", leafFrame.getState());
+ state.set("interiorFrame", interiorFrame.getState());
+ int tupleSize = Math.max(leafFrame.getBytesRequiredToWriteTuple(tuple),
+ interiorFrame.getBytesRequiredToWriteTuple(tuple));
+ state.put("tupleSize", tupleSize);
+ state.put("spaceNeeded", tupleWriter.bytesRequired(tuple) + slotSize);
+ state.put("spaceUsed", leafFrame.getBuffer().capacity() - leafFrame.getTotalFreeSpace());
+ state.put("leafMaxBytes", leafMaxBytes);
+ state.put("maxTupleSize", maxTupleSize);
+ LOGGER.error("failed to add tuple {}", state, e);
+ } catch (Throwable t) {
+ e.addSuppressed(t);
+ }
+ }
+}
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTree.java
index ae6bbaa..179f1da 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTree.java
@@ -24,8 +24,8 @@
import org.apache.hyracks.api.io.FileReference;
import org.apache.hyracks.api.util.HyracksConstants;
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
-import org.apache.hyracks.storage.am.btree.api.IBTreeFrame;
import org.apache.hyracks.storage.am.btree.api.IBTreeLeafFrame;
+import org.apache.hyracks.storage.am.btree.api.IDiskBTreeStatefulPointSearchCursor;
import org.apache.hyracks.storage.am.common.api.IPageManager;
import org.apache.hyracks.storage.am.common.api.ITreeIndexCursor;
import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
@@ -38,7 +38,6 @@
import org.apache.hyracks.storage.common.ISearchPredicate;
import org.apache.hyracks.storage.common.MultiComparator;
import org.apache.hyracks.storage.common.NoOpIndexCursorStats;
-import org.apache.hyracks.storage.common.buffercache.BufferCache;
import org.apache.hyracks.storage.common.buffercache.IBufferCache;
import org.apache.hyracks.storage.common.buffercache.ICachedPage;
import org.apache.hyracks.storage.common.file.BufferedFileHandle;
@@ -76,7 +75,8 @@
private void search(ITreeIndexCursor cursor, ISearchPredicate searchPred, BTreeOpContext ctx)
throws HyracksDataException {
ctx.reset();
- ctx.setPred((RangePredicate) searchPred);
+ RangePredicate rangePredicate = (RangePredicate) searchPred;
+ ctx.setPred(rangePredicate);
ctx.setCursor(cursor);
if (ctx.getPred().getLowKeyComparator() == null) {
ctx.getPred().setLowKeyComparator(ctx.getCmp());
@@ -87,25 +87,17 @@
cursor.setBufferCache(bufferCache);
cursor.setFileId(getFileId());
- if (cursor instanceof DiskBTreePointSearchCursor) {
- DiskBTreePointSearchCursor pointCursor = (DiskBTreePointSearchCursor) cursor;
+ if (cursor instanceof IDiskBTreeStatefulPointSearchCursor) {
+ IDiskBTreeStatefulPointSearchCursor pointCursor = (IDiskBTreeStatefulPointSearchCursor) cursor;
int lastPageId = pointCursor.getLastPageId();
- if (lastPageId != BufferCache.INVALID_PAGEID) {
- // check whether the last leaf page contains this key
- ICachedPage lastPage =
- bufferCache.pin(BufferedFileHandle.getDiskPageId(getFileId(), lastPageId), false);
- ctx.getLeafFrame().setPage(lastPage);
- if (fitInPage(ctx.getPred().getLowKey(), ctx.getPred().getLowKeyComparator(), ctx.getLeafFrame())) {
- // use this page
- ctx.getCursorInitialState().setPage(lastPage);
- ctx.getCursorInitialState().setPageId(lastPageId);
- pointCursor.open(ctx.getCursorInitialState(), searchPred);
+ if (lastPageId != IBufferCache.INVALID_PAGEID) {
+ if (fitInPage(ctx.getPred().getLowKey(), ctx.getPred().getLowKeyComparator(), pointCursor.getFrame())) {
+ pointCursor.setCursorToNextKey(searchPred);
return;
} else {
- // release the last page and clear the states of this cursor
+ // release the last page, clear the states of this cursor, and close the cursor
// then retry the search from root to leaf
- bufferCache.unpin(lastPage);
- pointCursor.clearSearchState();
+ cursor.close();
}
}
}
@@ -113,7 +105,7 @@
searchDown(rootNode, rootPage, ctx, cursor);
}
- private boolean fitInPage(ITupleReference key, MultiComparator comparator, IBTreeFrame frame)
+ private boolean fitInPage(ITupleReference key, MultiComparator comparator, ITreeIndexFrame frame)
throws HyracksDataException {
// assume that search keys are sorted (non-decreasing)
ITupleReference rightmostTuple = frame.getRightmostTuple();
@@ -191,14 +183,14 @@
}
@Override
- public DiskBTreeRangeSearchCursor createSearchCursor(boolean exclusive) {
+ public ITreeIndexCursor createSearchCursor(boolean exclusive) {
IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) btree.getLeafFrameFactory().createFrame();
return new DiskBTreeRangeSearchCursor(leafFrame, exclusive, (IIndexCursorStats) iap.getParameters()
.getOrDefault(HyracksConstants.INDEX_CURSOR_STATS, NoOpIndexCursorStats.INSTANCE));
}
@Override
- public BTreeRangeSearchCursor createPointCursor(boolean exclusive, boolean stateful) {
+ public ITreeIndexCursor createPointCursor(boolean exclusive, boolean stateful) {
IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) btree.getLeafFrameFactory().createFrame();
return new DiskBTreePointSearchCursor(leafFrame, exclusive, stateful);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTreePointSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTreePointSearchCursor.java
index 1bf3ecf..8fd9a96 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTreePointSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTreePointSearchCursor.java
@@ -21,13 +21,16 @@
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.storage.am.btree.api.IBTreeLeafFrame;
+import org.apache.hyracks.storage.am.btree.api.IDiskBTreeStatefulPointSearchCursor;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
import org.apache.hyracks.storage.am.common.ophelpers.FindTupleMode;
import org.apache.hyracks.storage.am.common.ophelpers.FindTupleNoExactMatchPolicy;
import org.apache.hyracks.storage.common.ICursorInitialState;
import org.apache.hyracks.storage.common.ISearchPredicate;
-import org.apache.hyracks.storage.common.buffercache.BufferCache;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
-public class DiskBTreePointSearchCursor extends DiskBTreeRangeSearchCursor {
+public class DiskBTreePointSearchCursor extends DiskBTreeRangeSearchCursor
+ implements IDiskBTreeStatefulPointSearchCursor {
/**
* A stateful cursor keeps the search state (last search page Id + index) across multiple searches
* until {@link #clearSearchState()} is called explicity
@@ -36,16 +39,14 @@
private boolean nextHasBeenCalled;
- private int lastPageId = BufferCache.INVALID_PAGEID;
+ private int lastPageId;
private int lastTupleIndex = 0;
public DiskBTreePointSearchCursor(IBTreeLeafFrame frame, boolean exclusiveLatchNodes, boolean stateful) {
super(frame, exclusiveLatchNodes);
this.stateful = stateful;
- }
-
- public DiskBTreePointSearchCursor(IBTreeLeafFrame frame, boolean exclusiveLatchNodes) {
- this(frame, exclusiveLatchNodes, false);
+ lastPageId = IBufferCache.INVALID_PAGEID;
+ lastTupleIndex = 0;
}
@Override
@@ -59,6 +60,12 @@
}
@Override
+ public void doClose() throws HyracksDataException {
+ clearSearchState();
+ super.doClose();
+ }
+
+ @Override
public void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
// in case open is called multiple times without closing
if (page != null) {
@@ -71,30 +78,10 @@
page = initialState.getPage();
isPageDirty = false;
frame.setPage(page);
-
- pred = (RangePredicate) searchPred;
- lowKeyCmp = pred.getLowKeyComparator();
- lowKey = pred.getLowKey();
-
- reusablePredicate.setLowKeyComparator(originalKeyCmp);
-
- lowKeyFtm = FindTupleMode.EXACT;
- lowKeyFtp = FindTupleNoExactMatchPolicy.NONE;
-
- nextHasBeenCalled = false;
-
- // only get the low key position
- tupleIndex = getLowKeyIndex();
- if (stateful) {
- lastPageId = pageId;
- if (tupleIndex >= 0) {
- lastTupleIndex = tupleIndex;
- } else {
- lastTupleIndex = -tupleIndex - 1;
- }
- }
+ setCursorToNextKey(searchPred);
}
+ @Override
public int getLastPageId() {
return lastPageId;
}
@@ -108,9 +95,38 @@
}
}
- public void clearSearchState() {
- this.lastPageId = BufferCache.INVALID_PAGEID;
+ @Override
+ public void setCursorToNextKey(ISearchPredicate searchPred) throws HyracksDataException {
+ pred = (RangePredicate) searchPred;
+ lowKeyCmp = pred.getLowKeyComparator();
+ lowKey = pred.getLowKey();
+
+ reusablePredicate.setLowKeyComparator(originalKeyCmp);
+
+ lowKeyFtm = FindTupleMode.EXACT;
+ lowKeyFtp = FindTupleNoExactMatchPolicy.NONE;
+
+ nextHasBeenCalled = false;
+
+ // only get the lowKey position
+ tupleIndex = getLowKeyIndex();
+ if (stateful) {
+ lastPageId = pageId;
+ if (tupleIndex >= 0) {
+ lastTupleIndex = tupleIndex;
+ } else {
+ lastTupleIndex = -tupleIndex - 1;
+ }
+ }
+ }
+
+ private void clearSearchState() {
+ this.lastPageId = IBufferCache.INVALID_PAGEID;
this.lastTupleIndex = 0;
}
+ @Override
+ public ITreeIndexFrame getFrame() {
+ return frame;
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
index eab8c96..1fe91d9 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
@@ -44,6 +44,7 @@
import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
import org.apache.hyracks.storage.am.common.api.ITupleFilter;
import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
+import org.apache.hyracks.storage.am.common.impls.DefaultTupleProjectorFactory;
import org.apache.hyracks.storage.am.common.impls.IndexAccessParameters;
import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
import org.apache.hyracks.storage.am.common.tuples.ReferenceFrameTupleReference;
@@ -54,6 +55,8 @@
import org.apache.hyracks.storage.common.IIndexCursor;
import org.apache.hyracks.storage.common.ISearchOperationCallback;
import org.apache.hyracks.storage.common.ISearchPredicate;
+import org.apache.hyracks.storage.common.projection.ITupleProjector;
+import org.apache.hyracks.storage.common.projection.ITupleProjectorFactory;
import org.apache.hyracks.util.IThreadStatsCollector;
import org.apache.logging.log4j.Level;
import org.apache.logging.log4j.LogManager;
@@ -107,6 +110,7 @@
protected final long outputLimit;
protected long outputCount = 0;
protected boolean finished;
+ protected final ITupleProjector tupleProjector;
// no filter and limit pushdown
public IndexSearchOperatorNodePushable(IHyracksTaskContext ctx, RecordDescriptor inputRecDesc, int partition,
@@ -116,7 +120,7 @@
IMissingWriterFactory nonFilterWriterFactory) throws HyracksDataException {
this(ctx, inputRecDesc, partition, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
retainInput, retainMissing, nonMatchWriterFactory, searchCallbackFactory, appendIndexFilter,
- nonFilterWriterFactory, null, -1, false, null, null);
+ nonFilterWriterFactory, null, -1, false, null, null, DefaultTupleProjectorFactory.INSTANCE);
}
public IndexSearchOperatorNodePushable(IHyracksTaskContext ctx, RecordDescriptor inputRecDesc, int partition,
@@ -125,7 +129,8 @@
ISearchOperationCallbackFactory searchCallbackFactory, boolean appendIndexFilter,
IMissingWriterFactory nonFilterWriterFactory, ITupleFilterFactory tupleFactoryFactory, long outputLimit,
boolean appendSearchCallbackProceedResult, byte[] searchCallbackProceedResultFalseValue,
- byte[] searchCallbackProceedResultTrueValue) throws HyracksDataException {
+ byte[] searchCallbackProceedResultTrueValue, ITupleProjectorFactory projectorFactory)
+ throws HyracksDataException {
this.ctx = ctx;
this.indexHelper = indexHelperFactory.create(ctx.getJobletContext().getServiceContext(), partition);
this.retainInput = retainInput;
@@ -162,6 +167,8 @@
if (this.tupleFilterFactory != null && this.retainMissing) {
throw new IllegalStateException("RetainMissing with tuple filter is not supported");
}
+
+ tupleProjector = projectorFactory.createTupleProjector(ctx);
}
protected abstract ISearchPredicate createSearchPredicate();
@@ -349,10 +356,7 @@
protected void writeTupleToOutput(ITupleReference tuple) throws IOException {
try {
- for (int i = 0; i < tuple.getFieldCount(); i++) {
- dos.write(tuple.getFieldData(i), tuple.getFieldStart(i), tuple.getFieldLength(i));
- tb.addFieldEndOffset();
- }
+ tupleProjector.project(tuple, dos, tb);
} catch (Exception e) {
throw e;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndex.java
index 81e528b..11368bf 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndex.java
@@ -19,9 +19,6 @@
package org.apache.hyracks.storage.am.common.impls;
-import java.util.ArrayList;
-import java.util.List;
-
import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import org.apache.hyracks.api.exceptions.ErrorCode;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -30,17 +27,9 @@
import org.apache.hyracks.storage.am.common.api.ITreeIndex;
import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
-import org.apache.hyracks.storage.am.common.api.ITreeIndexMetadataFrame;
-import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
-import org.apache.hyracks.storage.common.IIndexBulkLoader;
-import org.apache.hyracks.storage.common.MultiComparator;
import org.apache.hyracks.storage.common.buffercache.HaltOnFailureCallback;
import org.apache.hyracks.storage.common.buffercache.IBufferCache;
import org.apache.hyracks.storage.common.buffercache.ICachedPage;
-import org.apache.hyracks.storage.common.buffercache.IFIFOPageWriter;
-import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
-import org.apache.hyracks.storage.common.buffercache.PageWriteFailureCallback;
-import org.apache.hyracks.storage.common.compression.file.ICompressedPageWriter;
import org.apache.hyracks.storage.common.file.BufferedFileHandle;
public abstract class AbstractTreeIndex implements ITreeIndex {
@@ -229,118 +218,6 @@
return fieldCount;
}
- public abstract class AbstractTreeIndexBulkLoader extends PageWriteFailureCallback implements IIndexBulkLoader {
- protected final MultiComparator cmp;
- protected final int slotSize;
- protected final int leafMaxBytes;
- protected final int interiorMaxBytes;
- protected final ArrayList<NodeFrontier> nodeFrontiers = new ArrayList<>();
- protected final ITreeIndexMetadataFrame metaFrame;
- protected final ITreeIndexTupleWriter tupleWriter;
- protected ITreeIndexFrame leafFrame;
- protected ITreeIndexFrame interiorFrame;
- // Immutable bulk loaders write their root page at page -2, as needed e.g. by append-only file systems such as
- // HDFS. Since loading this tree relies on the root page actually being at that point, no further inserts into
- // that tree are allowed. Currently, this is not enforced.
- protected boolean releasedLatches;
- private final IFIFOPageWriter pageWriter;
- protected List<ICachedPage> pagesToWrite;
- private final ICompressedPageWriter compressedPageWriter;
-
- public AbstractTreeIndexBulkLoader(float fillFactor, IPageWriteCallback callback) throws HyracksDataException {
- leafFrame = leafFrameFactory.createFrame();
- interiorFrame = interiorFrameFactory.createFrame();
- metaFrame = freePageManager.createMetadataFrame();
-
- pageWriter = bufferCache.createFIFOWriter(callback, this);
-
- if (!isEmptyTree(leafFrame)) {
- throw HyracksDataException.create(ErrorCode.CANNOT_BULK_LOAD_NON_EMPTY_TREE);
- }
-
- this.cmp = MultiComparator.create(cmpFactories);
-
- leafFrame.setMultiComparator(cmp);
- interiorFrame.setMultiComparator(cmp);
-
- tupleWriter = leafFrame.getTupleWriter();
-
- NodeFrontier leafFrontier = new NodeFrontier(leafFrame.createTupleReference());
- leafFrontier.pageId = freePageManager.takePage(metaFrame);
- leafFrontier.page =
- bufferCache.confiscatePage(BufferedFileHandle.getDiskPageId(fileId, leafFrontier.pageId));
-
- interiorFrame.setPage(leafFrontier.page);
- interiorFrame.initBuffer((byte) 0);
- interiorMaxBytes = (int) (interiorFrame.getBuffer().capacity() * fillFactor);
-
- leafFrame.setPage(leafFrontier.page);
- leafFrame.initBuffer((byte) 0);
- leafMaxBytes = (int) (leafFrame.getBuffer().capacity() * fillFactor);
- slotSize = leafFrame.getSlotSize();
-
- nodeFrontiers.add(leafFrontier);
- pagesToWrite = new ArrayList<>();
- compressedPageWriter = bufferCache.getCompressedPageWriter(fileId);
- }
-
- protected void handleException() {
- // Unlatch and unpin pages that weren't in the queue to avoid leaking memory.
- compressedPageWriter.abort();
- for (NodeFrontier nodeFrontier : nodeFrontiers) {
- if (nodeFrontier != null && nodeFrontier.page != null) {
- ICachedPage frontierPage = nodeFrontier.page;
- if (frontierPage.confiscated()) {
- bufferCache.returnPage(frontierPage, false);
- }
- }
- }
- for (ICachedPage pageToDiscard : pagesToWrite) {
- if (pageToDiscard != null) {
- bufferCache.returnPage(pageToDiscard, false);
- }
- }
- releasedLatches = true;
- }
-
- @Override
- public void end() throws HyracksDataException {
- if (hasFailed()) {
- throw HyracksDataException.create(getFailure());
- }
- freePageManager.setRootPageId(rootPage);
- }
-
- protected void addLevel() throws HyracksDataException {
- NodeFrontier frontier = new NodeFrontier(tupleWriter.createTupleReference());
- frontier.page = bufferCache.confiscatePage(IBufferCache.INVALID_DPID);
- frontier.pageId = -1;
- frontier.lastTuple.setFieldCount(cmp.getKeyFieldCount());
- interiorFrame.setPage(frontier.page);
- interiorFrame.initBuffer((byte) nodeFrontiers.size());
- nodeFrontiers.add(frontier);
- }
-
- public ITreeIndexFrame getLeafFrame() {
- return leafFrame;
- }
-
- public void setLeafFrame(ITreeIndexFrame leafFrame) {
- this.leafFrame = leafFrame;
- }
-
- public void write(ICachedPage cPage) throws HyracksDataException {
- compressedPageWriter.prepareWrite(cPage);
- pageWriter.write(cPage);
- }
-
- @Override
- public void force() throws HyracksDataException {
- bufferCache.force(fileId, false);
- }
-
- }
-
public IBinaryComparatorFactory[] getCmpFactories() {
return cmpFactories;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndexBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndexBulkLoader.java
new file mode 100644
index 0000000..45a88a7
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndexBulkLoader.java
@@ -0,0 +1,171 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.common.impls;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hyracks.api.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.common.api.IPageManager;
+import org.apache.hyracks.storage.am.common.api.ITreeIndex;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexMetadataFrame;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleReference;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
+import org.apache.hyracks.storage.common.IIndexBulkLoader;
+import org.apache.hyracks.storage.common.MultiComparator;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.storage.common.buffercache.ICachedPage;
+import org.apache.hyracks.storage.common.buffercache.IFIFOPageWriter;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
+import org.apache.hyracks.storage.common.buffercache.PageWriteFailureCallback;
+import org.apache.hyracks.storage.common.compression.file.ICompressedPageWriter;
+import org.apache.hyracks.storage.common.file.BufferedFileHandle;
+
+public abstract class AbstractTreeIndexBulkLoader extends PageWriteFailureCallback implements IIndexBulkLoader {
+ protected final IBufferCache bufferCache;
+ protected final IPageManager freePageManager;
+ protected final AbstractTreeIndex treeIndex;
+ protected final int fileId;
+ protected final MultiComparator cmp;
+ protected final int slotSize;
+ protected final int leafMaxBytes;
+ protected final int interiorMaxBytes;
+ protected final ArrayList<NodeFrontier> nodeFrontiers = new ArrayList<>();
+ protected final ITreeIndexMetadataFrame metaFrame;
+ protected final ITreeIndexTupleWriter tupleWriter;
+ protected ITreeIndexFrame leafFrame;
+ protected ITreeIndexFrame interiorFrame;
+ // Immutable bulk loaders write their root page at page -2, as needed e.g. by append-only file systems such as
+ // HDFS. Since loading this tree relies on the root page actually being at that point, no further inserts into
+ // that tree are allowed. Currently, this is not enforced.
+ protected boolean releasedLatches;
+ private final IFIFOPageWriter pageWriter;
+ protected List<ICachedPage> pagesToWrite;
+ private final ICompressedPageWriter compressedPageWriter;
+
+ protected AbstractTreeIndexBulkLoader(float fillFactor, IPageWriteCallback callback, ITreeIndex index)
+ throws HyracksDataException {
+ this(fillFactor, callback, index, index.getLeafFrameFactory().createFrame());
+ }
+
+ protected AbstractTreeIndexBulkLoader(float fillFactor, IPageWriteCallback callback, ITreeIndex index,
+ ITreeIndexFrame leafFrame) throws HyracksDataException {
+ this.bufferCache = index.getBufferCache();
+ this.freePageManager = index.getPageManager();
+ this.fileId = index.getFileId();
+ this.treeIndex = (AbstractTreeIndex) index;
+ this.leafFrame = leafFrame;
+ interiorFrame = treeIndex.getInteriorFrameFactory().createFrame();
+ metaFrame = freePageManager.createMetadataFrame();
+
+ pageWriter = bufferCache.createFIFOWriter(callback, this);
+
+ if (!treeIndex.isEmptyTree(leafFrame)) {
+ throw HyracksDataException.create(ErrorCode.CANNOT_BULK_LOAD_NON_EMPTY_TREE);
+ }
+
+ this.cmp = MultiComparator.create(treeIndex.getCmpFactories());
+
+ leafFrame.setMultiComparator(cmp);
+ interiorFrame.setMultiComparator(cmp);
+
+ tupleWriter = leafFrame.getTupleWriter();
+ NodeFrontier leafFrontier = new NodeFrontier(createTupleReference());
+ leafFrontier.pageId = freePageManager.takePage(metaFrame);
+ leafFrontier.page = bufferCache.confiscatePage(BufferedFileHandle.getDiskPageId(fileId, leafFrontier.pageId));
+
+ interiorFrame.setPage(leafFrontier.page);
+ interiorFrame.initBuffer((byte) 0);
+ interiorMaxBytes = (int) (interiorFrame.getBuffer().capacity() * fillFactor);
+
+ leafFrame.setPage(leafFrontier.page);
+ leafFrame.initBuffer((byte) 0);
+ leafMaxBytes = (int) (leafFrame.getBuffer().capacity() * fillFactor);
+ slotSize = leafFrame.getSlotSize();
+
+ nodeFrontiers.add(leafFrontier);
+ pagesToWrite = new ArrayList<>();
+ compressedPageWriter = bufferCache.getCompressedPageWriter(fileId);
+ }
+
+ protected ITreeIndexTupleReference createTupleReference() {
+ return leafFrame.createTupleReference();
+ }
+
+ protected void handleException() {
+ // Unlatch and unpin pages that weren't in the queue to avoid leaking memory.
+ compressedPageWriter.abort();
+ for (NodeFrontier nodeFrontier : nodeFrontiers) {
+ if (nodeFrontier != null && nodeFrontier.page != null) {
+ ICachedPage frontierPage = nodeFrontier.page;
+ if (frontierPage.confiscated()) {
+ bufferCache.returnPage(frontierPage, false);
+ }
+ }
+ }
+ for (ICachedPage pageToDiscard : pagesToWrite) {
+ if (pageToDiscard != null) {
+ bufferCache.returnPage(pageToDiscard, false);
+ }
+ }
+ releasedLatches = true;
+ }
+
+ @Override
+ public void end() throws HyracksDataException {
+ if (hasFailed()) {
+ throw HyracksDataException.create(getFailure());
+ }
+ freePageManager.setRootPageId(treeIndex.getRootPageId());
+ }
+
+ protected void setRootPageId(int rootPage) {
+ treeIndex.rootPage = rootPage;
+ }
+
+ protected void addLevel() throws HyracksDataException {
+ NodeFrontier frontier = new NodeFrontier(tupleWriter.createTupleReference());
+ frontier.page = bufferCache.confiscatePage(IBufferCache.INVALID_DPID);
+ frontier.pageId = -1;
+ frontier.lastTuple.setFieldCount(cmp.getKeyFieldCount());
+ interiorFrame.setPage(frontier.page);
+ interiorFrame.initBuffer((byte) nodeFrontiers.size());
+ nodeFrontiers.add(frontier);
+ }
+
+ public ITreeIndexFrame getLeafFrame() {
+ return leafFrame;
+ }
+
+ public void setLeafFrame(ITreeIndexFrame leafFrame) {
+ this.leafFrame = leafFrame;
+ }
+
+ public void write(ICachedPage cPage) throws HyracksDataException {
+ compressedPageWriter.prepareWrite(cPage);
+ pageWriter.write(cPage);
+ }
+
+ @Override
+ public void force() throws HyracksDataException {
+ bufferCache.force(fileId, false);
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/DefaultTupleProjector.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/DefaultTupleProjector.java
new file mode 100644
index 0000000..00cb0c5
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/DefaultTupleProjector.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.common.impls;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.common.projection.ITupleProjector;
+
+class DefaultTupleProjector implements ITupleProjector {
+ public static final ITupleProjector INSTANCE = new DefaultTupleProjector();
+
+ private DefaultTupleProjector() {
+ }
+
+ @Override
+ public void project(ITupleReference tuple, DataOutput dos, ArrayTupleBuilder tb) throws IOException {
+ for (int i = 0; i < tuple.getFieldCount(); i++) {
+ dos.write(tuple.getFieldData(i), tuple.getFieldStart(i), tuple.getFieldLength(i));
+ tb.addFieldEndOffset();
+ }
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/DefaultTupleProjectorFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/DefaultTupleProjectorFactory.java
new file mode 100644
index 0000000..092982d
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/DefaultTupleProjectorFactory.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.common.impls;
+
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.common.projection.ITupleProjector;
+import org.apache.hyracks.storage.common.projection.ITupleProjectorFactory;
+
+public class DefaultTupleProjectorFactory implements ITupleProjectorFactory {
+ private static final long serialVersionUID = -4525893018744087821L;
+ public static final DefaultTupleProjectorFactory INSTANCE = new DefaultTupleProjectorFactory();
+
+ private DefaultTupleProjectorFactory() {
+ }
+
+ @Override
+ public ITupleProjector createTupleProjector(IHyracksTaskContext context) throws HyracksDataException {
+ return DefaultTupleProjector.INSTANCE;
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/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..9b8c353 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorNodePushable.java
@@ -34,10 +34,12 @@
import org.apache.hyracks.storage.am.common.api.ITreeIndex;
import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTree;
import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeBatchPointSearchCursor;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
import org.apache.hyracks.storage.common.IIndexCursor;
import org.apache.hyracks.storage.common.ISearchPredicate;
+import org.apache.hyracks.storage.common.projection.ITupleProjectorFactory;
public class LSMBTreeBatchPointSearchOperatorNodePushable extends BTreeSearchOperatorNodePushable {
@@ -48,18 +50,19 @@
boolean highKeyInclusive, int[] minFilterKeyFields, int[] maxFilterKeyFields,
IIndexDataflowHelperFactory indexHelperFactory, boolean retainInput, boolean retainMissing,
IMissingWriterFactory missingWriterFactory, ISearchOperationCallbackFactory searchCallbackFactory,
- ITupleFilterFactory tupleFilterFactory, long outputLimit) throws HyracksDataException {
+ ITupleFilterFactory tupleFilterFactory, long outputLimit, ITupleProjectorFactory tupleProjectorFactory)
+ throws HyracksDataException {
super(ctx, partition, inputRecDesc, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive,
minFilterKeyFields, maxFilterKeyFields, indexHelperFactory, retainInput, retainMissing,
missingWriterFactory, searchCallbackFactory, false, null, tupleFilterFactory, outputLimit, false, null,
- null);
+ null, tupleProjectorFactory);
this.keyFields = lowKeyFields;
}
@Override
protected IIndexCursor createCursor() throws HyracksDataException {
ILSMIndexAccessor lsmAccessor = (ILSMIndexAccessor) indexAccessor;
- return new LSMBTreeBatchPointSearchCursor(lsmAccessor.getOpContext());
+ return ((LSMBTree) index).createBatchPointSearchCursor(lsmAccessor.getOpContext());
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/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-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/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/projection/ITupleProjector.java
similarity index 66%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/projection/ITupleProjector.java
index 6095b26..8ca1a82 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/projection/ITupleProjector.java
@@ -16,9 +16,14 @@
* specific language governing permissions and limitations
* under the License.
*/
+package org.apache.hyracks.storage.common.projection;
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+import java.io.DataOutput;
+import java.io.IOException;
-select value v from range(1,2) v where v > ?;
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+public interface ITupleProjector {
+ void project(ITupleReference tuple, DataOutput dos, ArrayTupleBuilder tb) throws IOException;
+}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/projection/ITupleProjectorFactory.java
similarity index 61%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/projection/ITupleProjectorFactory.java
index 6095b26..ff9ecf9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/projection/ITupleProjectorFactory.java
@@ -16,9 +16,17 @@
* specific language governing permissions and limitations
* under the License.
*/
+package org.apache.hyracks.storage.common.projection;
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+import java.io.Serializable;
-select value v from range(1,2) v where v > ?;
+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/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;
}