[ASTERIXDB-3141][ASTERIXDB-3134] Allow querying columnar datasets
- user model changes: yes
- storage format changes: no
- interface changes: yes
Details:
This patch adds the ability to query columnar datasets.
Also, it teaches the compiler to read only the requested
columns. This patch also includes the ability to filter
mega-leaf nodes given a query predicate.
Interface changes:
- IMetadataProvider#getScannerRuntime()
* To allow projections for both data records and meta records
- IProjectionInfo
* Renamed to IProjectionFiltrationInfo
* Added getFilterExpression() for columnar filters
User model changes:
- After this change you can create columnar datasets
Example:
CREATE DATASET ExperDataset(ExperType)
PRIMARY KEY uid AUTOGENERATED
WITH {
"dataset-format":{"format":"column"}
};
- Added compiler property:
* compiler.column.filter
to enable/disable the usage of columnar filter
- Added storage properties:
* storage.column.max.tuple.count
An integer to tell the maximum number of
tuples to store per mega leaf node
* storage.column.free.space.tolerance
the percentage of tolerable empty space to
minimize column splitting
Change-Id: Ie9188bbd8463db22bf10c6871046c680528d5640
Reviewed-on: https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/17430
Integration-Tests: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: Wail Alkowaileet <wael.y.k@gmail.com>
Reviewed-by: Murtadha Hubail <mhubail@apache.org>
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 fed180b..a8dcb1f 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
@@ -21,14 +21,18 @@
import java.util.ArrayList;
import java.util.List;
+import org.apache.asterix.common.config.DatasetConfig;
import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.metadata.dataset.DatasetFormatInfo;
import org.apache.asterix.metadata.declared.DataSourceId;
import org.apache.asterix.metadata.declared.DataSourceIndex;
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.IndexUtil;
import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.optimizer.rules.am.BTreeJobGenParams;
import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -45,6 +49,7 @@
import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSourceIndex;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IProjectionFiltrationInfo;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractUnnestMapOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterUnnestMapOperator;
@@ -68,6 +73,7 @@
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;
+import org.apache.hyracks.storage.common.projection.ITupleProjectorFactory;
/**
* Contributes the runtime operator for an unnest-map representing a BTree search.
@@ -134,6 +140,7 @@
long outputLimit = -1;
boolean retainMissing = false;
IMissingWriterFactory nonMatchWriterFactory = null;
+ ITupleProjectorFactory tupleProjectorFactory = DefaultTupleProjectorFactory.INSTANCE;
switch (unnestMap.getOperatorTag()) {
case UNNEST_MAP:
UnnestMapOperator unnestMapOp = (UnnestMapOperator) unnestMap;
@@ -142,6 +149,15 @@
tupleFilterFactory = metadataProvider.createTupleFilterFactory(new IOperatorSchema[] { opSchema },
typeEnv, unnestMapOp.getSelectCondition().getValue(), context);
}
+ DatasetFormatInfo formatInfo = dataset.getDatasetFormatInfo();
+ if (isPrimaryIndex && formatInfo.getFormat() == DatasetConfig.DatasetFormat.COLUMN) {
+ IProjectionFiltrationInfo<?> projectionInfo = unnestMapOp.getDatasetProjectionInfo();
+ IProjectionFiltrationInfo<?> metaProjectionInfo = unnestMapOp.getMetaProjectionInfo();
+ ARecordType datasetType = (ARecordType) metadataProvider.findType(dataset);
+ ARecordType metaItemType = (ARecordType) metadataProvider.findMetaType(dataset);
+ tupleProjectorFactory = IndexUtil.createTupleProjectorFactory(formatInfo, projectionInfo,
+ metaProjectionInfo, datasetType, metaItemType, dataset.getPrimaryKeys().size());
+ }
break;
case LEFT_OUTER_UNNEST_MAP:
// By nature, LEFT_OUTER_UNNEST_MAP should generate missing (or null) values for non-matching tuples.
@@ -161,8 +177,7 @@
jobGenParams.isLowKeyInclusive(), jobGenParams.isHighKeyInclusive(), propagateFilter,
nonFilterWriterFactory, minFilterFieldIndexes, maxFilterFieldIndexes, tupleFilterFactory, outputLimit,
unnestMap.getGenerateCallBackProceedResultVar(),
- isPrimaryIndexPointSearch(op, context.getPhysicalOptimizationConfig()),
- DefaultTupleProjectorFactory.INSTANCE);
+ isPrimaryIndexPointSearch(op, context.getPhysicalOptimizationConfig()), tupleProjectorFactory);
IOperatorDescriptor opDesc = btreeSearch.first;
opDesc.setSourceLocation(unnestMap.getSourceLocation());
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 4665333..e6f4d65 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
@@ -85,11 +85,12 @@
CompilerProperties.COMPILER_SUBPLAN_NESTEDPUSHDOWN_KEY, CompilerProperties.COMPILER_ARRAYINDEX_KEY,
CompilerProperties.COMPILER_CBO_KEY, CompilerProperties.COMPILER_FORCE_JOIN_ORDER_KEY,
CompilerProperties.COMPILER_QUERY_PLAN_SHAPE_KEY, CompilerProperties.COMPILER_MIN_MEMORY_ALLOCATION_KEY,
- FunctionUtil.IMPORT_PRIVATE_FUNCTIONS, FuzzyUtils.SIM_FUNCTION_PROP_NAME,
- FuzzyUtils.SIM_THRESHOLD_PROP_NAME, StartFeedStatement.WAIT_FOR_COMPLETION,
- FeedActivityDetails.FEED_POLICY_NAME, FeedActivityDetails.COLLECT_LOCATIONS,
- SqlppQueryRewriter.INLINE_WITH_OPTION, SqlppExpressionToPlanTranslator.REWRITE_IN_AS_OR_OPTION,
- "hash_merge", "output-record-type", DisjunctivePredicateToJoinRule.REWRITE_OR_AS_JOIN_OPTION,
+ CompilerProperties.COMPILER_COLUMN_FILTER_KEY, FunctionUtil.IMPORT_PRIVATE_FUNCTIONS,
+ FuzzyUtils.SIM_FUNCTION_PROP_NAME, FuzzyUtils.SIM_THRESHOLD_PROP_NAME,
+ StartFeedStatement.WAIT_FOR_COMPLETION, FeedActivityDetails.FEED_POLICY_NAME,
+ FeedActivityDetails.COLLECT_LOCATIONS, SqlppQueryRewriter.INLINE_WITH_OPTION,
+ SqlppExpressionToPlanTranslator.REWRITE_IN_AS_OR_OPTION, "hash_merge", "output-record-type",
+ DisjunctivePredicateToJoinRule.REWRITE_OR_AS_JOIN_OPTION,
SetAsterixPhysicalOperatorsRule.REWRITE_ATTEMPT_BATCH_ASSIGN,
EquivalenceClassUtils.REWRITE_INTERNAL_QUERYUID_PK, SqlppQueryRewriter.SQL_COMPAT_OPTION));
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
index 1c3ef03..e63b865 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
@@ -79,7 +79,7 @@
import org.apache.asterix.optimizer.rules.PushLimitIntoPrimarySearchRule;
import org.apache.asterix.optimizer.rules.PushProperJoinThroughProduct;
import org.apache.asterix.optimizer.rules.PushSimilarityFunctionsBelowJoin;
-import org.apache.asterix.optimizer.rules.PushValueAccessToExternalDataScanRule;
+import org.apache.asterix.optimizer.rules.PushValueAccessToDataScanRule;
import org.apache.asterix.optimizer.rules.RemoveDuplicateFieldsRule;
import org.apache.asterix.optimizer.rules.RemoveLeftOuterUnnestForLeftOuterJoinRule;
import org.apache.asterix.optimizer.rules.RemoveOrReplaceDefaultNullCastRule;
@@ -172,7 +172,7 @@
return translationRules;
}
- public static final List<IAlgebraicRewriteRule> buildTypeInferenceRuleCollection() {
+ public static List<IAlgebraicRewriteRule> buildTypeInferenceRuleCollection() {
List<IAlgebraicRewriteRule> typeInfer = new LinkedList<>();
typeInfer.add(new InlineUnnestFunctionRule());
typeInfer.add(new InferTypesRule());
@@ -180,17 +180,17 @@
return typeInfer;
}
- public static final List<IAlgebraicRewriteRule> buildAutogenerateIDRuleCollection() {
+ public static List<IAlgebraicRewriteRule> buildAutogenerateIDRuleCollection() {
List<IAlgebraicRewriteRule> autogen = new LinkedList<>();
autogen.add(new IntroduceAutogenerateIDRule());
return autogen;
}
- public static final List<IAlgebraicRewriteRule> buildFulltextContainsRuleCollection() {
+ public static List<IAlgebraicRewriteRule> buildFulltextContainsRuleCollection() {
return Collections.singletonList(new FullTextContainsParameterCheckAndSetRule());
}
- public static final List<IAlgebraicRewriteRule> buildNormalizationRuleCollection(ICcApplicationContext appCtx) {
+ public static List<IAlgebraicRewriteRule> buildNormalizationRuleCollection(ICcApplicationContext appCtx) {
List<IAlgebraicRewriteRule> normalization = new LinkedList<>();
normalization.add(new CheckInsertUpsertReturningRule());
normalization.add(new IntroduceUnnestForCollectionToSequenceRule());
@@ -228,7 +228,7 @@
return normalization;
}
- public static final List<IAlgebraicRewriteRule> buildCondPushDownAndJoinInferenceRuleCollection() {
+ public static List<IAlgebraicRewriteRule> buildCondPushDownAndJoinInferenceRuleCollection() {
List<IAlgebraicRewriteRule> condPushDownAndJoinInference = new LinkedList<>();
condPushDownAndJoinInference.add(new PushSelectDownRule());
@@ -271,7 +271,7 @@
return condPushDownAndJoinInference;
}
- public static final List<IAlgebraicRewriteRule> buildLoadFieldsRuleCollection(ICcApplicationContext appCtx) {
+ public static List<IAlgebraicRewriteRule> buildLoadFieldsRuleCollection(ICcApplicationContext appCtx) {
List<IAlgebraicRewriteRule> fieldLoads = new LinkedList<>();
fieldLoads.add(new LoadRecordFieldsRule());
fieldLoads.add(new PushFieldAccessRule());
@@ -292,7 +292,7 @@
return fieldLoads;
}
- public static final List<IAlgebraicRewriteRule> buildConsolidationRuleCollection() {
+ public static List<IAlgebraicRewriteRule> buildConsolidationRuleCollection() {
List<IAlgebraicRewriteRule> consolidation = new LinkedList<>();
consolidation.add(new ConsolidateSelectsRule());
consolidation.add(new ConsolidateAssignsRule(false));
@@ -317,7 +317,7 @@
return consolidation;
}
- public static final List<IAlgebraicRewriteRule> buildAccessMethodRuleCollection() {
+ public static List<IAlgebraicRewriteRule> buildAccessMethodRuleCollection() {
List<IAlgebraicRewriteRule> accessMethod = new LinkedList<>();
accessMethod.add(new IntroduceSelectAccessMethodRule());
accessMethod.add(new IntroduceJoinAccessMethodRule());
@@ -330,7 +330,7 @@
return accessMethod;
}
- public static final List<IAlgebraicRewriteRule> buildPlanCleanupRuleCollection() {
+ public static List<IAlgebraicRewriteRule> buildPlanCleanupRuleCollection() {
List<IAlgebraicRewriteRule> planCleanupRules = new LinkedList<>();
planCleanupRules.add(new SwitchInnerJoinBranchRule());
planCleanupRules.add(new AsterixPushMapOperatorThroughUnionRule(LogicalOperatorTag.ASSIGN));
@@ -364,14 +364,14 @@
return planCleanupRules;
}
- public static final List<IAlgebraicRewriteRule> buildDataExchangeRuleCollection() {
+ public static List<IAlgebraicRewriteRule> buildDataExchangeRuleCollection() {
List<IAlgebraicRewriteRule> dataExchange = new LinkedList<>();
dataExchange.add(new SetExecutionModeRule());
dataExchange.add(new FindDataSourcesRule());
return dataExchange;
}
- public static final List<IAlgebraicRewriteRule> buildCBORuleCollection() {
+ public static List<IAlgebraicRewriteRule> buildCBORuleCollection() {
List<IAlgebraicRewriteRule> cbo = new LinkedList<>();
cbo.add(new ConsolidateSelectsRule());
cbo.add(new EnumerateJoinsRule(new JoinEnum()));
@@ -380,7 +380,7 @@
return cbo;
}
- public static final List<IAlgebraicRewriteRule> buildPhysicalRewritesAllLevelsRuleCollection() {
+ public static List<IAlgebraicRewriteRule> buildPhysicalRewritesAllLevelsRuleCollection() {
List<IAlgebraicRewriteRule> physicalRewritesAllLevels = new LinkedList<>();
physicalRewritesAllLevels.add(new PullSelectOutOfEqJoin());
physicalRewritesAllLevels.add(new ExtractBatchableExternalFunctionCallsRule());
@@ -408,7 +408,7 @@
return physicalRewritesAllLevels;
}
- public static final List<IAlgebraicRewriteRule> buildPhysicalRewritesTopLevelRuleCollection(
+ public static List<IAlgebraicRewriteRule> buildPhysicalRewritesTopLevelRuleCollection(
ICcApplicationContext appCtx) {
List<IAlgebraicRewriteRule> physicalRewritesTopLevel = new LinkedList<>();
physicalRewritesTopLevel.add(new PushNestedOrderByUnderPreSortedGroupByRule());
@@ -418,11 +418,17 @@
physicalRewritesTopLevel.add(new ConstantFoldingRule(appCtx));
physicalRewritesTopLevel.add(new PushLimitIntoOrderByRule());
//Must run before PushLimitIntoPrimarySearchRule to ensure the select condition is inspected
- physicalRewritesTopLevel.add(new PushValueAccessToExternalDataScanRule());
physicalRewritesTopLevel.add(new PushLimitIntoPrimarySearchRule());
// remove assigns that could become unused after PushLimitIntoPrimarySearchRule
physicalRewritesTopLevel.add(new RemoveUnusedAssignAndAggregateRule());
physicalRewritesTopLevel.add(new IntroduceProjectsRule());
+ //Infer the types for the pushed down condition
+ physicalRewritesTopLevel.add(new InferTypesRule());
+ /*
+ * Must run IntroduceProjectsRule before PushValueAccessToDataScanRule to ensure that no entire records are
+ * returned if they are projected out
+ */
+ physicalRewritesTopLevel.add(new PushValueAccessToDataScanRule());
physicalRewritesTopLevel.add(new SetAsterixPhysicalOperatorsRule());
physicalRewritesTopLevel.add(new IntroduceRapidFrameFlushProjectAssignRule());
physicalRewritesTopLevel.add(new SetExecutionModeRule());
@@ -430,7 +436,7 @@
return physicalRewritesTopLevel;
}
- public static final List<IAlgebraicRewriteRule> prepareForJobGenRuleCollection() {
+ public static List<IAlgebraicRewriteRule> prepareForJobGenRuleCollection() {
List<IAlgebraicRewriteRule> prepareForJobGenRewrites = new LinkedList<>();
prepareForJobGenRewrites.add(new InsertProjectBeforeUnionRule());
prepareForJobGenRewrites.add(new SetAsterixPhysicalOperatorsRule());
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushValueAccessToExternalDataScanRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushValueAccessToDataScanRule.java
similarity index 91%
rename from asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushValueAccessToExternalDataScanRule.java
rename to asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushValueAccessToDataScanRule.java
index 405e2bd..d43a5d1 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushValueAccessToExternalDataScanRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushValueAccessToDataScanRule.java
@@ -63,7 +63,7 @@
* The resulting record $$r will be {"personalInfo":{"age": *AGE*}, "salary": *SALARY*}
* and other fields will not be included in $$r.
*/
-public class PushValueAccessToExternalDataScanRule implements IAlgebraicRewriteRule {
+public class PushValueAccessToDataScanRule implements IAlgebraicRewriteRule {
//Initially, assume we need to run the rule
private boolean run = true;
@@ -76,7 +76,7 @@
}
/*
- * Only run the rewrite rule once and only if the plan contains a data-scan on an external dataset that
+ * Only run the rewrite rule once and only if the plan contains a data-scan on a dataset that
* support value access pushdown.
*/
run = shouldRun(context);
@@ -92,7 +92,7 @@
}
/**
- * Check whether the plan contains an external dataset that supports pushdown
+ * Check whether the plan contains a dataset that supports pushdown
*
* @param context optimization context
* @return true if the plan contains such dataset, false otherwise
@@ -117,7 +117,8 @@
String datasetName = dataSource.getId().getDatasourceName();
Dataset dataset = metadataProvider.findDataset(dataverse, datasetName);
- return dataset != null && dataset.getDatasetType() == DatasetConfig.DatasetType.EXTERNAL && ExternalDataUtils
- .supportsPushdown(((ExternalDatasetDetails) dataset.getDatasetDetails()).getProperties());
+ return dataset != null && ((dataset.getDatasetType() == DatasetConfig.DatasetType.EXTERNAL && ExternalDataUtils
+ .supportsPushdown(((ExternalDatasetDetails) dataset.getDatasetDetails()).getProperties()))
+ || dataset.getDatasetFormatInfo().getFormat() == DatasetConfig.DatasetFormat.COLUMN);
}
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/ExpectedSchemaBuilder.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/ExpectedSchemaBuilder.java
index b7632db..f972677 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/ExpectedSchemaBuilder.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/ExpectedSchemaBuilder.java
@@ -21,7 +21,9 @@
import static org.apache.asterix.optimizer.rules.pushdown.ExpressionValueAccessPushdownVisitor.ARRAY_FUNCTIONS;
import static org.apache.asterix.optimizer.rules.pushdown.ExpressionValueAccessPushdownVisitor.SUPPORTED_FUNCTIONS;
+import java.util.Collections;
import java.util.HashMap;
+import java.util.List;
import java.util.Map;
import org.apache.asterix.om.functions.BuiltinFunctions;
@@ -35,12 +37,15 @@
import org.apache.asterix.optimizer.rules.pushdown.schema.ObjectExpectedSchemaNode;
import org.apache.asterix.optimizer.rules.pushdown.schema.RootExpectedSchemaNode;
import org.apache.asterix.optimizer.rules.pushdown.schema.UnionExpectedSchemaNode;
-import org.apache.asterix.runtime.projection.DataProjectionInfo;
+import org.apache.asterix.runtime.projection.DataProjectionFiltrationInfo;
import org.apache.asterix.runtime.projection.FunctionCallInformation;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
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.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
@@ -53,31 +58,44 @@
* 2- the output type of getField("hashtags") is ARRAY
* 3- the output type of getItem(0) is ANY node
*/
-class ExpectedSchemaBuilder {
+public class ExpectedSchemaBuilder {
//Registered Variables
private final Map<LogicalVariable, IExpectedSchemaNode> varToNode;
- private final ExpectedSchemaNodeToIATypeTranslatorVisitor typeBuilder;
+ //Inferred type for expressions
+ private final Map<AbstractFunctionCallExpression, IExpectedSchemaNode> exprToNode;
public ExpectedSchemaBuilder() {
varToNode = new HashMap<>();
- typeBuilder = new ExpectedSchemaNodeToIATypeTranslatorVisitor();
+ exprToNode = new HashMap<>();
}
- public DataProjectionInfo createProjectionInfo(LogicalVariable recordVariable) {
+ public DataProjectionFiltrationInfo createProjectionInfo(LogicalVariable recordVariable) {
+ return createProjectionInfo(recordVariable, Collections.emptyMap(), null, null);
+ }
+
+ public DataProjectionFiltrationInfo createProjectionInfo(LogicalVariable recordVariable,
+ Map<ILogicalExpression, ARecordType> filterPaths, ILogicalExpression filterExpression,
+ Map<String, FunctionCallInformation> sourceInformationMap) {
IExpectedSchemaNode rootNode = varToNode.get(recordVariable);
- Map<String, FunctionCallInformation> sourceInformation = new HashMap<>();
- typeBuilder.reset(sourceInformation);
+
+ ExpectedSchemaNodeToIATypeTranslatorVisitor typeBuilder =
+ new ExpectedSchemaNodeToIATypeTranslatorVisitor(sourceInformationMap);
ARecordType recordType = (ARecordType) rootNode.accept(typeBuilder, null);
- return new DataProjectionInfo(recordType, sourceInformation);
+
+ return new DataProjectionFiltrationInfo(recordType, sourceInformationMap, filterPaths, filterExpression);
}
- public boolean setSchemaFromExpression(AbstractFunctionCallExpression expr, LogicalVariable producedVar) {
+ public boolean setSchemaFromExpression(AbstractFunctionCallExpression expr, LogicalVariable producedVar,
+ IVariableTypeEnvironment typeEnv) throws AlgebricksException {
//Parent always nested
- AbstractComplexExpectedSchemaNode parent = (AbstractComplexExpectedSchemaNode) buildNestedNode(expr);
+ AbstractComplexExpectedSchemaNode parent = (AbstractComplexExpectedSchemaNode) buildNestedNode(expr, typeEnv);
if (parent != null) {
IExpectedSchemaNode leaf =
new AnyExpectedSchemaNode(parent, expr.getSourceLocation(), expr.getFunctionIdentifier().getName());
- addChild(expr, parent, leaf);
+ addChild(expr, typeEnv, parent, leaf);
+
+ //Associate expression to node
+ exprToNode.put(expr, leaf);
if (producedVar != null) {
//Register the node if a variable is produced
varToNode.put(producedVar, leaf);
@@ -86,7 +104,7 @@
return parent != null;
}
- public void registerDataset(LogicalVariable recordVar, RootExpectedSchemaNode rootNode) {
+ public void registerRoot(LogicalVariable recordVar, RootExpectedSchemaNode rootNode) {
varToNode.put(recordVar, rootNode);
}
@@ -111,10 +129,19 @@
return !varToNode.isEmpty();
}
- private IExpectedSchemaNode buildNestedNode(ILogicalExpression expr) {
+ IExpectedSchemaNode getNode(LogicalVariable variable) {
+ return varToNode.get(variable);
+ }
+
+ IExpectedSchemaNode getNode(AbstractFunctionCallExpression expr) {
+ return exprToNode.get(expr);
+ }
+
+ private IExpectedSchemaNode buildNestedNode(ILogicalExpression expr, IVariableTypeEnvironment typeEnv)
+ throws AlgebricksException {
//The current node expression
AbstractFunctionCallExpression myExpr = (AbstractFunctionCallExpression) expr;
- if (!SUPPORTED_FUNCTIONS.contains(myExpr.getFunctionIdentifier())) {
+ if (!SUPPORTED_FUNCTIONS.contains(myExpr.getFunctionIdentifier()) || noArgsOrFirstArgIsConstant(myExpr)) {
//Return null if the function is not supported.
return null;
}
@@ -128,7 +155,8 @@
}
//Recursively create the parent nodes. Parent is always a nested node
- AbstractComplexExpectedSchemaNode newParent = (AbstractComplexExpectedSchemaNode) buildNestedNode(parentExpr);
+ AbstractComplexExpectedSchemaNode newParent =
+ (AbstractComplexExpectedSchemaNode) buildNestedNode(parentExpr, typeEnv);
//newParent could be null if the expression is not supported
if (newParent != null) {
//Parent expression must be a function call (as parent is a nested node)
@@ -136,18 +164,23 @@
//Get 'myType' as we will create the child type of the newParent
ExpectedSchemaNodeType myType = getExpectedNestedNodeType(myExpr);
/*
- * Create 'myNode'. It is a nested node because the function is either getField() or supported array
+ * Create 'myNode'. It is a nested node because the function is either getField() or a supported array
* function
*/
AbstractComplexExpectedSchemaNode myNode = AbstractComplexExpectedSchemaNode.createNestedNode(myType,
newParent, myExpr.getSourceLocation(), myExpr.getFunctionIdentifier().getName());
//Add myNode to the parent
- addChild(parentFuncExpr, newParent, myNode);
+ addChild(parentFuncExpr, typeEnv, newParent, myNode);
return myNode;
}
return null;
}
+ private boolean noArgsOrFirstArgIsConstant(AbstractFunctionCallExpression myExpr) {
+ List<Mutable<ILogicalExpression>> args = myExpr.getArguments();
+ return args.isEmpty() || args.get(0).getValue().getExpressionTag() == LogicalExpressionTag.CONSTANT;
+ }
+
private IExpectedSchemaNode changeNodeForVariable(LogicalVariable sourceVar,
AbstractFunctionCallExpression myExpr) {
//Get the associated node with the sourceVar (if any)
@@ -166,11 +199,11 @@
return newNode;
}
- private void addChild(AbstractFunctionCallExpression parentExpr, AbstractComplexExpectedSchemaNode parent,
- IExpectedSchemaNode child) {
+ private void addChild(AbstractFunctionCallExpression parentExpr, IVariableTypeEnvironment typeEnv,
+ AbstractComplexExpectedSchemaNode parent, IExpectedSchemaNode child) throws AlgebricksException {
switch (parent.getType()) {
case OBJECT:
- handleObject(parentExpr, parent, child);
+ handleObject(parentExpr, typeEnv, parent, child);
break;
case ARRAY:
handleArray(parent, child);
@@ -184,10 +217,18 @@
}
}
- private void handleObject(AbstractFunctionCallExpression parentExpr, AbstractComplexExpectedSchemaNode parent,
- IExpectedSchemaNode child) {
- ObjectExpectedSchemaNode objectNode = (ObjectExpectedSchemaNode) parent;
- objectNode.addChild(ConstantExpressionUtil.getStringArgument(parentExpr, 1), child);
+ private void handleObject(AbstractFunctionCallExpression parentExpr, IVariableTypeEnvironment typeEnv,
+ AbstractComplexExpectedSchemaNode parent, IExpectedSchemaNode child) throws AlgebricksException {
+ if (BuiltinFunctions.FIELD_ACCESS_BY_NAME.equals(parentExpr.getFunctionIdentifier())) {
+ ObjectExpectedSchemaNode objectNode = (ObjectExpectedSchemaNode) parent;
+ objectNode.addChild(ConstantExpressionUtil.getStringArgument(parentExpr, 1), child);
+ } else {
+ //FIELD_ACCESS_BY_INDEX
+ ARecordType recordType = (ARecordType) typeEnv.getType(parentExpr.getArguments().get(0).getValue());
+ ObjectExpectedSchemaNode objectNode = (ObjectExpectedSchemaNode) parent;
+ int fieldIdx = ConstantExpressionUtil.getIntArgument(parentExpr, 1);
+ objectNode.addChild(recordType.getFieldNames()[fieldIdx], child);
+ }
}
private void handleArray(AbstractComplexExpectedSchemaNode parent, IExpectedSchemaNode child) {
@@ -196,15 +237,15 @@
}
private void handleUnion(AbstractFunctionCallExpression parentExpr, AbstractComplexExpectedSchemaNode parent,
- IExpectedSchemaNode child) {
+ IExpectedSchemaNode child) throws AlgebricksException {
UnionExpectedSchemaNode unionNode = (UnionExpectedSchemaNode) parent;
ExpectedSchemaNodeType parentType = getExpectedNestedNodeType(parentExpr);
- addChild(parentExpr, unionNode.getChild(parentType), child);
+ addChild(parentExpr, null, unionNode.getChild(parentType), child);
}
private static ExpectedSchemaNodeType getExpectedNestedNodeType(AbstractFunctionCallExpression funcExpr) {
FunctionIdentifier fid = funcExpr.getFunctionIdentifier();
- if (BuiltinFunctions.FIELD_ACCESS_BY_NAME.equals(fid)) {
+ if (BuiltinFunctions.FIELD_ACCESS_BY_NAME.equals(fid) || BuiltinFunctions.FIELD_ACCESS_BY_INDEX.equals(fid)) {
return ExpectedSchemaNodeType.OBJECT;
} else if (ARRAY_FUNCTIONS.contains(fid)) {
return ExpectedSchemaNodeType.ARRAY;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/ExpectedSchemaNodeToIATypeTranslatorVisitor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/ExpectedSchemaNodeToIATypeTranslatorVisitor.java
index 0da2c05..32e4b18 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/ExpectedSchemaNodeToIATypeTranslatorVisitor.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/ExpectedSchemaNodeToIATypeTranslatorVisitor.java
@@ -37,7 +37,7 @@
import org.apache.asterix.optimizer.rules.pushdown.schema.ObjectExpectedSchemaNode;
import org.apache.asterix.optimizer.rules.pushdown.schema.RootExpectedSchemaNode;
import org.apache.asterix.optimizer.rules.pushdown.schema.UnionExpectedSchemaNode;
-import org.apache.asterix.runtime.projection.DataProjectionInfo;
+import org.apache.asterix.runtime.projection.DataProjectionFiltrationInfo;
import org.apache.asterix.runtime.projection.FunctionCallInformation;
import org.apache.asterix.runtime.projection.ProjectionFiltrationWarningFactoryProvider;
@@ -45,22 +45,22 @@
* This visitor translates the {@link IExpectedSchemaNode} to {@link IAType} record.
* The {@link IAType#getTypeName()} is used to map each {@link IAType} to its {@link FunctionCallInformation}
*/
-class ExpectedSchemaNodeToIATypeTranslatorVisitor implements IExpectedSchemaNodeVisitor<IAType, String> {
+public class ExpectedSchemaNodeToIATypeTranslatorVisitor implements IExpectedSchemaNodeVisitor<IAType, String> {
//Map typeName to source information
- private Map<String, FunctionCallInformation> sourceInformationMap;
+ private final Map<String, FunctionCallInformation> sourceInformationMap;
//To give a unique name for each type
private int counter;
- public void reset(Map<String, FunctionCallInformation> sourceInformationMap) {
+ public ExpectedSchemaNodeToIATypeTranslatorVisitor(Map<String, FunctionCallInformation> sourceInformationMap) {
this.sourceInformationMap = sourceInformationMap;
}
@Override
public IAType visit(RootExpectedSchemaNode node, String arg) {
if (node.isAllFields()) {
- return DataProjectionInfo.ALL_FIELDS_TYPE;
+ return DataProjectionFiltrationInfo.ALL_FIELDS_TYPE;
} else if (node.isEmpty()) {
- return DataProjectionInfo.EMPTY_TYPE;
+ return DataProjectionFiltrationInfo.EMPTY_TYPE;
}
return createRecordType(node, String.valueOf(counter++));
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/ExpressionValueAccessPushdownVisitor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/ExpressionValueAccessPushdownVisitor.java
index a8dfe1e..8096b04 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/ExpressionValueAccessPushdownVisitor.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/ExpressionValueAccessPushdownVisitor.java
@@ -29,21 +29,23 @@
import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
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.functions.AlgebricksBuiltinFunctions;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
class ExpressionValueAccessPushdownVisitor implements ILogicalExpressionReferenceTransform {
- //Set of supported type-check functions
- static final Set<FunctionIdentifier> TYPE_CHECK_FUNCTIONS = createSupportedTypeCheckFunctions();
+ //Set of allowed functions that can request a type in its entirety
+ static final Set<FunctionIdentifier> ALLOWED_FUNCTIONS = createAllowedFunctions();
//Set of supported array functions
static final Set<FunctionIdentifier> ARRAY_FUNCTIONS = createSupportedArrayFunctions();
- //Set of supported functions that we can pushdown
+ //Set of supported functions that we can push down
static final Set<FunctionIdentifier> SUPPORTED_FUNCTIONS = createSupportedFunctions();
private final ExpectedSchemaBuilder builder;
private List<LogicalVariable> producedVariables;
+ private IVariableTypeEnvironment typeEnv;
private int producedVariableIndex;
public ExpressionValueAccessPushdownVisitor(ExpectedSchemaBuilder builder) {
@@ -51,8 +53,9 @@
end();
}
- public void init(List<LogicalVariable> producedVariables) {
+ public void init(List<LogicalVariable> producedVariables, IVariableTypeEnvironment typeEnv) {
this.producedVariables = producedVariables;
+ this.typeEnv = typeEnv;
producedVariableIndex = 0;
}
@@ -62,12 +65,13 @@
//This for ensuring that the produced variables (if any) should be set
throw new IllegalStateException("init must be called first");
}
- pushValueAccessExpression(expression, getNextProducedVariable());
+ pushValueAccessExpression(expression, getNextProducedVariable(), typeEnv);
return false;
}
public void end() {
producedVariables = null;
+ typeEnv = null;
producedVariableIndex = -1;
}
@@ -80,7 +84,8 @@
/**
* Pushdown field access expressions and array access expressions down
*/
- private void pushValueAccessExpression(Mutable<ILogicalExpression> exprRef, LogicalVariable producedVar) {
+ private void pushValueAccessExpression(Mutable<ILogicalExpression> exprRef, LogicalVariable producedVar,
+ IVariableTypeEnvironment typeEnv) throws AlgebricksException {
final ILogicalExpression expr = exprRef.getValue();
if (skipPushdown(expr)) {
return;
@@ -88,7 +93,7 @@
final AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
- if (isSuccessfullyPushedDown(funcExpr, producedVar)) {
+ if (isSuccessfullyPushedDown(funcExpr, producedVar, typeEnv)) {
//We successfully pushed down the value access function
return;
}
@@ -138,23 +143,24 @@
*/
private boolean isTypeCheckOnVariable(ILogicalExpression expression) {
AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expression;
- return TYPE_CHECK_FUNCTIONS.contains(funcExpr.getFunctionIdentifier())
+ return ALLOWED_FUNCTIONS.contains(funcExpr.getFunctionIdentifier())
&& funcExpr.getArguments().get(0).getValue().getExpressionTag() == LogicalExpressionTag.VARIABLE;
}
- private void pushValueAccessExpressionArg(List<Mutable<ILogicalExpression>> exprList) {
+ private void pushValueAccessExpressionArg(List<Mutable<ILogicalExpression>> exprList) throws AlgebricksException {
for (Mutable<ILogicalExpression> exprRef : exprList) {
/*
* We need to set the produced variable as null here as the produced variable will not correspond to the
* nested expression.
*/
- pushValueAccessExpression(exprRef, null);
+ pushValueAccessExpression(exprRef, null, typeEnv);
}
}
- private boolean isSuccessfullyPushedDown(AbstractFunctionCallExpression funcExpr, LogicalVariable producedVar) {
+ private boolean isSuccessfullyPushedDown(AbstractFunctionCallExpression funcExpr, LogicalVariable producedVar,
+ IVariableTypeEnvironment typeEnv) throws AlgebricksException {
return SUPPORTED_FUNCTIONS.contains(funcExpr.getFunctionIdentifier())
- && builder.setSchemaFromExpression(funcExpr, producedVar);
+ && builder.setSchemaFromExpression(funcExpr, producedVar, typeEnv);
}
private void unregisterVariableIfNeeded(LogicalVariable variable) {
@@ -169,15 +175,17 @@
private static Set<FunctionIdentifier> createSupportedFunctions() {
Set<FunctionIdentifier> supportedFunctions = new HashSet<>();
- //For objects, only field-access-by-name is supported
supportedFunctions.add(BuiltinFunctions.FIELD_ACCESS_BY_NAME);
+ supportedFunctions.add(BuiltinFunctions.FIELD_ACCESS_BY_INDEX);
supportedFunctions.addAll(ARRAY_FUNCTIONS);
return supportedFunctions;
}
- private static Set<FunctionIdentifier> createSupportedTypeCheckFunctions() {
+ private static Set<FunctionIdentifier> createAllowedFunctions() {
return Set.of(BuiltinFunctions.IS_ARRAY, BuiltinFunctions.IS_OBJECT, BuiltinFunctions.IS_ATOMIC,
BuiltinFunctions.IS_NUMBER, BuiltinFunctions.IS_BOOLEAN, BuiltinFunctions.IS_STRING,
- AlgebricksBuiltinFunctions.IS_MISSING, AlgebricksBuiltinFunctions.IS_NULL, BuiltinFunctions.IS_UNKNOWN);
+ AlgebricksBuiltinFunctions.IS_MISSING, AlgebricksBuiltinFunctions.IS_NULL, BuiltinFunctions.IS_UNKNOWN,
+ BuiltinFunctions.LT, BuiltinFunctions.LE, BuiltinFunctions.EQ, BuiltinFunctions.GT, BuiltinFunctions.GE,
+ BuiltinFunctions.SCALAR_SQL_COUNT);
}
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/ExpressionValueFilterPushdown.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/ExpressionValueFilterPushdown.java
new file mode 100644
index 0000000..8ca2372
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/ExpressionValueFilterPushdown.java
@@ -0,0 +1,214 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.optimizer.rules.pushdown;
+
+import static org.apache.asterix.metadata.utils.ColumnFilterBuilder.COMPARE_FUNCTIONS;
+import static org.apache.asterix.metadata.utils.ColumnFilterBuilder.PUSHABLE_FUNCTIONS;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.asterix.common.config.DatasetConfig;
+import org.apache.asterix.metadata.declared.DatasetDataSource;
+import org.apache.asterix.metadata.entities.Dataset;
+import org.apache.asterix.om.base.IAObject;
+import org.apache.asterix.om.constants.AsterixConstantValue;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.optimizer.rules.pushdown.schema.AnyExpectedSchemaNode;
+import org.apache.asterix.optimizer.rules.pushdown.schema.ColumnFilterPathBuilderVisitor;
+import org.apache.asterix.optimizer.rules.pushdown.schema.ExpectedSchemaNodeType;
+import org.apache.asterix.optimizer.rules.pushdown.schema.IExpectedSchemaNode;
+import org.apache.asterix.runtime.projection.FunctionCallInformation;
+import org.apache.asterix.runtime.projection.ProjectionFiltrationWarningFactoryProvider;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractScanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+/**
+ * Pushdown {@link SelectOperator} condition to the dataset to allow filtering mega leaf nodes.
+ * This is currently only allowed for {@link DatasetConfig.DatasetFormat#COLUMN}
+ * TODO allow partial filter with AND (e.g., lowercase(stringVal) == "some_text" AND intVal > 10 --push--> intVal > 10 )
+ * TODO Filter could prevent REPLICATE (i.e., we can scan a dataset twice due to the fact one scan is filtered and
+ * TODO the other is not) or (both have different filters)
+ * TODO part of this class could potentially be used for external data dynamic prefixes
+ */
+class ExpressionValueFilterPushdown {
+ private final ExpectedSchemaBuilder builder;
+ private final ColumnFilterPathBuilderVisitor pathBuilder;
+ private final Map<AbstractScanOperator, Map<ILogicalExpression, ARecordType>> datasetFilterPaths;
+ private final Map<AbstractScanOperator, ILogicalExpression> datasetFilterExpression;
+ private final Map<AbstractScanOperator, Map<String, FunctionCallInformation>> scanSourceInformationMaps;
+ private final Set<AbstractScanOperator> registeredScans;
+ private final boolean columnFilterEnabled;
+
+ ExpressionValueFilterPushdown(ExpectedSchemaBuilder builder, boolean columnFilterEnabled) {
+ this.builder = builder;
+ pathBuilder = new ColumnFilterPathBuilderVisitor();
+ datasetFilterPaths = new HashMap<>();
+ datasetFilterExpression = new HashMap<>();
+ scanSourceInformationMaps = new HashMap<>();
+ registeredScans = new HashSet<>();
+ this.columnFilterEnabled = columnFilterEnabled;
+ }
+
+ public void registerDataset(AbstractScanOperator op, DatasetDataSource source) {
+ if (!columnFilterEnabled) {
+ return;
+ }
+
+ Dataset dataset = source.getDataset();
+ if (dataset.getDatasetType() == DatasetConfig.DatasetType.INTERNAL
+ && dataset.getDatasetFormatInfo().getFormat() == DatasetConfig.DatasetFormat.COLUMN) {
+ registeredScans.add(op);
+ }
+ }
+
+ /**
+ * Try to push the condition down to dataset scan/access
+ *
+ * @param selectOp the select operator
+ */
+ public void addFilterExpression(SelectOperator selectOp, AbstractScanOperator scanOp) {
+ if (datasetFilterPaths.containsKey(scanOp)) {
+ // Most bottom SELECT was applied, other selects should be ignored
+ return;
+ }
+ Map<ILogicalExpression, ARecordType> filterPaths = new HashMap<>();
+ Map<String, FunctionCallInformation> sourceInformationMap = new HashMap<>();
+ Mutable<ILogicalExpression> conditionRef = selectOp.getCondition();
+ if (addPaths(conditionRef, filterPaths, sourceInformationMap)) {
+ datasetFilterPaths.put(scanOp, filterPaths);
+ datasetFilterExpression.put(scanOp, conditionRef.getValue());
+ scanSourceInformationMaps.put(scanOp, sourceInformationMap);
+ }
+ }
+
+ public Map<ILogicalExpression, ARecordType> getFilterPaths(AbstractScanOperator scanOp) {
+ return datasetFilterPaths.getOrDefault(scanOp, Collections.emptyMap());
+ }
+
+ public ILogicalExpression getFilterExpression(AbstractScanOperator scanOp) {
+ return datasetFilterExpression.get(scanOp);
+ }
+
+ public Map<String, FunctionCallInformation> getSourceInformationMap(AbstractScanOperator scanOp) {
+ return scanSourceInformationMaps.getOrDefault(scanOp, new HashMap<>());
+ }
+
+ private boolean addPaths(Mutable<ILogicalExpression> exprRef, Map<ILogicalExpression, ARecordType> filterPaths,
+ Map<String, FunctionCallInformation> sourceInformationMap) {
+ ILogicalExpression expr = exprRef.getValue();
+ if (!isFunctionExpression(expr)) {
+ return false;
+ }
+
+ AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
+ FunctionIdentifier fid = ((AbstractFunctionCallExpression) expr).getFunctionIdentifier();
+
+ if (!PUSHABLE_FUNCTIONS.contains(fid)) {
+ return false;
+ }
+
+ if (COMPARE_FUNCTIONS.contains(fid)) {
+ return addPaths(funcExpr, filterPaths, sourceInformationMap);
+ }
+ //AND/OR descend to the expression tree
+ return addPathsForArgs(funcExpr, filterPaths, sourceInformationMap);
+ }
+
+ private boolean addPaths(AbstractFunctionCallExpression funcExpr, Map<ILogicalExpression, ARecordType> filterPaths,
+ Map<String, FunctionCallInformation> sourceInformationMap) {
+ List<Mutable<ILogicalExpression>> args = funcExpr.getArguments();
+
+ ILogicalExpression left = args.get(0).getValue();
+ ILogicalExpression right = args.get(1).getValue();
+
+ if (isConstantExpression(left)) {
+ return addPaths(funcExpr, right, left, filterPaths, sourceInformationMap, true);
+ } else if (isConstantExpression(right)) {
+ return addPaths(funcExpr, left, right, filterPaths, sourceInformationMap, false);
+ }
+ // No constants, return false
+ return false;
+ }
+
+ private boolean addPaths(AbstractFunctionCallExpression funcExpr, ILogicalExpression columnExpr,
+ ILogicalExpression constExpr, Map<ILogicalExpression, ARecordType> filterPaths,
+ Map<String, FunctionCallInformation> sourceInformationMap, boolean leftConstant) {
+ IExpectedSchemaNode node;
+ if (isFunctionExpression(columnExpr)) {
+ node = builder.getNode((AbstractFunctionCallExpression) columnExpr);
+ } else {
+ //Variable
+ node = builder.getNode(((VariableReferenceExpression) columnExpr).getVariableReference());
+ }
+
+ if (node == null || node.getType() != ExpectedSchemaNodeType.ANY) {
+ // Expression cannot be pushed (e.g., $$r.getField("x") + 1) or had been accessed as a nested value
+ // Bail out
+ return false;
+ }
+
+ AnyExpectedSchemaNode leafNode = (AnyExpectedSchemaNode) node;
+ IAObject constantValue = ((AsterixConstantValue) ((ConstantExpression) constExpr).getValue()).getObject();
+
+ String functionName = funcExpr.getFunctionIdentifier().getName();
+ SourceLocation sourceLocation = funcExpr.getSourceLocation();
+ FunctionCallInformation functionCallInfo = new FunctionCallInformation(functionName, sourceLocation,
+ ProjectionFiltrationWarningFactoryProvider.getIncomparableTypesFactory(leftConstant));
+
+ ARecordType path = pathBuilder.buildPath(leafNode, constantValue, sourceInformationMap, functionCallInfo);
+ filterPaths.put(columnExpr, path);
+ return true;
+ }
+
+ private boolean addPathsForArgs(AbstractFunctionCallExpression funcExpr,
+ Map<ILogicalExpression, ARecordType> filterPaths,
+ Map<String, FunctionCallInformation> sourceInformationMap) {
+ List<Mutable<ILogicalExpression>> args = funcExpr.getArguments();
+ boolean add = true;
+ for (int i = 0; add && i < args.size(); i++) {
+ add = addPaths(args.get(i), filterPaths, sourceInformationMap);
+ }
+ return add;
+ }
+
+ private static boolean isFunctionExpression(ILogicalExpression expr) {
+ return expr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL;
+ }
+
+ private static boolean isConstantExpression(ILogicalExpression expr) {
+ return expr.getExpressionTag() == LogicalExpressionTag.CONSTANT;
+ }
+
+ public boolean allowsPushdown(AbstractScanOperator lastSeenScan) {
+ return columnFilterEnabled && lastSeenScan != null && registeredScans.contains(lastSeenScan);
+ }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/OperatorValueAccessPushdownVisitor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/OperatorValueAccessPushdownVisitor.java
index 05bc161..8fe8ce7 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/OperatorValueAccessPushdownVisitor.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/OperatorValueAccessPushdownVisitor.java
@@ -18,6 +18,7 @@
*/
package org.apache.asterix.optimizer.rules.pushdown;
+import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
@@ -25,15 +26,20 @@
import java.util.Set;
import org.apache.asterix.common.config.DatasetConfig;
+import org.apache.asterix.common.config.DatasetConfig.DatasetFormat;
import org.apache.asterix.common.metadata.DataverseName;
import org.apache.asterix.external.util.ExternalDataUtils;
import org.apache.asterix.metadata.declared.DataSource;
+import org.apache.asterix.metadata.declared.DataSourceId;
import org.apache.asterix.metadata.declared.DatasetDataSource;
import org.apache.asterix.metadata.declared.MetadataProvider;
import org.apache.asterix.metadata.entities.Dataset;
import org.apache.asterix.metadata.entities.ExternalDatasetDetails;
import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.utils.ConstantExpressionUtil;
import org.apache.asterix.optimizer.rules.pushdown.schema.RootExpectedSchemaNode;
+import org.apache.asterix.runtime.projection.FunctionCallInformation;
import org.apache.commons.lang3.mutable.Mutable;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
@@ -44,7 +50,9 @@
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.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractScanOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
@@ -88,28 +96,60 @@
* This visitor visits the entire plan and tries to build the information of the required values from all dataset
*/
public class OperatorValueAccessPushdownVisitor implements ILogicalOperatorVisitor<Void, Void> {
+ private static final List<LogicalVariable> EMPTY_VARIABLES = Collections.emptyList();
private final IOptimizationContext context;
//Requested schema builder. It is only expected schema not a definite one
private final ExpectedSchemaBuilder builder;
//To visit every expression in each operator
private final ExpressionValueAccessPushdownVisitor pushdownVisitor;
+ private final ExpressionValueFilterPushdown filterPushdown;
//Datasets that allow pushdowns
- private final Map<LogicalVariable, DataSourceScanOperator> registeredDatasets;
+ private final Map<LogicalVariable, AbstractScanOperator> registeredDatasets;
+ //Datasets that allow pushdowns and has meta
+ private final Map<LogicalVariable, AbstractScanOperator> registeredMetas;
//visitedOperators so we do not visit the same operator twice (in case of REPLICATE)
private final Set<ILogicalOperator> visitedOperators;
+ //Last scan operator seen
+ private AbstractScanOperator lastSeenScan;
public OperatorValueAccessPushdownVisitor(IOptimizationContext context) {
this.context = context;
builder = new ExpectedSchemaBuilder();
registeredDatasets = new HashMap<>();
+ registeredMetas = new HashMap<>();
pushdownVisitor = new ExpressionValueAccessPushdownVisitor(builder);
+ filterPushdown = new ExpressionValueFilterPushdown(builder,
+ context.getPhysicalOptimizationConfig().isColumnFilterEnabled());
visitedOperators = new HashSet<>();
}
public void finish() {
- for (Map.Entry<LogicalVariable, DataSourceScanOperator> scan : registeredDatasets.entrySet()) {
- scan.getValue().setProjectionInfo(builder.createProjectionInfo(scan.getKey()));
+ for (Map.Entry<LogicalVariable, AbstractScanOperator> entry : registeredDatasets.entrySet()) {
+ AbstractScanOperator scanOp = entry.getValue();
+ Map<ILogicalExpression, ARecordType> filterPaths = filterPushdown.getFilterPaths(scanOp);
+ ILogicalExpression filterExpression = filterPushdown.getFilterExpression(scanOp);
+ Map<String, FunctionCallInformation> sourceInformationMap = filterPushdown.getSourceInformationMap(scanOp);
+ if (scanOp.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
+ DataSourceScanOperator scan = (DataSourceScanOperator) scanOp;
+ scan.setDatasetProjectionInfo(builder.createProjectionInfo(entry.getKey(), filterPaths,
+ filterExpression, sourceInformationMap));
+ } else {
+ UnnestMapOperator unnest = (UnnestMapOperator) scanOp;
+ unnest.setDatasetProjectionInfo(builder.createProjectionInfo(entry.getKey(), filterPaths,
+ filterExpression, sourceInformationMap));
+ }
+ }
+
+ for (Map.Entry<LogicalVariable, AbstractScanOperator> entry : registeredMetas.entrySet()) {
+ AbstractScanOperator abstractScan = entry.getValue();
+ if (abstractScan.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
+ DataSourceScanOperator scan = (DataSourceScanOperator) abstractScan;
+ scan.setMetaProjectionInfo(builder.createProjectionInfo(entry.getKey()));
+ } else {
+ UnnestMapOperator unnest = (UnnestMapOperator) abstractScan;
+ unnest.setMetaProjectionInfo(builder.createProjectionInfo(entry.getKey()));
+ }
}
}
@@ -127,11 +167,18 @@
for (Mutable<ILogicalOperator> child : op.getInputs()) {
child.getValue().accept(this, null);
}
+ IVariableTypeEnvironment typeEnv = op.computeOutputTypeEnvironment(context);
visitedOperators.add(op);
//Initiate the pushdown visitor
- pushdownVisitor.init(producedVariables);
+ pushdownVisitor.init(producedVariables, typeEnv);
//pushdown any expression the operator has
op.acceptExpressionTransform(pushdownVisitor);
+
+ if (filterPushdown.allowsPushdown(lastSeenScan) && op.getOperatorTag() == LogicalOperatorTag.SELECT) {
+ //Push filters down
+ filterPushdown.addFilterExpression((SelectOperator) op, lastSeenScan);
+ }
+
pushdownVisitor.end();
}
@@ -144,10 +191,8 @@
@Override
public Void visitProjectOperator(ProjectOperator op, Void arg) throws AlgebricksException {
visitInputs(op);
- if (op.getVariables().isEmpty()) {
- //If the variables are empty and the next operator is DataSourceScanOperator, then set empty record
- setEmptyRecord(op.getInputs().get(0).getValue());
- }
+ //Set as empty records for data-scan or unnest-map if certain variables are projected out
+ setEmptyRecord(op.getInputs().get(0).getValue(), op.getVariables());
return null;
}
@@ -157,20 +202,21 @@
*/
@Override
public Void visitDataScanOperator(DataSourceScanOperator op, Void arg) throws AlgebricksException {
+ DatasetDataSource datasetDataSource = getDatasetDataSourceIfApplicable((DataSource) op.getDataSource());
+ registerDatasetIfApplicable(datasetDataSource, op);
visitInputs(op);
- DatasetDataSource datasetDataSource = getDatasetDataSourceIfApplicable(op);
- if (datasetDataSource != null) {
- LogicalVariable recordVar = datasetDataSource.getDataRecordVariable(op.getVariables());
- if (!builder.isVariableRegistered(recordVar)) {
- /*
- * This is the first time we see the dataset, and we know we might only need part of the record.
- * Register the dataset to prepare for value access expression pushdowns.
- * Initially, we will request the entire record.
- */
- builder.registerDataset(recordVar, RootExpectedSchemaNode.ALL_FIELDS_ROOT_NODE);
- registeredDatasets.put(recordVar, op);
- }
- }
+ return null;
+ }
+
+ /**
+ * From the {@link UnnestMapOperator}, we need to register the payload variable (record variable) to check
+ * which expression in the plan is using it.
+ */
+ @Override
+ public Void visitUnnestMapOperator(UnnestMapOperator op, Void arg) throws AlgebricksException {
+ visitInputs(op);
+ DatasetDataSource datasetDataSource = getDatasetDataSourceIfApplicable(getDataSourceFromUnnestMapOperator(op));
+ registerDatasetIfApplicable(datasetDataSource, op);
return null;
}
@@ -183,7 +229,7 @@
* It is local aggregate and has agg-sql-count function with a constant argument. Set empty record if the
* input operator is DataSourceScanOperator
*/
- setEmptyRecord(op.getInputs().get(0).getValue());
+ setEmptyRecord(op.getInputs().get(0).getValue(), EMPTY_VARIABLES);
}
return null;
}
@@ -196,11 +242,10 @@
/**
* The role of this method is:
- * 1- Check whether the dataset is an external dataset and allows value access pushdowns
+ * 1- Check whether the datasource allows value access pushdowns
* 2- return the actual DatasetDataSource
*/
- private DatasetDataSource getDatasetDataSourceIfApplicable(DataSourceScanOperator scan) throws AlgebricksException {
- DataSource dataSource = (DataSource) scan.getDataSource();
+ private DatasetDataSource getDatasetDataSourceIfApplicable(DataSource dataSource) throws AlgebricksException {
if (dataSource == null) {
return null;
}
@@ -211,9 +256,11 @@
Dataset dataset = mp.findDataset(dataverse, datasetName);
//Only external dataset can have pushed down expressions
- if (dataset == null || dataset.getDatasetType() == DatasetConfig.DatasetType.INTERNAL
- || dataset.getDatasetType() == DatasetConfig.DatasetType.EXTERNAL && !ExternalDataUtils
- .supportsPushdown(((ExternalDatasetDetails) dataset.getDatasetDetails()).getProperties())) {
+ if (dataset.getDatasetType() == DatasetConfig.DatasetType.EXTERNAL
+ && !ExternalDataUtils
+ .supportsPushdown(((ExternalDatasetDetails) dataset.getDatasetDetails()).getProperties())
+ || dataset.getDatasetType() == DatasetConfig.DatasetType.INTERNAL
+ && dataset.getDatasetFormatInfo().getFormat() == DatasetFormat.ROW) {
return null;
}
@@ -221,24 +268,116 @@
}
/**
- * If the inputOp is a {@link DataSourceScanOperator}, then set the projected value needed as empty record
+ * Find datasource from {@link UnnestMapOperator}
*
- * @param inputOp an operator that is potentially a {@link DataSourceScanOperator}
+ * @param unnest unnest map operator
+ * @return datasource
+ */
+ private DataSource getDataSourceFromUnnestMapOperator(UnnestMapOperator unnest) throws AlgebricksException {
+ AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) unnest.getExpressionRef().getValue();
+ String dataverse = ConstantExpressionUtil.getStringArgument(funcExpr, 2);
+ String dataset = ConstantExpressionUtil.getStringArgument(funcExpr, 3);
+ if (!ConstantExpressionUtil.getStringArgument(funcExpr, 0).equals(dataset)) {
+ return null;
+ }
+
+ DataSourceId dsid = new DataSourceId(DataverseName.createBuiltinDataverseName(dataverse), dataset);
+ MetadataProvider metadataProvider = (MetadataProvider) context.getMetadataProvider();
+ return metadataProvider.findDataSource(dsid);
+ }
+
+ private void registerDatasetIfApplicable(DatasetDataSource datasetDataSource, AbstractScanOperator op) {
+ if (datasetDataSource != null) {
+ LogicalVariable recordVar = datasetDataSource.getDataRecordVariable(op.getVariables());
+ if (!builder.isVariableRegistered(recordVar)) {
+ /*
+ * This is the first time we see the dataset, and we know we might only need part of the record.
+ * Register the dataset to prepare for value access expression pushdowns.
+ * Initially, we will request the entire record.
+ */
+ builder.registerRoot(recordVar, RootExpectedSchemaNode.ALL_FIELDS_ROOT_NODE);
+ filterPushdown.registerDataset(op, datasetDataSource);
+ registeredDatasets.put(recordVar, op);
+
+ if (datasetDataSource.hasMeta()) {
+ /*
+ * The dataset has meta. Register the meta root variable as another root for the dataset and add
+ * it the metaVar to the registered metas
+ */
+ LogicalVariable metaVar = datasetDataSource.getMetaVariable(op.getVariables());
+ builder.registerRoot(metaVar, RootExpectedSchemaNode.ALL_FIELDS_ROOT_NODE);
+ registeredMetas.put(metaVar, op);
+ }
+ }
+ lastSeenScan = op;
+ }
+ }
+
+ /**
+ * If the inputOp is a {@link DataSourceScanOperator} or {@link UnnestMapOperator}, then set the projected value
+ * needed as empty record if any variable originated from either operators are not in {@code retainedVariables}
+ *
+ * @param inputOp an operator that is potentially a {@link DataSourceScanOperator} or a {@link
+ * UnnestMapOperator}
+ * @param retainedVariables variables that should be retained
* @see #visitAggregateOperator(AggregateOperator, Void)
* @see #visitProjectOperator(ProjectOperator, Void)
*/
- private void setEmptyRecord(ILogicalOperator inputOp) throws AlgebricksException {
+ private void setEmptyRecord(ILogicalOperator inputOp, List<LogicalVariable> retainedVariables)
+ throws AlgebricksException {
+ LogicalOperatorTag tag = inputOp.getOperatorTag();
+ if (tag != LogicalOperatorTag.DATASOURCESCAN && tag != LogicalOperatorTag.UNNEST_MAP) {
+ return;
+ }
+
+ DataSource dataSource;
+ List<LogicalVariable> variables;
+ Mutable<ILogicalExpression> selectCondition;
if (inputOp.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
DataSourceScanOperator scan = (DataSourceScanOperator) inputOp;
- DatasetDataSource datasetDataSource = getDatasetDataSourceIfApplicable(scan);
- if (datasetDataSource != null) {
- //We know that we only need the count of objects. So return empty objects only
- LogicalVariable recordVar = datasetDataSource.getDataRecordVariable(scan.getVariables());
- /*
- * Set the root node as EMPTY_ROOT_NODE (i.e., no fields will be read from disk). We register the
- * dataset with EMPTY_ROOT_NODE so that we skip pushdowns on empty node.
- */
- builder.registerDataset(recordVar, RootExpectedSchemaNode.EMPTY_ROOT_NODE);
+ dataSource = (DataSource) scan.getDataSource();
+ variables = scan.getVariables();
+ selectCondition = scan.getSelectCondition();
+ } else {
+ UnnestMapOperator unnest = (UnnestMapOperator) inputOp;
+ dataSource = getDataSourceFromUnnestMapOperator(unnest);
+ variables = unnest.getVariables();
+ selectCondition = unnest.getSelectCondition();
+ }
+
+ DatasetDataSource datasetDataSource = getDatasetDataSourceIfApplicable(dataSource);
+
+ if (datasetDataSource == null) {
+ //Does not support pushdown
+ return;
+ }
+
+ Set<LogicalVariable> selectConditionVariables = new HashSet<>();
+ if (selectCondition != null) {
+ //Get the used variables for a select condition
+ selectCondition.getValue().getUsedVariables(selectConditionVariables);
+ }
+
+ //We know that we only need the count of objects. So return empty objects only
+ LogicalVariable recordVar = datasetDataSource.getDataRecordVariable(variables);
+
+ /*
+ * If the recordVar is not retained by an upper operator and not used by a select condition, then return empty
+ * record instead of the entire record.
+ */
+ if (!retainedVariables.contains(recordVar) && !selectConditionVariables.contains(recordVar)) {
+ /*
+ * Set the root node as EMPTY_ROOT_NODE (i.e., no fields will be read from disk). We register the
+ * dataset with EMPTY_ROOT_NODE so that we skip pushdowns on empty node.
+ */
+ builder.registerRoot(recordVar, RootExpectedSchemaNode.EMPTY_ROOT_NODE);
+ }
+
+ if (datasetDataSource.hasMeta()) {
+ //Do the same for meta
+ LogicalVariable metaVar = datasetDataSource.getMetaVariable(variables);
+ if (!retainedVariables.contains(metaVar)) {
+ builder.registerRoot(metaVar, RootExpectedSchemaNode.EMPTY_ROOT_NODE);
}
}
}
@@ -405,12 +544,6 @@
}
@Override
- public Void visitUnnestMapOperator(UnnestMapOperator op, Void arg) throws AlgebricksException {
- visitInputs(op);
- return null;
- }
-
- @Override
public Void visitLeftOuterUnnestMapOperator(LeftOuterUnnestMapOperator op, Void arg) throws AlgebricksException {
visitInputs(op);
return null;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/schema/ColumnFilterPathBuilderVisitor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/schema/ColumnFilterPathBuilderVisitor.java
new file mode 100644
index 0000000..3f83834
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/schema/ColumnFilterPathBuilderVisitor.java
@@ -0,0 +1,152 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.optimizer.rules.pushdown.schema;
+
+import java.util.Map;
+
+import org.apache.asterix.om.base.IAObject;
+import org.apache.asterix.om.types.AOrderedListType;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.types.IATypeVisitor;
+import org.apache.asterix.runtime.projection.FunctionCallInformation;
+import org.apache.asterix.runtime.projection.ProjectionFiltrationWarningFactoryProvider;
+
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
+public class ColumnFilterPathBuilderVisitor implements IExpectedSchemaNodeVisitor<IAType, IExpectedSchemaNode> {
+
+ private IAType type;
+ private Map<String, FunctionCallInformation> sourceInformationMap;
+ private int counter = 0;
+
+ public ARecordType buildPath(AnyExpectedSchemaNode anyNode, IAObject constant,
+ Map<String, FunctionCallInformation> sourceInformationMap, FunctionCallInformation compareFunctionInfo) {
+ this.type = rename(constant.getType());
+ this.sourceInformationMap = sourceInformationMap;
+
+ sourceInformationMap.put(type.getTypeName(), compareFunctionInfo);
+ return (ARecordType) anyNode.accept(this, anyNode);
+ }
+
+ @Override
+ public IAType visit(RootExpectedSchemaNode node, IExpectedSchemaNode arg) {
+ type = getRecordType(node, type, arg, getTypeName());
+ return type;
+ }
+
+ @Override
+ public IAType visit(ObjectExpectedSchemaNode node, IExpectedSchemaNode arg) {
+ type = getRecordType(node, type, arg, getTypeName());
+ sourceInformationMap.put(type.getTypeName(), createFunctionCallInformation(arg));
+ return node.getParent().accept(this, node);
+ }
+
+ @Override
+ public IAType visit(ArrayExpectedSchemaNode node, IExpectedSchemaNode arg) {
+ type = new AOrderedListType(type, getTypeName());
+ sourceInformationMap.put(type.getTypeName(), createFunctionCallInformation(arg));
+ return node.getParent().accept(this, node);
+ }
+
+ @Override
+ public IAType visit(UnionExpectedSchemaNode node, IExpectedSchemaNode arg) {
+ sourceInformationMap.put(type.getTypeName(), createFunctionCallInformation(arg));
+ return node.getParent().accept(this, arg);
+ }
+
+ @Override
+ public IAType visit(AnyExpectedSchemaNode node, IExpectedSchemaNode arg) {
+ return node.getParent().accept(this, node);
+ }
+
+ private static ARecordType getRecordType(ObjectExpectedSchemaNode objectNode, IAType childType,
+ IExpectedSchemaNode childNode, String typeName) {
+ String key = objectNode.getChildFieldName(childNode);
+ IAType[] fieldTypes = { childType };
+ String[] fieldNames = { key };
+
+ return new ARecordType(typeName, fieldNames, fieldTypes, false);
+ }
+
+ private String getTypeName() {
+ return "FilterPath" + counter++;
+ }
+
+ private FunctionCallInformation createFunctionCallInformation(IExpectedSchemaNode node) {
+ return new FunctionCallInformation(node.getFunctionName(), node.getSourceLocation(),
+ ProjectionFiltrationWarningFactoryProvider.TYPE_MISMATCH_FACTORY);
+ }
+
+ private IAType rename(IAType type) {
+ return new RenamedType(type, getTypeName());
+ }
+
+ private static class RenamedType implements IAType {
+ private static final long serialVersionUID = 992690669300951839L;
+ private final IAType originalType;
+ private final String name;
+
+ RenamedType(IAType originalType, String name) {
+ this.originalType = originalType;
+ this.name = name;
+ }
+
+ @Override
+ public IAType getType() {
+ return originalType.getType();
+ }
+
+ @Override
+ public boolean deepEqual(IAObject obj) {
+ return originalType.deepEqual(obj);
+ }
+
+ @Override
+ public int hash() {
+ return originalType.hash();
+ }
+
+ @Override
+ public ATypeTag getTypeTag() {
+ return originalType.getTypeTag();
+ }
+
+ @Override
+ public String getDisplayName() {
+ return originalType.getDisplayName();
+ }
+
+ @Override
+ public String getTypeName() {
+ return name;
+ }
+
+ @Override
+ public <R, T> R accept(IATypeVisitor<R, T> visitor, T arg) {
+ return visitor.visitFlat(this, arg);
+ }
+
+ @Override
+ public ObjectNode toJSON() {
+ return originalType.toJSON();
+ }
+ }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/schema/ObjectExpectedSchemaNode.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/schema/ObjectExpectedSchemaNode.java
index 02307c0..69812a3 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/schema/ObjectExpectedSchemaNode.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/schema/ObjectExpectedSchemaNode.java
@@ -22,6 +22,8 @@
import java.util.Map;
import java.util.Set;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.IAType;
import org.apache.hyracks.api.exceptions.SourceLocation;
public class ObjectExpectedSchemaNode extends AbstractComplexExpectedSchemaNode {
@@ -37,9 +39,8 @@
return children.entrySet();
}
- public IExpectedSchemaNode addChild(String fieldName, IExpectedSchemaNode child) {
+ public void addChild(String fieldName, IExpectedSchemaNode child) {
children.put(fieldName, child);
- return child;
}
@Override
@@ -54,9 +55,22 @@
@Override
public void replaceChild(IExpectedSchemaNode oldNode, IExpectedSchemaNode newNode) {
+ String fieldName = getChildFieldName(oldNode);
+ children.replace(fieldName, newNode);
+ }
+
+ protected IAType getType(IAType childType, IExpectedSchemaNode childNode, String typeName) {
+ String key = getChildFieldName(childNode);
+ IAType[] fieldTypes = { childType };
+ String[] fieldNames = { key };
+
+ return new ARecordType("typeName", fieldNames, fieldTypes, false);
+ }
+
+ protected String getChildFieldName(IExpectedSchemaNode requestedChild) {
String key = null;
for (Map.Entry<String, IExpectedSchemaNode> child : children.entrySet()) {
- if (child.getValue() == oldNode) {
+ if (child.getValue() == requestedChild) {
key = child.getKey();
break;
}
@@ -64,8 +78,8 @@
if (key == null) {
//this should not happen
- throw new IllegalStateException("Node " + oldNode.getType() + " is not a child");
+ throw new IllegalStateException("Node " + requestedChild.getType() + " is not a child");
}
- children.replace(key, newNode);
+ return key;
}
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/QueryIndexDatasource.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/QueryIndexDatasource.java
index 52cd17c..ee018f9 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/QueryIndexDatasource.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/QueryIndexDatasource.java
@@ -37,7 +37,7 @@
import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSource;
import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSourcePropertiesProvider;
-import org.apache.hyracks.algebricks.core.algebra.metadata.IProjectionInfo;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IProjectionFiltrationInfo;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
import org.apache.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
@@ -74,9 +74,7 @@
ARecordType type = (ARecordType) iType;
IAType[] fieldTypes = type.getFieldTypes();
schemaTypes = new IAType[fieldTypes.length];
- for (int i = 0; i < schemaTypes.length; i++) {
- schemaTypes[i] = fieldTypes[i];
- }
+ System.arraycopy(fieldTypes, 0, schemaTypes, 0, schemaTypes.length);
}
@Override
@@ -104,7 +102,8 @@
List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars,
ITupleFilterFactory tupleFilterFactory, long outputLimit, IOperatorSchema opSchema,
IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig,
- IProjectionInfo<?> projectionInfo) throws AlgebricksException {
+ IProjectionFiltrationInfo<?> projectionInfo, IProjectionFiltrationInfo<?> metaProjectionInfo)
+ throws AlgebricksException {
return metadataProvider.buildBtreeRuntime(jobSpec, opSchema, typeEnv, context, true, false, null, ds, indexName,
null, null, true, true, false, null, null, null, tupleFilterFactory, outputLimit, false, false,
DefaultTupleProjectorFactory.INSTANCE);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/io/PersistedResourceRegistry.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/io/PersistedResourceRegistry.java
index 4850bf6..82c70a3 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/io/PersistedResourceRegistry.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/io/PersistedResourceRegistry.java
@@ -25,6 +25,7 @@
import java.util.Objects;
import java.util.Optional;
+import org.apache.asterix.column.ColumnManagerFactory;
import org.apache.asterix.common.context.AsterixVirtualBufferCacheProvider;
import org.apache.asterix.common.context.CorrelatedPrefixMergePolicyFactory;
import org.apache.asterix.common.context.DatasetInfoProvider;
@@ -103,6 +104,7 @@
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.storage.am.common.data.PointablePrimitiveValueProviderFactory;
import org.apache.hyracks.storage.am.common.freepage.AppendOnlyLinkedMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.lsm.btree.column.dataflow.LSMColumnBTreeLocalResource;
import org.apache.hyracks.storage.am.lsm.btree.dataflow.ExternalBTreeLocalResource;
import org.apache.hyracks.storage.am.lsm.btree.dataflow.ExternalBTreeWithBuddyLocalResource;
import org.apache.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeLocalResource;
@@ -162,6 +164,7 @@
registeredClasses.put("ExternalBTreeLocalResource", ExternalBTreeLocalResource.class);
registeredClasses.put("ExternalBTreeWithBuddyLocalResource", ExternalBTreeWithBuddyLocalResource.class);
registeredClasses.put("ExternalRTreeLocalResource", ExternalRTreeLocalResource.class);
+ registeredClasses.put("LSMColumnBTreeLocalResource", LSMColumnBTreeLocalResource.class);
// ILSMMergePolicyFactory
registeredClasses.put("NoMergePolicyFactory", NoMergePolicyFactory.class);
@@ -305,6 +308,9 @@
//External Libraries
registeredClasses.put("LibraryDescriptor", LibraryDescriptor.class);
+
+ //IColumnManagerFactory
+ registeredClasses.put("ColumnManagerFactory", ColumnManagerFactory.class);
}
@Override
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 b0e5885..775672c 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
@@ -75,6 +75,7 @@
import org.apache.asterix.common.config.DatasetConfig.IndexType;
import org.apache.asterix.common.config.DatasetConfig.TransactionState;
import org.apache.asterix.common.config.GlobalConfig;
+import org.apache.asterix.common.config.StorageProperties;
import org.apache.asterix.common.dataflow.ICcApplicationContext;
import org.apache.asterix.common.exceptions.ACIDException;
import org.apache.asterix.common.exceptions.AsterixException;
@@ -165,6 +166,7 @@
import org.apache.asterix.metadata.MetadataManager;
import org.apache.asterix.metadata.MetadataTransactionContext;
import org.apache.asterix.metadata.bootstrap.MetadataBuiltinEntities;
+import org.apache.asterix.metadata.dataset.DatasetFormatInfo;
import org.apache.asterix.metadata.dataset.hints.DatasetHints;
import org.apache.asterix.metadata.dataset.hints.DatasetHints.DatasetNodegroupCardinalityHint;
import org.apache.asterix.metadata.declared.MetadataProvider;
@@ -654,7 +656,7 @@
protected static void validateCompactionPolicy(String compactionPolicy,
Map<String, String> compactionPolicyProperties, MetadataTransactionContext mdTxnCtx,
- boolean isExternalDataset, SourceLocation sourceLoc) throws CompilationException, Exception {
+ boolean isExternalDataset, SourceLocation sourceLoc) throws Exception {
CompactionPolicy compactionPolicyEntity = MetadataManager.INSTANCE.getCompactionPolicy(mdTxnCtx,
MetadataConstants.METADATA_DATAVERSE_NAME, compactionPolicy);
if (compactionPolicyEntity == null) {
@@ -755,6 +757,10 @@
Dataset dataset = null;
Datatype itemTypeEntity = null, metaItemTypeEntity = null;
boolean itemTypeAdded = false, metaItemTypeAdded = false;
+
+ StorageProperties storageProperties = metadataProvider.getStorageProperties();
+ DatasetFormatInfo datasetFormatInfo = dd.getDatasetFormatInfo(storageProperties.getColumnMaxTupleCount(),
+ storageProperties.getColumnFreeSpaceTolerance());
try {
// Check if the dataverse exists
Dataverse dv = MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dataverseName);
@@ -861,7 +867,7 @@
dataset = new Dataset(dataverseName, datasetName, itemTypeDataverseName, itemTypeName,
metaItemTypeDataverseName, metaItemTypeName, ngName, compactionPolicy, compactionPolicyProperties,
datasetDetails, dd.getHints(), dsType, DatasetIdFactory.generateDatasetId(),
- MetadataUtil.PENDING_ADD_OP, compressionScheme);
+ MetadataUtil.PENDING_ADD_OP, compressionScheme, datasetFormatInfo);
MetadataManager.INSTANCE.addDataset(metadataProvider.getMetadataTxnContext(), dataset);
if (itemTypeIsInline) {
@@ -1443,15 +1449,12 @@
"full-text filter type is null");
}
- switch (filterType) {
- case FIELD_TYPE_STOPWORDS: {
- filterDescriptor = new StopwordsFullTextFilterDescriptor(dataverseName,
- stmtCreateFilter.getFilterName(), stmtCreateFilter.getStopwordsList());
- break;
- }
- default:
- throw new CompilationException(ErrorCode.COMPILATION_ERROR, stmtCreateFilter.getSourceLocation(),
- "Unexpected full-text filter type: " + filterType);
+ if (FIELD_TYPE_STOPWORDS.equals(filterType)) {
+ filterDescriptor = new StopwordsFullTextFilterDescriptor(dataverseName, stmtCreateFilter.getFilterName(),
+ stmtCreateFilter.getStopwordsList());
+ } else {
+ throw new CompilationException(ErrorCode.COMPILATION_ERROR, stmtCreateFilter.getSourceLocation(),
+ "Unexpected full-text filter type: " + filterType);
}
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
@@ -1525,8 +1528,7 @@
}
}
- ImmutableList.Builder<IFullTextFilterDescriptor> filterDescriptorsBuilder =
- ImmutableList.<IFullTextFilterDescriptor> builder();
+ ImmutableList.Builder<IFullTextFilterDescriptor> filterDescriptorsBuilder = ImmutableList.builder();
for (String filterName : filterNames) {
FullTextFilterMetadataEntity filterMetadataEntity =
MetadataManager.INSTANCE.getFullTextFilter(mdTxnCtx, dataverseName, filterName);
@@ -3966,9 +3968,8 @@
ActiveEntityEventsListener listener =
(ActiveEntityEventsListener) activeNotificationHandler.getListener(feedId);
if (listener != null && listener.getState() != ActivityState.STOPPED) {
- throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
- "Feed " + feedId + " is currently active and connected to the following " + dataset(PLURAL) + "\n"
- + listener.toString());
+ throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc, "Feed " + feedId
+ + " is currently active and connected to the following " + dataset(PLURAL) + "\n" + listener);
} else if (listener != null) {
listener.unregister();
}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/pushdown-plans/pushdown-plans.06.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/pushdown-plans/pushdown-plans.06.query.sqlpp
index 93b09d0..e8f7963 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/pushdown-plans/pushdown-plans.06.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/pushdown-plans/pushdown-plans.06.query.sqlpp
@@ -17,7 +17,7 @@
* under the License.
*/
/*
-* Description : Pushdown "p.entities.urls"
+* Description : Pushdown "p.entities.urls[*].display_url"
* Expected Res : Success
* Date : July 23th 2021
*/
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 389b120..4e6a68e 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
@@ -12,6 +12,7 @@
"compiler\.arrayindex" : true,
"compiler.batch.lookup" : false,
"compiler.cbo" : false,
+ "compiler.column.filter" : false,
"compiler\.external\.field\.pushdown" : true,
"compiler.forcejoinorder" : false,
"compiler\.framesize" : 32768,
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 e68df5f..d025d7a 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
@@ -12,6 +12,7 @@
"compiler\.arrayindex" : true,
"compiler.batch.lookup" : false,
"compiler.cbo" : false,
+ "compiler.column.filter" : false,
"compiler\.external\.field\.pushdown" : true,
"compiler.forcejoinorder" : false,
"compiler\.framesize" : 32768,
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 a6fa211..7f2dcb1 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
@@ -12,6 +12,7 @@
"compiler\.arrayindex" : true,
"compiler.batch.lookup" : false,
"compiler.cbo" : false,
+ "compiler.column.filter" : false,
"compiler\.external\.field\.pushdown" : true,
"compiler.forcejoinorder" : false,
"compiler\.framesize" : 32768,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.06.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.06.plan
index 7a7a890..42508a1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.06.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.06.plan
@@ -18,7 +18,7 @@
-- ASSIGN |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$p] <- test.ParquetDataset1 condition (gt(sql-count($$p.getField("entities").getField("urls")), 10)) limit 10 project ({entities:{urls:any}}) [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ data-scan []<-[$$p] <- test.ParquetDataset1 condition (gt(sql-count($$p.getField("entities").getField("urls")), 10)) limit 10 project ({entities:{urls:[{display_url:any}]}}) [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
-- DATASOURCE_SCAN |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/ColumnManager.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/ColumnManager.java
new file mode 100644
index 0000000..c1402c8
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/ColumnManager.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.column;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.asterix.column.operation.lsm.flush.FlushColumnMetadata;
+import org.apache.asterix.column.operation.lsm.merge.MergeColumnTupleProjector;
+import org.apache.asterix.column.operation.lsm.merge.MergeColumnWriteMetadata;
+import org.apache.asterix.column.values.IColumnValuesReaderFactory;
+import org.apache.asterix.column.values.IColumnValuesWriterFactory;
+import org.apache.asterix.column.values.reader.ColumnValueReaderFactory;
+import org.apache.asterix.column.values.writer.ColumnValuesWriterFactory;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnManager;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnMetadata;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnTupleIterator;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnWriteMultiPageOp;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.projection.IColumnTupleProjector;
+
+public final class ColumnManager implements IColumnManager {
+ private final ARecordType datasetType;
+ private final ARecordType metaType;
+ private final List<List<String>> primaryKeys;
+ private final List<Integer> keySourceIndicator;
+ private final MergeColumnTupleProjector mergeColumnTupleProjector;
+
+ ColumnManager(ARecordType datasetType, ARecordType metaType, List<List<String>> primaryKeys,
+ List<Integer> keySourceIndicator) {
+ this.datasetType = datasetType;
+ this.metaType = metaType;
+ this.primaryKeys = primaryKeys;
+ this.keySourceIndicator = keySourceIndicator;
+ IColumnValuesReaderFactory readerFactory = new ColumnValueReaderFactory();
+ mergeColumnTupleProjector =
+ new MergeColumnTupleProjector(datasetType, metaType, primaryKeys.size(), readerFactory);
+ }
+
+ @Override
+ public IColumnMetadata activate() throws HyracksDataException {
+ Mutable<IColumnWriteMultiPageOp> multiPageOpRef = new MutableObject<>();
+ IColumnValuesWriterFactory factory = new ColumnValuesWriterFactory(multiPageOpRef);
+ return new FlushColumnMetadata(datasetType, metaType, primaryKeys, keySourceIndicator, factory, multiPageOpRef);
+ }
+
+ @Override
+ public IColumnMetadata activate(IValueReference metadata) throws HyracksDataException {
+ try {
+ Mutable<IColumnWriteMultiPageOp> multiPageOpRef = new MutableObject<>();
+ IColumnValuesWriterFactory writerFactory = new ColumnValuesWriterFactory(multiPageOpRef);
+ return FlushColumnMetadata.create(datasetType, metaType, primaryKeys, keySourceIndicator, writerFactory,
+ multiPageOpRef, metadata);
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ }
+
+ @Override
+ public IColumnMetadata createMergeColumnMetadata(IValueReference metadata,
+ List<IColumnTupleIterator> componentsTuples) throws HyracksDataException {
+ try {
+ return MergeColumnWriteMetadata.create(datasetType, metaType, primaryKeys.size(), new MutableObject<>(),
+ metadata, componentsTuples);
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+
+ }
+
+ @Override
+ public IColumnTupleProjector getMergeColumnProjector() {
+ return mergeColumnTupleProjector;
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/ColumnManagerFactory.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/ColumnManagerFactory.java
new file mode 100644
index 0000000..8e35416
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/ColumnManagerFactory.java
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.column;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.asterix.column.operation.lsm.flush.FlushColumnTupleReaderWriterFactory;
+import org.apache.asterix.column.operation.lsm.load.LoadColumnTupleReaderWriterFactory;
+import org.apache.asterix.column.operation.lsm.merge.MergeColumnTupleReaderWriterFactory;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.AbstractColumnTupleReaderWriterFactory;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnManager;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnManagerFactory;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
+public final class ColumnManagerFactory implements IColumnManagerFactory {
+ private static final long serialVersionUID = -5003159552950739235L;
+ private final ARecordType datasetType;
+ private final ARecordType metaType;
+ private final List<List<String>> primaryKeys;
+ private final List<Integer> keySourceIndicator;
+ private final int pageSize;
+ private final int maxTupleCount;
+ private final float tolerance;
+
+ public ColumnManagerFactory(ARecordType datasetType, ARecordType metaType, List<List<String>> primaryKeys,
+ List<Integer> keySourceIndicator, int pageSize, int maxTupleCount, float tolerance) {
+ this.pageSize = pageSize;
+ this.maxTupleCount = maxTupleCount;
+ this.tolerance = tolerance;
+
+ this.datasetType = datasetType;
+ if (keySourceIndicator.size() != 1) {
+ throw new UnsupportedOperationException(
+ "Primary keys split between meta-type and datasetType is not supported");
+ }
+ this.keySourceIndicator = keySourceIndicator;
+ this.metaType = metaType;
+ this.primaryKeys = primaryKeys;
+ }
+
+ @Override
+ public IColumnManager createColumnManager() {
+ return new ColumnManager(datasetType, metaType, primaryKeys, keySourceIndicator);
+ }
+
+ @Override
+ public AbstractColumnTupleReaderWriterFactory getLoadColumnTupleReaderWriterFactory() {
+ return new LoadColumnTupleReaderWriterFactory(pageSize, maxTupleCount, tolerance);
+ }
+
+ @Override
+ public AbstractColumnTupleReaderWriterFactory getFlushColumnTupleReaderWriterFactory() {
+ return new FlushColumnTupleReaderWriterFactory(pageSize, maxTupleCount, tolerance);
+ }
+
+ @Override
+ public AbstractColumnTupleReaderWriterFactory createMergeColumnTupleReaderWriterFactory() {
+ return new MergeColumnTupleReaderWriterFactory(pageSize, maxTupleCount, tolerance);
+ }
+
+ @Override
+ public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+ ObjectNode json = registry.getClassIdentifier(getClass(), serialVersionUID);
+ json.putPOJO("datasetType", datasetType.toJson(registry));
+ if (metaType != null) {
+ json.putPOJO("metaType", metaType.toJson(registry));
+ }
+
+ json.put("pageSize", pageSize);
+ json.put("maxTupleCount", maxTupleCount);
+ json.put("tolerance", tolerance);
+
+ ArrayNode primaryKeysArray = json.putArray("primaryKeys");
+ for (List<String> primaryKey : primaryKeys) {
+ ArrayNode primaryKeyArray = primaryKeysArray.addArray();
+ for (String path : primaryKey) {
+ primaryKeyArray.add(path);
+ }
+ }
+
+ ArrayNode keySourceIndicatorNode = json.putArray("keySourceIndicator");
+ for (int keySourceIndicatorInt : keySourceIndicator) {
+ keySourceIndicatorNode.add(keySourceIndicatorInt);
+ }
+ return json;
+ }
+
+ public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json)
+ throws HyracksDataException {
+ ARecordType datasetType = (ARecordType) registry.deserialize(json.get("datasetType"));
+ JsonNode metaItemTypeNode = json.get("metaType");
+ ARecordType metaType = null;
+ if (metaItemTypeNode != null) {
+ metaType = (ARecordType) registry.deserialize(metaItemTypeNode);
+ }
+
+ int pageSize = json.get("pageSize").asInt();
+ int maxTupleCount = json.get("maxTupleCount").asInt();
+ float tolerance = (float) json.get("tolerance").asDouble();
+
+ List<List<String>> primaryKeys = new ArrayList<>();
+ ArrayNode primaryKeysNode = (ArrayNode) json.get("primaryKeys");
+ for (int i = 0; i < primaryKeysNode.size(); i++) {
+ List<String> primaryKey = new ArrayList<>();
+ ArrayNode primaryKeyNode = (ArrayNode) primaryKeysNode.get(i);
+ for (int j = 0; j < primaryKeyNode.size(); j++) {
+ primaryKey.add(primaryKeyNode.get(i).asText());
+ }
+ primaryKeys.add(primaryKey);
+ }
+
+ List<Integer> keySourceIndicator = new ArrayList<>();
+ ArrayNode keySourceIndicatorNode = (ArrayNode) json.get("keySourceIndicator");
+ for (int i = 0; i < keySourceIndicatorNode.size(); i++) {
+ keySourceIndicator.add(keySourceIndicatorNode.get(i).asInt());
+ }
+
+ return new ColumnManagerFactory(datasetType, metaType, primaryKeys, keySourceIndicator, pageSize, maxTupleCount,
+ tolerance);
+ }
+
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/AssemblerBuilderVisitor.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/AssemblerBuilderVisitor.java
new file mode 100644
index 0000000..969a091
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/AssemblerBuilderVisitor.java
@@ -0,0 +1,223 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.column.assembler;
+
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.asterix.column.assembler.value.IValueGetter;
+import org.apache.asterix.column.assembler.value.IValueGetterFactory;
+import org.apache.asterix.column.metadata.schema.AbstractSchemaNode;
+import org.apache.asterix.column.metadata.schema.ISchemaNodeVisitor;
+import org.apache.asterix.column.metadata.schema.ObjectSchemaNode;
+import org.apache.asterix.column.metadata.schema.UnionSchemaNode;
+import org.apache.asterix.column.metadata.schema.collection.AbstractCollectionSchemaNode;
+import org.apache.asterix.column.metadata.schema.primitive.PrimitiveSchemaNode;
+import org.apache.asterix.column.operation.query.QueryColumnMetadata;
+import org.apache.asterix.column.values.IColumnValuesReader;
+import org.apache.asterix.column.values.IColumnValuesReaderFactory;
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AbstractCollectionType;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+
+import it.unimi.dsi.fastutil.ints.IntArrayList;
+import it.unimi.dsi.fastutil.ints.IntList;
+
+public class AssemblerBuilderVisitor implements ISchemaNodeVisitor<AbstractValueAssembler, AssemblerInfo> {
+ private static final BitSet NO_DECLARED_FIELDS = new BitSet(0);
+ private final QueryColumnMetadata columnMetadata;
+ private final IColumnValuesReaderFactory readerFactory;
+ private final List<AbstractPrimitiveValueAssembler> valueAssemblers;
+ private final IValueGetterFactory valueGetterFactory;
+ private AbstractValueAssembler rootAssembler;
+
+ //Recursion info
+ private final IntList delimiters;
+ private int level;
+
+ public AssemblerBuilderVisitor(QueryColumnMetadata columnMetadata, IColumnValuesReaderFactory readerFactory,
+ IValueGetterFactory valueGetterFactory) {
+ this.columnMetadata = columnMetadata;
+ this.readerFactory = readerFactory;
+ this.valueGetterFactory = valueGetterFactory;
+ valueAssemblers = new ArrayList<>();
+ delimiters = new IntArrayList();
+ }
+
+ public List<AbstractPrimitiveValueAssembler> createValueAssemblers(AbstractSchemaNode requestedSchema,
+ ARecordType declaredType) throws HyracksDataException {
+ EmptyAssembler root = new EmptyAssembler();
+ AssemblerInfo info = new AssemblerInfo(declaredType, root);
+ level = 0;
+ rootAssembler = requestedSchema.accept(this, info);
+ return valueAssemblers;
+ }
+
+ public AbstractValueAssembler getRootAssembler() {
+ return rootAssembler;
+ }
+
+ @Override
+ public AbstractValueAssembler visit(ObjectSchemaNode objectNode, AssemblerInfo info) throws HyracksDataException {
+ ObjectValueAssembler objectAssembler = new ObjectValueAssembler(level, info);
+ level++;
+
+ BitSet declaredFields = handleDeclaredFields(objectNode, info, objectAssembler);
+ IntList childrenFieldNameIndexes = objectNode.getChildrenFieldNameIndexes();
+ if (declaredFields.length() < childrenFieldNameIndexes.size()) {
+ //Open fields are requested
+ for (int i = 0; i < childrenFieldNameIndexes.size(); i++) {
+ int fieldNameIndex = childrenFieldNameIndexes.getInt(i);
+ AbstractSchemaNode childNode = objectNode.getChild(fieldNameIndex);
+ if (childNode.getTypeTag() != ATypeTag.MISSING && !declaredFields.get(fieldNameIndex)) {
+ IAType childType = getChildType(childNode, BuiltinType.ANY);
+ IValueReference fieldName = columnMetadata.getFieldNamesDictionary().getFieldName(fieldNameIndex);
+ //The last child should be a delegate
+ boolean delegate = i == childrenFieldNameIndexes.size() - 1;
+ AssemblerInfo childInfo = new AssemblerInfo(childType, objectAssembler, delegate, fieldName);
+ childNode.accept(this, childInfo);
+ }
+ }
+ }
+
+ level--;
+ return objectAssembler;
+ }
+
+ private BitSet handleDeclaredFields(ObjectSchemaNode objectNode, AssemblerInfo info,
+ ObjectValueAssembler objectAssembler) throws HyracksDataException {
+ ARecordType declaredType = (ARecordType) info.getDeclaredType();
+ if (declaredType == DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE) {
+ return NO_DECLARED_FIELDS;
+ }
+ BitSet processedFields = new BitSet();
+ String[] declaredFieldNames = declaredType.getFieldNames();
+ IAType[] declaredFieldTypes = declaredType.getFieldTypes();
+
+ // The last child of a declared field can be a delegate iff all requested fields are declared
+ boolean containsDelegate = objectNode.getChildren().size() == declaredFieldTypes.length;
+ for (int i = 0; i < declaredFieldTypes.length; i++) {
+ String fieldName = declaredFieldNames[i];
+ int fieldNameIndex = columnMetadata.getFieldNamesDictionary().getFieldNameIndex(fieldName);
+ //Check if the declared field was requested
+ AbstractSchemaNode childNode = objectNode.getChild(fieldNameIndex);
+ if (childNode.getTypeTag() != ATypeTag.MISSING) {
+ IAType childType = getChildType(childNode, declaredFieldTypes[i]);
+ processedFields.set(fieldNameIndex);
+ boolean delegate = containsDelegate && i == declaredFieldTypes.length - 1;
+ AssemblerInfo childInfo = new AssemblerInfo(childType, objectAssembler, delegate, i);
+ childNode.accept(this, childInfo);
+ }
+ }
+ return processedFields;
+ }
+
+ @Override
+ public AbstractValueAssembler visit(AbstractCollectionSchemaNode collectionNode, AssemblerInfo info)
+ throws HyracksDataException {
+ AbstractCollectionType declaredType = (AbstractCollectionType) info.getDeclaredType();
+ AbstractSchemaNode itemNode = collectionNode.getItemNode();
+
+ ArrayValueAssembler arrayAssembler = itemNode.getTypeTag() == ATypeTag.UNION
+ ? new ArrayWithUnionValueAssembler(level, info, valueAssemblers.size(), itemNode)
+ : new ArrayValueAssembler(level, info, valueAssemblers.size());
+ delimiters.add(level - 1);
+ level++;
+
+ IAType itemDeclaredType = getChildType(itemNode, declaredType.getItemType());
+ AssemblerInfo itemInfo = new AssemblerInfo(itemDeclaredType, arrayAssembler, false);
+ itemNode.accept(this, itemInfo);
+
+ //Add the array assembler to the last repeated value assembler
+ RepeatedPrimitiveValueAssembler repeatedAssembler =
+ (RepeatedPrimitiveValueAssembler) valueAssemblers.get(valueAssemblers.size() - 1);
+ repeatedAssembler.addArray(arrayAssembler);
+
+ level--;
+ delimiters.removeInt(delimiters.size() - 1);
+ return arrayAssembler;
+ }
+
+ @Override
+ public AbstractValueAssembler visit(UnionSchemaNode unionNode, AssemblerInfo info) throws HyracksDataException {
+ /*
+ * UnionSchemaNode does not actually exist. We know the parent of the union could have items of multiple types.
+ * Thus, the union's parent is the actual parent for all the union types
+ */
+ Collection<AbstractSchemaNode> children = unionNode.getChildren().values();
+ int index = 0;
+ for (AbstractSchemaNode node : children) {
+ IAType unionDeclaredType = getChildType(node, info.getDeclaredType());
+ boolean delegate = info.isDelegate() && index++ == children.size() - 1;
+ AssemblerInfo unionInfo = new AssemblerInfo(unionDeclaredType, info.getParent(), delegate,
+ info.getFieldName(), info.getFieldIndex(), true);
+ node.accept(this, unionInfo);
+ }
+ return info.getParent();
+ }
+
+ @Override
+ public AbstractValueAssembler visit(PrimitiveSchemaNode primitiveNode, AssemblerInfo info) {
+ AbstractPrimitiveValueAssembler assembler;
+ IValueGetter valueGetter = valueGetterFactory.createValueGetter(primitiveNode.getTypeTag());
+ if (!delimiters.isEmpty()) {
+ IColumnValuesReader reader = readerFactory.createValueReader(primitiveNode.getTypeTag(),
+ primitiveNode.getColumnIndex(), level, getDelimiters());
+ assembler = new RepeatedPrimitiveValueAssembler(level, info, reader, valueGetter);
+
+ } else {
+ IColumnValuesReader reader = readerFactory.createValueReader(primitiveNode.getTypeTag(),
+ primitiveNode.getColumnIndex(), level, primitiveNode.isPrimaryKey());
+ assembler = new PrimitiveValueAssembler(level, info, reader, valueGetter);
+ }
+ valueAssemblers.add(assembler);
+ return assembler;
+ }
+
+ private int[] getDelimiters() {
+ int numOfDelimiters = delimiters.size();
+ int[] reversed = new int[numOfDelimiters];
+ for (int i = 0; i < numOfDelimiters; i++) {
+ reversed[i] = delimiters.getInt(numOfDelimiters - i - 1);
+ }
+ return reversed;
+ }
+
+ private IAType getChildType(AbstractSchemaNode childNode, IAType childType) {
+ if (childType.getTypeTag() != ATypeTag.ANY) {
+ return childType;
+ }
+ ATypeTag childTypeTag = childNode.getTypeTag();
+ if (childTypeTag == ATypeTag.UNION) {
+ //Union type could return any type
+ return BuiltinType.ANY;
+ } else if (childTypeTag.isDerivedType()) {
+ return DefaultOpenFieldType.getDefaultOpenFieldType(childTypeTag);
+ } else {
+ return BuiltinType.getBuiltinType(childTypeTag);
+ }
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/query/ColumnAssembler.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/query/ColumnAssembler.java
new file mode 100644
index 0000000..c329d67
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/query/ColumnAssembler.java
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.column.operation.query;
+
+import java.util.List;
+
+import org.apache.asterix.column.assembler.AbstractPrimitiveValueAssembler;
+import org.apache.asterix.column.assembler.AssemblerBuilderVisitor;
+import org.apache.asterix.column.assembler.ObjectValueAssembler;
+import org.apache.asterix.column.assembler.value.IValueGetterFactory;
+import org.apache.asterix.column.bytes.stream.in.AbstractBytesInputStream;
+import org.apache.asterix.column.metadata.schema.AbstractSchemaNode;
+import org.apache.asterix.column.values.IColumnValuesReaderFactory;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+
+public final class ColumnAssembler {
+ private final List<AbstractPrimitiveValueAssembler> assemblers;
+ private final ObjectValueAssembler rootAssembler;
+ private int numberOfTuples;
+ private int tupleIndex;
+
+ public ColumnAssembler(AbstractSchemaNode node, ARecordType declaredType, QueryColumnMetadata columnMetadata,
+ IColumnValuesReaderFactory readerFactory, IValueGetterFactory valueGetterFactory)
+ throws HyracksDataException {
+ AssemblerBuilderVisitor builderVisitor =
+ new AssemblerBuilderVisitor(columnMetadata, readerFactory, valueGetterFactory);
+ assemblers = builderVisitor.createValueAssemblers(node, declaredType);
+ rootAssembler = (ObjectValueAssembler) builderVisitor.getRootAssembler();
+ }
+
+ public void reset(int numberOfTuples) {
+ this.numberOfTuples = numberOfTuples;
+ tupleIndex = 0;
+ }
+
+ public void resetColumn(AbstractBytesInputStream stream, int startIndex, int ordinal) throws HyracksDataException {
+ assemblers.get(ordinal).reset(stream, startIndex, numberOfTuples);
+ }
+
+ public int getColumnIndex(int ordinal) {
+ return assemblers.get(ordinal).getColumnIndex();
+ }
+
+ public boolean hasNext() {
+ return tupleIndex < numberOfTuples;
+ }
+
+ public IValueReference nextValue() throws HyracksDataException {
+ rootAssembler.start();
+ if (tupleIndex == numberOfTuples) {
+ rootAssembler.end();
+ //return empty record
+ return rootAssembler.getValue();
+ }
+
+ int index = 0;
+ while (index < assemblers.size()) {
+ AbstractPrimitiveValueAssembler assembler = assemblers.get(index);
+ int groupIndex = assembler.next();
+ if (groupIndex != AbstractPrimitiveValueAssembler.NEXT_ASSEMBLER) {
+ index = groupIndex;
+ } else {
+ index++;
+ }
+ }
+
+ tupleIndex++;
+ rootAssembler.end();
+ return rootAssembler.getValue();
+ }
+
+ public int getNumberOfColumns() {
+ return assemblers.size();
+ }
+
+ public void skip(int count) throws HyracksDataException {
+ for (int i = 0; i < assemblers.size(); i++) {
+ assemblers.get(i).skip(count);
+ }
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/query/QueryColumnMetadata.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/query/QueryColumnMetadata.java
new file mode 100644
index 0000000..6c51240
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/query/QueryColumnMetadata.java
@@ -0,0 +1,205 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.column.operation.query;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.asterix.column.assembler.value.IValueGetterFactory;
+import org.apache.asterix.column.metadata.AbstractColumnImmutableReadMetadata;
+import org.apache.asterix.column.metadata.FieldNamesDictionary;
+import org.apache.asterix.column.metadata.schema.AbstractSchemaNode;
+import org.apache.asterix.column.metadata.schema.ObjectSchemaNode;
+import org.apache.asterix.column.metadata.schema.visitor.SchemaClipperVisitor;
+import org.apache.asterix.column.values.IColumnValuesReader;
+import org.apache.asterix.column.values.IColumnValuesReaderFactory;
+import org.apache.asterix.column.values.reader.filter.FilterAccessorProvider;
+import org.apache.asterix.column.values.reader.filter.IColumnFilterEvaluator;
+import org.apache.asterix.column.values.reader.filter.IColumnFilterEvaluatorFactory;
+import org.apache.asterix.column.values.reader.filter.IColumnFilterValueAccessor;
+import org.apache.asterix.column.values.reader.filter.evaluator.FalseColumnFilterEvaluator;
+import org.apache.asterix.column.values.reader.filter.evaluator.TrueColumnFilterEvaluator;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.runtime.projection.DataProjectionFiltrationInfo;
+import org.apache.asterix.runtime.projection.FunctionCallInformation;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.data.std.primitive.IntegerPointable;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.AbstractColumnTupleReader;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+/**
+ * Query column metadata which is used to resolve the requested values in a query
+ */
+public class QueryColumnMetadata extends AbstractColumnImmutableReadMetadata {
+ private static final Logger LOGGER = LogManager.getLogger();
+ private final FieldNamesDictionary fieldNamesDictionary;
+ private final IColumnValuesReader[] primaryKeyReaders;
+ private final IColumnFilterEvaluator filterEvaluator;
+ private final List<IColumnFilterValueAccessor> filterValueAccessors;
+
+ protected final ColumnAssembler assembler;
+
+ protected QueryColumnMetadata(ARecordType datasetType, ARecordType metaType,
+ IColumnValuesReader[] primaryKeyReaders, IValueReference serializedMetadata,
+ FieldNamesDictionary fieldNamesDictionary, ObjectSchemaNode root, IColumnValuesReaderFactory readerFactory,
+ IValueGetterFactory valueGetterFactory, IColumnFilterEvaluator filterEvaluator,
+ List<IColumnFilterValueAccessor> filterValueAccessors) throws HyracksDataException {
+ super(datasetType, metaType, primaryKeyReaders.length, serializedMetadata, -1);
+ this.fieldNamesDictionary = fieldNamesDictionary;
+ this.assembler = new ColumnAssembler(root, datasetType, this, readerFactory, valueGetterFactory);
+ this.primaryKeyReaders = primaryKeyReaders;
+ this.filterEvaluator = filterEvaluator;
+ this.filterValueAccessors = filterValueAccessors;
+ }
+
+ public final ColumnAssembler getAssembler() {
+ return assembler;
+ }
+
+ public final FieldNamesDictionary getFieldNamesDictionary() {
+ return fieldNamesDictionary;
+ }
+
+ public final IColumnValuesReader[] getPrimaryKeyReaders() {
+ return primaryKeyReaders;
+ }
+
+ public IColumnFilterEvaluator getFilterEvaluator() {
+ return filterEvaluator;
+ }
+
+ public List<IColumnFilterValueAccessor> getFilterValueAccessors() {
+ return filterValueAccessors;
+ }
+
+ /* *****************************************************
+ * Non-final methods
+ * *****************************************************
+ */
+
+ public boolean containsMeta() {
+ return false;
+ }
+
+ @Override
+ public int getColumnIndex(int ordinal) {
+ return assembler.getColumnIndex(ordinal);
+ }
+
+ @Override
+ public int getNumberOfProjectedColumns() {
+ return assembler.getNumberOfColumns();
+ }
+
+ @Override
+ public int getNumberOfColumns() {
+ return assembler.getNumberOfColumns();
+ }
+
+ @Override
+ public AbstractColumnTupleReader createTupleReader() {
+ return new QueryColumnTupleReader(this);
+ }
+
+ /**
+ * Create {@link QueryColumnMetadata} that would be used to determine the requested values
+ *
+ * @param datasetType dataset declared type
+ * @param numberOfPrimaryKeys number of PKs
+ * @param serializedMetadata inferred metadata (schema)
+ * @param readerFactory column reader factory
+ * @param valueGetterFactory value serializer
+ * @param requestedType the requested schema
+ * @return query metadata
+ */
+ public static QueryColumnMetadata create(ARecordType datasetType, int numberOfPrimaryKeys,
+ IValueReference serializedMetadata, IColumnValuesReaderFactory readerFactory,
+ IValueGetterFactory valueGetterFactory, ARecordType requestedType,
+ Map<String, FunctionCallInformation> functionCallInfoMap,
+ IColumnFilterEvaluatorFactory filterEvaluatorFactory, IWarningCollector warningCollector)
+ throws IOException {
+ byte[] bytes = serializedMetadata.getByteArray();
+ int offset = serializedMetadata.getStartOffset();
+ int length = serializedMetadata.getLength();
+
+ int fieldNamesStart = offset + IntegerPointable.getInteger(bytes, offset + FIELD_NAMES_POINTER);
+ int metaRootStart = IntegerPointable.getInteger(bytes, offset + META_SCHEMA_POINTER);
+ int metaRootSize =
+ metaRootStart < 0 ? 0 : IntegerPointable.getInteger(bytes, offset + PATH_INFO_POINTER) - metaRootStart;
+ DataInput input = new DataInputStream(new ByteArrayInputStream(bytes, fieldNamesStart, length));
+
+ //FieldNames
+ FieldNamesDictionary fieldNamesDictionary = FieldNamesDictionary.deserialize(input);
+
+ //Schema
+ ObjectSchemaNode root = (ObjectSchemaNode) AbstractSchemaNode.deserialize(input, null);
+ //Skip metaRoot (if exists)
+ input.skipBytes(metaRootSize);
+
+ //Clip schema
+ SchemaClipperVisitor clipperVisitor =
+ new SchemaClipperVisitor(fieldNamesDictionary, functionCallInfoMap, warningCollector);
+ ObjectSchemaNode clippedRoot = clip(requestedType, root, clipperVisitor);
+
+ FilterAccessorProvider filterAccessorProvider = new FilterAccessorProvider(root, clipperVisitor);
+ IColumnFilterEvaluator filterEvaluator = filterEvaluatorFactory.create(filterAccessorProvider);
+ List<IColumnFilterValueAccessor> filterValueAccessors = filterAccessorProvider.getFilterAccessors();
+
+ IColumnValuesReader[] primaryKeyReaders = createPrimaryKeyReaders(input, readerFactory, numberOfPrimaryKeys);
+
+ if (LOGGER.isInfoEnabled() && filterEvaluator != TrueColumnFilterEvaluator.INSTANCE) {
+ String filterString = filterEvaluator == FalseColumnFilterEvaluator.INSTANCE ? "SKIP_ALL"
+ : filterEvaluatorFactory.toString();
+ LOGGER.info("Filter: {}", filterString);
+ }
+
+ return new QueryColumnMetadata(datasetType, null, primaryKeyReaders, serializedMetadata, fieldNamesDictionary,
+ clippedRoot, readerFactory, valueGetterFactory, filterEvaluator, filterValueAccessors);
+ }
+
+ protected static ObjectSchemaNode clip(ARecordType requestedType, ObjectSchemaNode root,
+ SchemaClipperVisitor clipperVisitor) {
+ ObjectSchemaNode clippedRoot;
+ if (requestedType.getTypeName().equals(DataProjectionFiltrationInfo.ALL_FIELDS_TYPE.getTypeName())) {
+ clippedRoot = root;
+ } else {
+ clippedRoot = (ObjectSchemaNode) requestedType.accept(clipperVisitor, root);
+ }
+ return clippedRoot;
+ }
+
+ protected static IColumnValuesReader[] createPrimaryKeyReaders(DataInput input,
+ IColumnValuesReaderFactory readerFactory, int numberOfPrimaryKeys) throws IOException {
+ //skip number of columns
+ input.readInt();
+
+ IColumnValuesReader[] primaryKeyReaders = new IColumnValuesReader[numberOfPrimaryKeys];
+ for (int i = 0; i < numberOfPrimaryKeys; i++) {
+ primaryKeyReaders[i] = readerFactory.createValueReader(input);
+ }
+ return primaryKeyReaders;
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/query/QueryColumnTupleProjector.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/query/QueryColumnTupleProjector.java
new file mode 100644
index 0000000..8865b13
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/query/QueryColumnTupleProjector.java
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.column.operation.query;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.asterix.column.assembler.value.ValueGetterFactory;
+import org.apache.asterix.column.tuple.AssembledTupleReference;
+import org.apache.asterix.column.tuple.QueryColumnTupleReference;
+import org.apache.asterix.column.values.reader.ColumnValueReaderFactory;
+import org.apache.asterix.column.values.reader.filter.IColumnFilterEvaluatorFactory;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.runtime.projection.FunctionCallInformation;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.projection.IColumnProjectionInfo;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.projection.IColumnTupleProjector;
+
+class QueryColumnTupleProjector implements IColumnTupleProjector {
+ protected final ARecordType datasetType;
+ protected final ARecordType requestedType;
+ protected final int numberOfPrimaryKeys;
+ protected final Map<String, FunctionCallInformation> functionCallInfoMap;
+ protected final IWarningCollector warningCollector;
+ protected final IColumnFilterEvaluatorFactory filterEvaluator;
+ private final AssembledTupleReference assembledTupleReference;
+
+ QueryColumnTupleProjector(ARecordType datasetType, int numberOfPrimaryKeys, ARecordType requestedType,
+ Map<String, FunctionCallInformation> functionCallInfoMap, IColumnFilterEvaluatorFactory filterEvaluator,
+ IWarningCollector warningCollector) {
+ this.datasetType = datasetType;
+ this.numberOfPrimaryKeys = numberOfPrimaryKeys;
+ this.requestedType = requestedType;
+ this.functionCallInfoMap = functionCallInfoMap;
+ this.filterEvaluator = filterEvaluator;
+ this.warningCollector = warningCollector;
+ assembledTupleReference = new AssembledTupleReference(getNumberOfTupleFields());
+ }
+
+ @Override
+ public IColumnProjectionInfo createProjectionInfo(IValueReference serializedMetadata) throws HyracksDataException {
+ try {
+ return QueryColumnMetadata.create(datasetType, numberOfPrimaryKeys, serializedMetadata,
+ new ColumnValueReaderFactory(), ValueGetterFactory.INSTANCE, requestedType, functionCallInfoMap,
+ filterEvaluator, warningCollector);
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ }
+
+ @Override
+ public final ITupleReference project(ITupleReference tuple, DataOutput dos, ArrayTupleBuilder tb)
+ throws IOException {
+ for (int i = 0; i < numberOfPrimaryKeys; i++) {
+ dos.write(tuple.getFieldData(i), tuple.getFieldStart(i), tuple.getFieldLength(i));
+ tb.addFieldEndOffset();
+ }
+ if (isColumnar(tuple)) {
+ IValueReference assembledRecord = getAssembledValue(tuple);
+ dos.write(assembledRecord.getByteArray(), assembledRecord.getStartOffset(), assembledRecord.getLength());
+ } else {
+ dos.write(tuple.getFieldData(numberOfPrimaryKeys), tuple.getFieldStart(numberOfPrimaryKeys),
+ tuple.getFieldLength(numberOfPrimaryKeys));
+ }
+ tb.addFieldEndOffset();
+ //Write meta (if any)
+ writeMeta(tuple, dos, tb);
+
+ return assembledTupleReference.reset(tb);
+ }
+
+ protected boolean isColumnar(ITupleReference tuple) {
+ return tuple instanceof QueryColumnTupleReference;
+ }
+
+ protected IValueReference getAssembledValue(ITupleReference tuple) throws HyracksDataException {
+ QueryColumnTupleReference columnTuple = (QueryColumnTupleReference) tuple;
+ return columnTuple.getAssembledValue();
+ }
+
+ protected void writeMeta(ITupleReference tuple, DataOutput dos, ArrayTupleBuilder tb) throws IOException {
+ //NoOp
+ }
+
+ protected int getNumberOfTupleFields() {
+ return numberOfPrimaryKeys + 1;
+ }
+
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/query/QueryColumnTupleProjectorFactory.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/query/QueryColumnTupleProjectorFactory.java
new file mode 100644
index 0000000..7daa877
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/query/QueryColumnTupleProjectorFactory.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.column.operation.query;
+
+import java.util.Map;
+
+import org.apache.asterix.column.values.reader.filter.IColumnFilterEvaluatorFactory;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.runtime.projection.DataProjectionFiltrationInfo;
+import org.apache.asterix.runtime.projection.FunctionCallInformation;
+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.storage.common.projection.ITupleProjector;
+import org.apache.hyracks.storage.common.projection.ITupleProjectorFactory;
+
+public class QueryColumnTupleProjectorFactory implements ITupleProjectorFactory {
+ private static final long serialVersionUID = 2130283796584264219L;
+ private final ARecordType datasetType;
+ private final ARecordType metaType;
+ private final int numberOfPrimaryKeys;
+ private final ARecordType requestedType;
+ private final ARecordType requestedMetaType;
+ private final Map<String, FunctionCallInformation> functionCallInfo;
+ private final Map<String, FunctionCallInformation> metaFunctionCallInfo;
+ private final IColumnFilterEvaluatorFactory filterEvaluator;
+
+ public QueryColumnTupleProjectorFactory(ARecordType datasetType, ARecordType metaType, int numberOfPrimaryKeys,
+ ARecordType requestedType, Map<String, FunctionCallInformation> functionCallInfo,
+ ARecordType requestedMetaType, Map<String, FunctionCallInformation> metaFunctionCallInfo,
+ IColumnFilterEvaluatorFactory filterEvaluator) {
+ this.datasetType = datasetType;
+ this.metaType = metaType;
+ this.numberOfPrimaryKeys = numberOfPrimaryKeys;
+ this.requestedType = requestedType;
+ this.functionCallInfo = functionCallInfo;
+ this.requestedMetaType = requestedMetaType;
+ this.metaFunctionCallInfo = metaFunctionCallInfo;
+ this.filterEvaluator = filterEvaluator;
+ }
+
+ @Override
+ public ITupleProjector createTupleProjector(IHyracksTaskContext context) throws HyracksDataException {
+ IWarningCollector warningCollector = context.getWarningCollector();
+ if (requestedMetaType == null
+ || DataProjectionFiltrationInfo.EMPTY_TYPE.getTypeName().equals(requestedMetaType.getTypeName())) {
+ /*
+ * Either the dataset does not contain meta record or none of the meta columns were requested. Thus,
+ * ignore reading the meta columns (if exist)
+ */
+ return new QueryColumnTupleProjector(datasetType, numberOfPrimaryKeys, requestedType, functionCallInfo,
+ filterEvaluator, warningCollector);
+ }
+ //The query requested some or all of the meta columns
+ return new QueryColumnWithMetaTupleProjector(datasetType, metaType, numberOfPrimaryKeys, requestedType,
+ functionCallInfo, requestedMetaType, metaFunctionCallInfo, filterEvaluator, warningCollector);
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/query/QueryColumnTupleReader.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/query/QueryColumnTupleReader.java
new file mode 100644
index 0000000..36e47ec
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/query/QueryColumnTupleReader.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.column.operation.query;
+
+import org.apache.asterix.column.metadata.AbstractColumnImmutableReadMetadata;
+import org.apache.asterix.column.tuple.QueryColumnTupleReference;
+import org.apache.asterix.column.tuple.QueryColumnWithMetaTupleReference;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.AbstractColumnTupleReader;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnReadMultiPageOp;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnTupleIterator;
+import org.apache.hyracks.storage.am.lsm.btree.column.impls.btree.ColumnBTreeReadLeafFrame;
+
+public class QueryColumnTupleReader extends AbstractColumnTupleReader {
+ private final QueryColumnMetadata columnMetadata;
+
+ public QueryColumnTupleReader(AbstractColumnImmutableReadMetadata columnMetadata) {
+ this.columnMetadata = (QueryColumnMetadata) columnMetadata;
+ }
+
+ @Override
+ public IColumnTupleIterator createTupleIterator(ColumnBTreeReadLeafFrame frame, int index,
+ IColumnReadMultiPageOp multiPageOp) {
+ if (columnMetadata.containsMeta()) {
+ return new QueryColumnWithMetaTupleReference(index, frame, columnMetadata, multiPageOp);
+ }
+ return new QueryColumnTupleReference(index, frame, columnMetadata, multiPageOp);
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/query/QueryColumnWithMetaMetadata.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/query/QueryColumnWithMetaMetadata.java
new file mode 100644
index 0000000..6f0c974
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/query/QueryColumnWithMetaMetadata.java
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.column.operation.query;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.asterix.column.assembler.value.IValueGetterFactory;
+import org.apache.asterix.column.metadata.FieldNamesDictionary;
+import org.apache.asterix.column.metadata.schema.AbstractSchemaNode;
+import org.apache.asterix.column.metadata.schema.ObjectSchemaNode;
+import org.apache.asterix.column.metadata.schema.visitor.SchemaClipperVisitor;
+import org.apache.asterix.column.values.IColumnValuesReader;
+import org.apache.asterix.column.values.IColumnValuesReaderFactory;
+import org.apache.asterix.column.values.reader.filter.FilterAccessorProvider;
+import org.apache.asterix.column.values.reader.filter.IColumnFilterEvaluator;
+import org.apache.asterix.column.values.reader.filter.IColumnFilterEvaluatorFactory;
+import org.apache.asterix.column.values.reader.filter.IColumnFilterValueAccessor;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.runtime.projection.FunctionCallInformation;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.data.std.primitive.IntegerPointable;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.AbstractColumnTupleReader;
+
+/**
+ * Query column metadata (with metaRecord)
+ */
+public final class QueryColumnWithMetaMetadata extends QueryColumnMetadata {
+ private final ColumnAssembler metaAssembler;
+
+ private QueryColumnWithMetaMetadata(ARecordType datasetType, ARecordType metaType,
+ IColumnValuesReader[] primaryKeyReaders, IValueReference serializedMetadata,
+ FieldNamesDictionary fieldNamesDictionary, ObjectSchemaNode root, ObjectSchemaNode metaRoot,
+ IColumnValuesReaderFactory readerFactory, IValueGetterFactory valueGetterFactory,
+ IColumnFilterEvaluator filterEvaluator, List<IColumnFilterValueAccessor> filterValueAccessors)
+ throws HyracksDataException {
+ super(datasetType, metaType, primaryKeyReaders, serializedMetadata, fieldNamesDictionary, root, readerFactory,
+ valueGetterFactory, filterEvaluator, filterValueAccessors);
+ metaAssembler = new ColumnAssembler(metaRoot, metaType, this, readerFactory, valueGetterFactory);
+ }
+
+ @Override
+ public boolean containsMeta() {
+ return true;
+ }
+
+ @Override
+ public int getColumnIndex(int ordinal) {
+ int metaColumnCount = metaAssembler.getNumberOfColumns();
+ if (ordinal >= metaColumnCount) {
+ return assembler.getColumnIndex(ordinal - metaColumnCount);
+ }
+ return metaAssembler.getColumnIndex(ordinal);
+ }
+
+ @Override
+ public int getNumberOfProjectedColumns() {
+ return metaAssembler.getNumberOfColumns() + assembler.getNumberOfColumns();
+ }
+
+ @Override
+ public int getNumberOfColumns() {
+ return getNumberOfProjectedColumns();
+ }
+
+ @Override
+ public AbstractColumnTupleReader createTupleReader() {
+ return new QueryColumnTupleReader(this);
+ }
+
+ public ColumnAssembler getMetaAssembler() {
+ return metaAssembler;
+ }
+
+ /**
+ * Create {@link QueryColumnWithMetaMetadata} that would be used to determine the requested values
+ *
+ * @param datasetType dataset declared type
+ * @param metaType meta declared type
+ * @param numberOfPrimaryKeys number of PKs
+ * @param serializedMetadata inferred metadata (schema)
+ * @param readerFactory column reader factory
+ * @param valueGetterFactory value serializer
+ * @param requestedType the requested schema
+ * @return query metadata
+ */
+ public static QueryColumnWithMetaMetadata create(ARecordType datasetType, ARecordType metaType,
+ int numberOfPrimaryKeys, IValueReference serializedMetadata, IColumnValuesReaderFactory readerFactory,
+ IValueGetterFactory valueGetterFactory, ARecordType requestedType,
+ Map<String, FunctionCallInformation> functionCallInfo, ARecordType metaRequestedType,
+ Map<String, FunctionCallInformation> metaFunctionCallInfo,
+ IColumnFilterEvaluatorFactory filterEvaluatorFactory, IWarningCollector warningCollector)
+ throws IOException {
+ byte[] bytes = serializedMetadata.getByteArray();
+ int offset = serializedMetadata.getStartOffset();
+ int length = serializedMetadata.getLength();
+
+ int fieldNamesStart = offset + IntegerPointable.getInteger(bytes, offset + FIELD_NAMES_POINTER);
+ DataInput input = new DataInputStream(new ByteArrayInputStream(bytes, fieldNamesStart, length));
+
+ //FieldNames
+ FieldNamesDictionary fieldNamesDictionary = FieldNamesDictionary.deserialize(input);
+
+ //Schema
+ ObjectSchemaNode root = (ObjectSchemaNode) AbstractSchemaNode.deserialize(input, null);
+ ObjectSchemaNode metaRoot = (ObjectSchemaNode) AbstractSchemaNode.deserialize(input, null);
+
+ //Clip dataset schema
+ SchemaClipperVisitor clipperVisitor =
+ new SchemaClipperVisitor(fieldNamesDictionary, functionCallInfo, warningCollector);
+ ObjectSchemaNode clippedRoot = clip(requestedType, root, clipperVisitor);
+
+ //Clip meta schema
+ SchemaClipperVisitor metaClipperVisitor =
+ new SchemaClipperVisitor(fieldNamesDictionary, metaFunctionCallInfo, warningCollector);
+ ObjectSchemaNode metaClippedRoot = clip(metaRequestedType, metaRoot, metaClipperVisitor);
+
+ FilterAccessorProvider filterAccessorProvider = new FilterAccessorProvider(root, metaRoot, clipperVisitor);
+ IColumnFilterEvaluator filterEvaluator = filterEvaluatorFactory.create(filterAccessorProvider);
+ List<IColumnFilterValueAccessor> filterValueAccessors = filterAccessorProvider.getFilterAccessors();
+
+ IColumnValuesReader[] primaryKeyReaders = createPrimaryKeyReaders(input, readerFactory, numberOfPrimaryKeys);
+
+ return new QueryColumnWithMetaMetadata(datasetType, metaType, primaryKeyReaders, serializedMetadata,
+ fieldNamesDictionary, clippedRoot, metaClippedRoot, readerFactory, valueGetterFactory, filterEvaluator,
+ filterValueAccessors);
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/query/QueryColumnWithMetaTupleProjector.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/query/QueryColumnWithMetaTupleProjector.java
new file mode 100644
index 0000000..e8f9045
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/query/QueryColumnWithMetaTupleProjector.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.column.operation.query;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.asterix.column.assembler.value.ValueGetterFactory;
+import org.apache.asterix.column.tuple.QueryColumnWithMetaTupleReference;
+import org.apache.asterix.column.values.reader.ColumnValueReaderFactory;
+import org.apache.asterix.column.values.reader.filter.IColumnFilterEvaluatorFactory;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.runtime.projection.FunctionCallInformation;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.projection.IColumnProjectionInfo;
+
+class QueryColumnWithMetaTupleProjector extends QueryColumnTupleProjector {
+ private final ARecordType metaType;
+ private final ARecordType requestedMetaType;
+ private final Map<String, FunctionCallInformation> metaFunctionCallInfoMap;
+
+ QueryColumnWithMetaTupleProjector(ARecordType datasetType, ARecordType metaType, int numberOfPrimaryKeys,
+ ARecordType requestedType, Map<String, FunctionCallInformation> functionCallInfoMap,
+ ARecordType requestedMetaType, Map<String, FunctionCallInformation> metaFunctionCallInfoMap,
+ IColumnFilterEvaluatorFactory filterEvaluator, IWarningCollector warningCollector) {
+ super(datasetType, numberOfPrimaryKeys, requestedType, functionCallInfoMap, filterEvaluator, warningCollector);
+ this.metaType = metaType;
+ this.requestedMetaType = requestedMetaType;
+ this.metaFunctionCallInfoMap = metaFunctionCallInfoMap;
+ }
+
+ @Override
+ public IColumnProjectionInfo createProjectionInfo(IValueReference serializedMetadata) throws HyracksDataException {
+ try {
+ return QueryColumnWithMetaMetadata.create(datasetType, metaType, numberOfPrimaryKeys, serializedMetadata,
+ new ColumnValueReaderFactory(), ValueGetterFactory.INSTANCE, requestedType, functionCallInfoMap,
+ requestedMetaType, metaFunctionCallInfoMap, filterEvaluator, warningCollector);
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ }
+
+ @Override
+ protected boolean isColumnar(ITupleReference tuple) {
+ return tuple instanceof QueryColumnWithMetaTupleReference;
+ }
+
+ @Override
+ protected IValueReference getAssembledValue(ITupleReference tuple) throws HyracksDataException {
+ QueryColumnWithMetaTupleReference columnTuple = (QueryColumnWithMetaTupleReference) tuple;
+ return columnTuple.getAssembledValue();
+ }
+
+ @Override
+ protected void writeMeta(ITupleReference tuple, DataOutput dos, ArrayTupleBuilder tb) throws IOException {
+ if (tuple instanceof QueryColumnWithMetaTupleReference) {
+ QueryColumnWithMetaTupleReference columnTuple = (QueryColumnWithMetaTupleReference) tuple;
+ IValueReference assembledRecord = columnTuple.getMetaAssembledValue();
+ dos.write(assembledRecord.getByteArray(), assembledRecord.getStartOffset(), assembledRecord.getLength());
+ } else {
+ dos.write(tuple.getFieldData(numberOfPrimaryKeys + 1), tuple.getFieldStart(numberOfPrimaryKeys + 1),
+ tuple.getFieldLength(numberOfPrimaryKeys + 1));
+ }
+ tb.addFieldEndOffset();
+ }
+
+ @Override
+ protected int getNumberOfTupleFields() {
+ return super.getNumberOfTupleFields() + 1;
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/tuple/AssembledTupleReference.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/tuple/AssembledTupleReference.java
new file mode 100644
index 0000000..6659e78
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/tuple/AssembledTupleReference.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.column.tuple;
+
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+public class AssembledTupleReference implements ITupleReference {
+ private final int fieldCount;
+ private final int[] offsets;
+ private final int[] lengths;
+ private byte[] data;
+
+ public AssembledTupleReference(int fieldCount) {
+ this.fieldCount = fieldCount;
+ offsets = new int[fieldCount];
+ lengths = new int[fieldCount];
+ }
+
+ public ITupleReference reset(ArrayTupleBuilder tb) {
+ data = tb.getByteArray();
+ int[] fieldEndOffsets = tb.getFieldEndOffsets();
+
+ int j = fieldEndOffsets.length - 1;
+ for (int i = fieldCount - 1; i >= 0; i--) {
+ offsets[i] = j == 0 ? 0 : fieldEndOffsets[j - 1];
+ lengths[i] = fieldEndOffsets[j] - offsets[i];
+ j--;
+ }
+ return this;
+ }
+
+ @Override
+ public int getFieldCount() {
+ return fieldCount;
+ }
+
+ @Override
+ public byte[] getFieldData(int fIdx) {
+ return data;
+ }
+
+ @Override
+ public int getFieldStart(int fIdx) {
+ return offsets[fIdx];
+ }
+
+ @Override
+ public int getFieldLength(int fIdx) {
+ return lengths[fIdx];
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/tuple/QueryColumnTupleReference.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/tuple/QueryColumnTupleReference.java
new file mode 100644
index 0000000..e286235
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/tuple/QueryColumnTupleReference.java
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.column.tuple;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import org.apache.asterix.column.bytes.stream.in.AbstractBytesInputStream;
+import org.apache.asterix.column.operation.query.ColumnAssembler;
+import org.apache.asterix.column.operation.query.QueryColumnMetadata;
+import org.apache.asterix.column.values.IColumnValuesReader;
+import org.apache.asterix.column.values.reader.filter.FilterAccessorProvider;
+import org.apache.asterix.column.values.reader.filter.IColumnFilterEvaluator;
+import org.apache.asterix.column.values.reader.filter.IColumnFilterValueAccessor;
+import org.apache.asterix.column.values.writer.filters.AbstractColumnFilterWriter;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnBufferProvider;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnReadMultiPageOp;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.projection.IColumnProjectionInfo;
+import org.apache.hyracks.storage.am.lsm.btree.column.impls.btree.ColumnBTreeReadLeafFrame;
+
+public final class QueryColumnTupleReference extends AbstractAsterixColumnTupleReference {
+ private final ColumnAssembler assembler;
+ private final IColumnFilterEvaluator filterEvaluator;
+ private final List<IColumnFilterValueAccessor> filterValueAccessors;
+
+ public QueryColumnTupleReference(int componentIndex, ColumnBTreeReadLeafFrame frame,
+ QueryColumnMetadata columnMetadata, IColumnReadMultiPageOp multiPageOp) {
+ super(componentIndex, frame, columnMetadata, multiPageOp);
+ assembler = columnMetadata.getAssembler();
+ filterEvaluator = columnMetadata.getFilterEvaluator();
+ filterValueAccessors = columnMetadata.getFilterValueAccessors();
+ }
+
+ @Override
+ protected IColumnValuesReader[] getPrimaryKeyReaders(IColumnProjectionInfo info) {
+ return ((QueryColumnMetadata) info).getPrimaryKeyReaders();
+ }
+
+ @Override
+ protected boolean startNewPage(ByteBuffer pageZero, int numberOfColumns, int numberOfTuples) {
+ //Skip to filters
+ pageZero.position(pageZero.position() + numberOfColumns * Integer.BYTES);
+ //Set filters' values
+ FilterAccessorProvider.setFilterValues(filterValueAccessors, pageZero, numberOfColumns);
+ //Skip filters
+ pageZero.position(pageZero.position() + numberOfColumns * AbstractColumnFilterWriter.FILTER_SIZE);
+ //Check if we should read all column pages
+ boolean readColumns = filterEvaluator.evaluate();
+ assembler.reset(readColumns ? numberOfTuples : 0);
+ return readColumns;
+ }
+
+ @Override
+ protected void startColumn(IColumnBufferProvider buffersProvider, int startIndex, int ordinal, int numberOfTuples)
+ throws HyracksDataException {
+ AbstractBytesInputStream columnStream = columnStreams[ordinal];
+ columnStream.reset(buffersProvider);
+ assembler.resetColumn(columnStream, startIndex, ordinal);
+ }
+
+ @Override
+ public void skip(int count) throws HyracksDataException {
+ assembler.skip(count);
+ }
+
+ public IValueReference getAssembledValue() throws HyracksDataException {
+ return assembler.nextValue();
+ }
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/tuple/QueryColumnWithMetaTupleReference.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/tuple/QueryColumnWithMetaTupleReference.java
new file mode 100644
index 0000000..a5cedc1
--- /dev/null
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/tuple/QueryColumnWithMetaTupleReference.java
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.column.tuple;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import org.apache.asterix.column.bytes.stream.in.AbstractBytesInputStream;
+import org.apache.asterix.column.operation.query.ColumnAssembler;
+import org.apache.asterix.column.operation.query.QueryColumnMetadata;
+import org.apache.asterix.column.operation.query.QueryColumnWithMetaMetadata;
+import org.apache.asterix.column.values.IColumnValuesReader;
+import org.apache.asterix.column.values.reader.filter.FilterAccessorProvider;
+import org.apache.asterix.column.values.reader.filter.IColumnFilterEvaluator;
+import org.apache.asterix.column.values.reader.filter.IColumnFilterValueAccessor;
+import org.apache.asterix.column.values.writer.filters.AbstractColumnFilterWriter;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnBufferProvider;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnReadMultiPageOp;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.projection.IColumnProjectionInfo;
+import org.apache.hyracks.storage.am.lsm.btree.column.impls.btree.ColumnBTreeReadLeafFrame;
+
+public final class QueryColumnWithMetaTupleReference extends AbstractAsterixColumnTupleReference {
+ private final ColumnAssembler assembler;
+ private final ColumnAssembler metaAssembler;
+ private final IColumnFilterEvaluator filterEvaluator;
+ private final List<IColumnFilterValueAccessor> filterValueAccessors;
+
+ public QueryColumnWithMetaTupleReference(int componentIndex, ColumnBTreeReadLeafFrame frame,
+ QueryColumnMetadata columnMetadata, IColumnReadMultiPageOp multiPageOp) {
+ super(componentIndex, frame, columnMetadata, multiPageOp);
+ assembler = columnMetadata.getAssembler();
+ metaAssembler = ((QueryColumnWithMetaMetadata) columnMetadata).getMetaAssembler();
+ filterEvaluator = columnMetadata.getFilterEvaluator();
+ filterValueAccessors = columnMetadata.getFilterValueAccessors();
+ }
+
+ @Override
+ protected IColumnValuesReader[] getPrimaryKeyReaders(IColumnProjectionInfo info) {
+ return ((QueryColumnMetadata) info).getPrimaryKeyReaders();
+ }
+
+ @Override
+ protected boolean startNewPage(ByteBuffer pageZero, int numberOfColumns, int numberOfTuples) {
+ //Skip to filters
+ pageZero.position(pageZero.position() + numberOfColumns * Integer.BYTES);
+ //Set filters' values
+ FilterAccessorProvider.setFilterValues(filterValueAccessors, pageZero, numberOfColumns);
+ //Skip filters
+ pageZero.position(pageZero.position() + numberOfColumns * AbstractColumnFilterWriter.FILTER_SIZE);
+ //Check if we should read all column pages
+ boolean readColumns = filterEvaluator.evaluate();
+ assembler.reset(readColumns ? numberOfTuples : 0);
+ metaAssembler.reset(readColumns ? numberOfTuples : 0);
+ return readColumns;
+ }
+
+ @Override
+ protected void startColumn(IColumnBufferProvider buffersProvider, int startIndex, int ordinal, int numberOfTuples)
+ throws HyracksDataException {
+ AbstractBytesInputStream columnStream = columnStreams[ordinal];
+ columnStream.reset(buffersProvider);
+ int metaColumnCount = metaAssembler.getNumberOfColumns();
+ if (ordinal >= metaColumnCount) {
+ assembler.resetColumn(columnStream, startIndex, ordinal - metaColumnCount);
+ } else {
+ metaAssembler.resetColumn(columnStream, startIndex, ordinal);
+ }
+ }
+
+ @Override
+ public void skip(int count) throws HyracksDataException {
+ metaAssembler.skip(count);
+ assembler.skip(count);
+ }
+
+ public IValueReference getAssembledValue() throws HyracksDataException {
+ return assembler.nextValue();
+ }
+
+ public IValueReference getMetaAssembledValue() throws HyracksDataException {
+ return metaAssembler.nextValue();
+ }
+}
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 9296339..5466a3b 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
@@ -118,7 +118,11 @@
COMPILER_QUERYPLANSHAPE(
STRING,
AlgebricksConfig.QUERY_PLAN_SHAPE_DEFAULT,
- "Set the mode for forcing the shape of the query plan");
+ "Set the mode for forcing the shape of the query plan"),
+ COMPILER_COLUMN_FILTER(
+ BOOLEAN,
+ AlgebricksConfig.COLUMN_FILTER_DEFAULT,
+ "Enable/disable the use of column min/max filters");
private final IOptionType type;
private final Object defaultValue;
@@ -198,6 +202,8 @@
public static final String COMPILER_QUERY_PLAN_SHAPE_KEY = Option.COMPILER_QUERYPLANSHAPE.ini();
+ public static final String COMPILER_COLUMN_FILTER_KEY = Option.COMPILER_COLUMN_FILTER.ini();
+
public static final int COMPILER_PARALLELISM_AS_STORAGE = 0;
public CompilerProperties(PropertiesAccessor accessor) {
@@ -296,4 +302,8 @@
return AlgebricksConfig.QUERY_PLAN_SHAPE_DEFAULT;
return queryPlanShapeMode;
}
+
+ public boolean isColumnFilter() {
+ return accessor.getBoolean(Option.COMPILER_COLUMN_FILTER);
+ }
}
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 478bd46..3f15eef 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
@@ -18,6 +18,9 @@
*/
package org.apache.asterix.common.config;
+import java.util.HashMap;
+import java.util.Map;
+
public class DatasetConfig {
/*
@@ -42,7 +45,7 @@
LENGTH_PARTITIONED_WORD_INVIX,
LENGTH_PARTITIONED_NGRAM_INVIX,
ARRAY,
- SAMPLE;
+ SAMPLE
}
public enum TransactionState {
@@ -79,4 +82,34 @@
*/
APPEND_OP
}
+
+ public enum DatasetFormat {
+ /**
+ * Row format using ADM
+ */
+ ROW("row"),
+ /**
+ * Column format using AMAX
+ */
+ COLUMN("column");
+
+ private final String format;
+ private static final Map<String, DatasetFormat> FORMATS = createFormats();
+
+ DatasetFormat(String format) {
+ this.format = format;
+ }
+
+ private static Map<String, DatasetFormat> createFormats() {
+ Map<String, DatasetFormat> formats = new HashMap<>();
+ for (DatasetFormat format : DatasetFormat.values()) {
+ formats.put(format.format, format);
+ }
+ return formats;
+ }
+
+ public static DatasetFormat getFormat(String format) {
+ return FORMATS.get(format.trim().toLowerCase());
+ }
+ }
}
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 e1edc1c..0e8ac55 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
@@ -87,6 +87,8 @@
compilerProperties.getForceJoinOrderMode());
String queryPlanShape = getString(querySpecificConfig, CompilerProperties.COMPILER_QUERY_PLAN_SHAPE_KEY,
compilerProperties.getQueryPlanShapeMode());
+ boolean columnFilter = getBoolean(querySpecificConfig, CompilerProperties.COMPILER_COLUMN_FILTER_KEY,
+ compilerProperties.isColumnFilter());
PhysicalOptimizationConfig physOptConf = new PhysicalOptimizationConfig();
physOptConf.setFrameSize(frameSize);
@@ -110,6 +112,7 @@
physOptConf.setCBOTestMode(cboTest);
physOptConf.setForceJoinOrderMode(forceJoinOrder);
physOptConf.setQueryPlanShapeMode(queryPlanShape);
+ physOptConf.setColumnFilter(columnFilter);
return physOptConf;
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/StorageProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/StorageProperties.java
index 12c9c68..5b99fa0 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/StorageProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/StorageProperties.java
@@ -56,12 +56,14 @@
STORAGE_COMPRESSION_BLOCK(STRING, "snappy"),
STORAGE_DISK_FORCE_BYTES(LONG_BYTE_UNIT, StorageUtil.getLongSizeInBytes(16, MEGABYTE)),
STORAGE_IO_SCHEDULER(STRING, "greedy"),
- STORAGE_WRITE_RATE_LIMIT(LONG_BYTE_UNIT, 0l),
+ STORAGE_WRITE_RATE_LIMIT(LONG_BYTE_UNIT, 0L),
STORAGE_MAX_CONCURRENT_FLUSHES_PER_PARTITION(NONNEGATIVE_INTEGER, 2),
STORAGE_MAX_SCHEDULED_MERGES_PER_PARTITION(NONNEGATIVE_INTEGER, 8),
STORAGE_MAX_CONCURRENT_MERGES_PER_PARTITION(NONNEGATIVE_INTEGER, 2),
STORAGE_GLOBAL_CLEANUP(BOOLEAN, true),
- STORAGE_GLOBAL_CLEANUP_TIMEOUT(POSITIVE_INTEGER, (int) TimeUnit.MINUTES.toSeconds(10));
+ STORAGE_GLOBAL_CLEANUP_TIMEOUT(POSITIVE_INTEGER, (int) TimeUnit.MINUTES.toSeconds(10)),
+ STORAGE_COLUMN_MAX_TUPLE_COUNT(NONNEGATIVE_INTEGER, 15000),
+ STORAGE_COLUMN_FREE_SPACE_TOLERANCE(DOUBLE, 0.15);
private final IOptionType interpreter;
private final Object defaultValue;
@@ -129,6 +131,11 @@
return "Indicates whether or not global storage cleanup is performed";
case STORAGE_GLOBAL_CLEANUP_TIMEOUT:
return "The maximum time to wait for nodes to respond to global storage cleanup requests";
+ case STORAGE_COLUMN_MAX_TUPLE_COUNT:
+ return "The maximum number of tuples to be stored per a mega leaf page";
+ case STORAGE_COLUMN_FREE_SPACE_TOLERANCE:
+ return "The percentage of the maximum tolerable empty space for a physical mega leaf page (e.g.,"
+ + " 0.15 means a physical page with 15% or less empty space is tolerable)";
default:
throw new IllegalStateException("NYI: " + this);
}
@@ -265,4 +272,12 @@
public int getDiskForcePages() {
return (int) (accessor.getLong(Option.STORAGE_DISK_FORCE_BYTES) / getBufferCachePageSize());
}
+
+ public int getColumnMaxTupleCount() {
+ return accessor.getInt(Option.STORAGE_COLUMN_MAX_TUPLE_COUNT);
+ }
+
+ public float getColumnFreeSpaceTolerance() {
+ return (float) accessor.getDouble(Option.STORAGE_COLUMN_FREE_SPACE_TOLERANCE);
+ }
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/AsterixTypeToParquetTypeVisitor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/AsterixTypeToParquetTypeVisitor.java
index 68d5b86..af8733f 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/AsterixTypeToParquetTypeVisitor.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/AsterixTypeToParquetTypeVisitor.java
@@ -34,7 +34,7 @@
import org.apache.asterix.om.types.AbstractCollectionType;
import org.apache.asterix.om.types.IAType;
import org.apache.asterix.om.types.IATypeVisitor;
-import org.apache.asterix.runtime.projection.DataProjectionInfo;
+import org.apache.asterix.runtime.projection.DataProjectionFiltrationInfo;
import org.apache.asterix.runtime.projection.FunctionCallInformation;
import org.apache.hyracks.api.exceptions.SourceLocation;
import org.apache.hyracks.api.exceptions.Warning;
@@ -67,9 +67,9 @@
public MessageType clipType(ARecordType rootType, MessageType fileSchema,
Map<String, FunctionCallInformation> funcInfo) {
- if (rootType == DataProjectionInfo.EMPTY_TYPE) {
+ if (rootType == DataProjectionFiltrationInfo.EMPTY_TYPE) {
return EMPTY_PARQUET_MESSAGE;
- } else if (rootType == DataProjectionInfo.ALL_FIELDS_TYPE) {
+ } else if (rootType == DataProjectionFiltrationInfo.ALL_FIELDS_TYPE) {
return fileSchema;
}
Types.MessageTypeBuilder builder = Types.buildMessage();
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 d730898..1605360 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
@@ -72,7 +72,7 @@
import org.apache.asterix.om.types.IAType;
import org.apache.asterix.om.types.TypeTagUtil;
import org.apache.asterix.runtime.evaluators.common.NumberUtils;
-import org.apache.asterix.runtime.projection.DataProjectionInfo;
+import org.apache.asterix.runtime.projection.DataProjectionFiltrationInfo;
import org.apache.asterix.runtime.projection.FunctionCallInformation;
import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -242,7 +242,7 @@
public static boolean isTrue(Map<String, String> configuration, String key) {
String value = configuration.get(key);
- return value == null ? false : Boolean.valueOf(value);
+ return value != null && Boolean.valueOf(value);
}
// Currently not used.
@@ -389,8 +389,7 @@
/**
* Fills the configuration of the external dataset and its adapter with default values if not provided by user.
*
- * @param configuration
- * external data configuration
+ * @param configuration external data configuration
*/
public static void defaultConfiguration(Map<String, String> configuration) {
String format = configuration.get(ExternalDataConstants.KEY_FORMAT);
@@ -412,10 +411,8 @@
* Prepares the configuration of the external data and its adapter by filling the information required by
* adapters and parsers.
*
- * @param adapterName
- * adapter name
- * @param configuration
- * external data configuration
+ * @param adapterName adapter name
+ * @param configuration external data configuration
*/
public static void prepare(String adapterName, Map<String, String> configuration) {
if (!configuration.containsKey(ExternalDataConstants.KEY_READER)) {
@@ -437,8 +434,7 @@
* Normalizes the values of certain parameters of the adapter configuration. This should happen before persisting
* the metadata (e.g. when creating external datasets or feeds) and when creating an adapter factory.
*
- * @param configuration
- * external data configuration
+ * @param configuration external data configuration
*/
public static void normalize(Map<String, String> configuration) {
// normalize the "format" parameter
@@ -458,10 +454,8 @@
/**
* Validates the parameter values of the adapter configuration. This should happen after normalizing the values.
*
- * @param configuration
- * external data configuration
- * @throws HyracksDataException
- * HyracksDataException
+ * @param configuration external data configuration
+ * @throws HyracksDataException HyracksDataException
*/
public static void validate(Map<String, String> configuration) throws HyracksDataException {
String format = configuration.get(ExternalDataConstants.KEY_FORMAT);
@@ -523,8 +517,7 @@
* Validates adapter specific external dataset properties. Specific properties for different adapters should be
* validated here
*
- * @param configuration
- * properties
+ * @param configuration properties
*/
public static void validateAdapterSpecificProperties(Map<String, String> configuration, SourceLocation srcLoc,
IWarningCollector collector, IApplicationContext appCtx) throws CompilationException {
@@ -552,8 +545,7 @@
/**
* Regex matches all the provided patterns against the provided path
*
- * @param path
- * path to check against
+ * @param path path to check against
* @return {@code true} if all patterns match, {@code false} otherwise
*/
public static boolean matchPatterns(List<Matcher> matchers, String path) {
@@ -568,8 +560,7 @@
/**
* Converts the wildcard to proper regex
*
- * @param pattern
- * wildcard pattern to convert
+ * @param pattern wildcard pattern to convert
* @return regex expression
*/
public static String patternToRegex(String pattern) {
@@ -658,8 +649,7 @@
/**
* Adjusts the prefix (if needed) and returns it
*
- * @param configuration
- * configuration
+ * @param configuration configuration
*/
public static String getPrefix(Map<String, String> configuration) {
return getPrefix(configuration, true);
@@ -692,10 +682,8 @@
}
/**
- * @param configuration
- * configuration map
- * @throws CompilationException
- * Compilation exception
+ * @param configuration configuration map
+ * @throws CompilationException Compilation exception
*/
public static void validateIncludeExclude(Map<String, String> configuration) throws CompilationException {
// Ensure that include and exclude are not provided at the same time + ensure valid format or property
@@ -779,10 +767,8 @@
/**
* Validate Parquet dataset's declared type and configuration
*
- * @param properties
- * external dataset configuration
- * @param datasetRecordType
- * dataset declared type
+ * @param properties external dataset configuration
+ * @param datasetRecordType dataset declared type
*/
public static void validateParquetTypeAndConfiguration(Map<String, String> properties,
ARecordType datasetRecordType) throws CompilationException {
@@ -804,8 +790,8 @@
|| ExternalDataConstants.FORMAT_PARQUET.equals(properties.get(ExternalDataConstants.KEY_FORMAT));
}
- public static void setExternalDataProjectionInfo(DataProjectionInfo projectionInfo, Map<String, String> properties)
- throws IOException {
+ public static void setExternalDataProjectionInfo(DataProjectionFiltrationInfo projectionInfo,
+ Map<String, String> properties) throws IOException {
properties.put(ExternalDataConstants.KEY_REQUESTED_FIELDS,
serializeExpectedTypeToString(projectionInfo.getProjectionInfo()));
properties.put(ExternalDataConstants.KEY_HADOOP_ASTERIX_FUNCTION_CALL_INFORMATION,
@@ -815,19 +801,19 @@
/**
* Serialize {@link ARecordType} as Base64 string to pass it to {@link org.apache.hadoop.conf.Configuration}
*
- * @param expectedType
- * expected type
+ * @param expectedType expected type
* @return the expected type as Base64 string
*/
private static String serializeExpectedTypeToString(ARecordType expectedType) throws IOException {
- if (expectedType == DataProjectionInfo.EMPTY_TYPE || expectedType == DataProjectionInfo.ALL_FIELDS_TYPE) {
+ if (expectedType == DataProjectionFiltrationInfo.EMPTY_TYPE
+ || expectedType == DataProjectionFiltrationInfo.ALL_FIELDS_TYPE) {
//Return the type name of EMPTY_TYPE and ALL_FIELDS_TYPE
return expectedType.getTypeName();
}
ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutputStream);
Base64.Encoder encoder = Base64.getEncoder();
- DataProjectionInfo.writeTypeField(expectedType, dataOutputStream);
+ DataProjectionFiltrationInfo.writeTypeField(expectedType, dataOutputStream);
return encoder.encodeToString(byteArrayOutputStream.toByteArray());
}
@@ -835,8 +821,7 @@
* Serialize {@link FunctionCallInformation} map as Base64 string to pass it to
* {@link org.apache.hadoop.conf.Configuration}
*
- * @param functionCallInfoMap
- * function information map
+ * @param functionCallInfoMap function information map
* @return function information map as Base64 string
*/
static String serializeFunctionCallInfoToString(Map<String, FunctionCallInformation> functionCallInfoMap)
@@ -844,7 +829,7 @@
ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutputStream);
Base64.Encoder encoder = Base64.getEncoder();
- DataProjectionInfo.writeFunctionCallInformationMapField(functionCallInfoMap, dataOutputStream);
+ DataProjectionFiltrationInfo.writeFunctionCallInformationMapField(functionCallInfoMap, dataOutputStream);
return encoder.encodeToString(byteArrayOutputStream.toByteArray());
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/HDFSUtils.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/HDFSUtils.java
index 3506216..a4e47ca 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/HDFSUtils.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/HDFSUtils.java
@@ -42,7 +42,7 @@
import org.apache.asterix.external.input.stream.HDFSInputStream;
import org.apache.asterix.external.util.ExternalDataConstants.ParquetOptions;
import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.runtime.projection.DataProjectionInfo;
+import org.apache.asterix.runtime.projection.DataProjectionFiltrationInfo;
import org.apache.asterix.runtime.projection.FunctionCallInformation;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.BlockLocation;
@@ -242,7 +242,7 @@
String requestedValues = configuration.get(ExternalDataConstants.KEY_REQUESTED_FIELDS);
if (requestedValues == null) {
//No value is requested, return the entire record
- requestedValues = DataProjectionInfo.ALL_FIELDS_TYPE.getTypeName();
+ requestedValues = DataProjectionFiltrationInfo.ALL_FIELDS_TYPE.getTypeName();
} else {
//Subset of the values were requested, set the functionCallInformation
conf.set(ExternalDataConstants.KEY_HADOOP_ASTERIX_FUNCTION_CALL_INFORMATION,
@@ -284,18 +284,18 @@
public static ARecordType getExpectedType(Configuration configuration) throws IOException {
String encoded = configuration.get(ExternalDataConstants.KEY_REQUESTED_FIELDS, "");
- if (encoded.isEmpty() || encoded.equals(DataProjectionInfo.ALL_FIELDS_TYPE.getTypeName())) {
+ if (encoded.isEmpty() || encoded.equals(DataProjectionFiltrationInfo.ALL_FIELDS_TYPE.getTypeName())) {
//By default, return the entire records
- return DataProjectionInfo.ALL_FIELDS_TYPE;
- } else if (encoded.equals(DataProjectionInfo.EMPTY_TYPE.getTypeName())) {
+ return DataProjectionFiltrationInfo.ALL_FIELDS_TYPE;
+ } else if (encoded.equals(DataProjectionFiltrationInfo.EMPTY_TYPE.getTypeName())) {
//No fields were requested
- return DataProjectionInfo.EMPTY_TYPE;
+ return DataProjectionFiltrationInfo.EMPTY_TYPE;
}
//A subset of the fields was requested
Base64.Decoder decoder = Base64.getDecoder();
byte[] typeBytes = decoder.decode(encoded);
DataInputStream dataInputStream = new DataInputStream(new ByteArrayInputStream(typeBytes));
- return DataProjectionInfo.createTypeField(dataInputStream);
+ return DataProjectionFiltrationInfo.createTypeField(dataInputStream);
}
public static void setFunctionCallInformationMap(Map<String, FunctionCallInformation> funcCallInfoMap,
@@ -311,7 +311,7 @@
Base64.Decoder decoder = Base64.getDecoder();
byte[] functionCallInfoMapBytes = decoder.decode(encoded);
DataInputStream dataInputStream = new DataInputStream(new ByteArrayInputStream(functionCallInfoMapBytes));
- return DataProjectionInfo.createFunctionCallInformationMap(dataInputStream);
+ return DataProjectionFiltrationInfo.createFunctionCallInformationMap(dataInputStream);
}
return null;
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DatasetDecl.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DatasetDecl.java
index 942e39b..8f48db0 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DatasetDecl.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DatasetDecl.java
@@ -20,6 +20,7 @@
import java.util.Map;
+import org.apache.asterix.common.config.DatasetConfig;
import org.apache.asterix.common.config.DatasetConfig.DatasetType;
import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.common.metadata.DataverseName;
@@ -31,6 +32,7 @@
import org.apache.asterix.lang.common.util.ConfigurationUtil;
import org.apache.asterix.lang.common.util.DatasetDeclParametersUtil;
import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
+import org.apache.asterix.metadata.dataset.DatasetFormatInfo;
import org.apache.asterix.object.base.AdmObjectNode;
import org.apache.asterix.object.base.IAdmNode;
import org.apache.asterix.runtime.compression.CompressionManager;
@@ -43,7 +45,7 @@
protected final DatasetType datasetType;
protected final IDatasetDetailsDecl datasetDetailsDecl;
protected final Map<String, String> hints;
- private AdmObjectNode withObjectNode;
+ private final AdmObjectNode withObjectNode;
protected final boolean ifNotExists;
public DatasetDecl(DataverseName dataverse, Identifier name, TypeExpression itemType, TypeExpression metaItemType,
@@ -132,6 +134,23 @@
.getOptionalString(DatasetDeclParametersUtil.STORAGE_BLOCK_COMPRESSION_SCHEME_PARAMETER_NAME);
}
+ public DatasetFormatInfo getDatasetFormatInfo(int defaultMaxTupleCount, float defaultFreeSpaceTolerance) {
+ final AdmObjectNode datasetFormatNode =
+ (AdmObjectNode) withObjectNode.get(DatasetDeclParametersUtil.DATASET_FORMAT_PARAMETER_NAME);
+ if (datasetType != DatasetType.INTERNAL || datasetFormatNode == null) {
+ return DatasetFormatInfo.DEFAULT;
+ }
+ DatasetConfig.DatasetFormat datasetFormat = DatasetConfig.DatasetFormat.getFormat(
+ datasetFormatNode.getOptionalString(DatasetDeclParametersUtil.DATASET_FORMAT_FORMAT_PARAMETER_NAME));
+ int maxTupleCount = datasetFormatNode.getOptionalInt(
+ DatasetDeclParametersUtil.DATASET_FORMAT_MAX_TUPLE_COUNT_PARAMETER_NAME, defaultMaxTupleCount);
+ float freeSpaceTolerance = datasetFormatNode.getOptionalFloat(
+ DatasetDeclParametersUtil.DATASET_FORMAT_FREE_SPACE_TOLERANCE_PARAMETER_NAME,
+ defaultFreeSpaceTolerance);
+
+ return new DatasetFormatInfo(datasetFormat, maxTupleCount, freeSpaceTolerance);
+ }
+
public Map<String, String> getHints() {
return hints;
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/DatasetDeclParametersUtil.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/DatasetDeclParametersUtil.java
index 0aef69f..882f61c 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/DatasetDeclParametersUtil.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/DatasetDeclParametersUtil.java
@@ -62,6 +62,16 @@
*/
public static final String NODE_GROUP_NAME = "node-group";
public static final String NODE_GROUP_NAME_PARAMETER_NAME = "name";
+
+ /* ***********************************************
+ * Dataset Format Type
+ * ***********************************************
+ */
+ public static final String DATASET_FORMAT_PARAMETER_NAME = "dataset-format";
+ public static final String DATASET_FORMAT_FORMAT_PARAMETER_NAME = "format";
+ public static final String DATASET_FORMAT_MAX_TUPLE_COUNT_PARAMETER_NAME = "max-tuple-count";
+ public static final String DATASET_FORMAT_FREE_SPACE_TOLERANCE_PARAMETER_NAME = "free-space-tolerance";
+
/* ***********************************************
* Private members
* ***********************************************
@@ -90,11 +100,12 @@
}
private static ARecordType getWithObjectType() {
- final String[] withNames =
- { MERGE_POLICY_PARAMETER_NAME, STORAGE_BLOCK_COMPRESSION_PARAMETER_NAME, NODE_GROUP_NAME };
+ final String[] withNames = { MERGE_POLICY_PARAMETER_NAME, STORAGE_BLOCK_COMPRESSION_PARAMETER_NAME,
+ NODE_GROUP_NAME, DATASET_FORMAT_PARAMETER_NAME };
final IAType[] withTypes = { AUnionType.createUnknownableType(getMergePolicyType()),
AUnionType.createUnknownableType(getStorageBlockCompressionType()),
- AUnionType.createUnknownableType(getNodeGroupType()) };
+ AUnionType.createUnknownableType(getNodeGroupType()),
+ AUnionType.createUnknownableType(getDatasetFormatType()) };
return new ARecordType("withObject", withNames, withTypes, false);
}
@@ -134,8 +145,8 @@
}
/**
- * Adjusts dataset inline type definition if it has primary key specification:
- * forces NOT UNKNOWN on fields that are part of primary key.
+ * Adjusts dataset inline type definition if it has primary key specification:
+ * forces NOT UNKNOWN on fields that are part of primary key.
*/
public static void adjustInlineTypeDecl(TypeExpression typeDecl, List<List<String>> primaryKeyFields,
List<Integer> primaryKeySources, boolean isMeta) {
@@ -166,4 +177,12 @@
throw new IllegalStateException(typeDecl.getTypeKind().toString());
}
}
+
+ private static ARecordType getDatasetFormatType() {
+ final String[] formatFieldNames = { DATASET_FORMAT_FORMAT_PARAMETER_NAME,
+ DATASET_FORMAT_MAX_TUPLE_COUNT_PARAMETER_NAME, DATASET_FORMAT_FREE_SPACE_TOLERANCE_PARAMETER_NAME };
+ final IAType[] formatFieldTypes = { BuiltinType.ASTRING, AUnionType.createUnknownableType(BuiltinType.AINT64),
+ AUnionType.createUnknownableType(BuiltinType.ADOUBLE) };
+ return new ARecordType(DATASET_FORMAT_PARAMETER_NAME, formatFieldNames, formatFieldTypes, false);
+ }
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
index f00090a..f1156f7 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
@@ -196,6 +196,10 @@
"BlockLevelStorageCompression";
public static final String DATASET_ARECORD_DATASET_COMPRESSION_SCHEME_FIELD_NAME = "DatasetCompressionScheme";
public static final String DATASET_ARECORD_REBALANCE_FIELD_NAME = "rebalanceCount";
+ public static final String DATASET_ARECORD_DATASET_FORMAT_FIELD_NAME = "DatasetFormat";
+ public static final String DATASET_ARECORD_DATASET_FORMAT_FORMAT_FIELD_NAME = "Format";
+ public static final String DATASET_ARECORD_DATASET_MAX_TUPLE_COUNT_FIELD_NAME = "MaxTupleCount";
+ public static final String DATASET_ARECORD_DATASET_FREE_SPACE_TOLERANCE_FIELD_NAME = "FreeSpaceTolerance";
public static final ARecordType DATASET_RECORDTYPE = createRecordType(
// RecordTypeName
RECORD_NAME_DATASET,
@@ -574,7 +578,7 @@
return recordType;
}
- public static final ARecordType createPropertiesRecordType() {
+ public static ARecordType createPropertiesRecordType() {
return createRecordType(
// RecordTypeName
null,
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/dataset/DatasetFormatInfo.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/dataset/DatasetFormatInfo.java
new file mode 100644
index 0000000..38951a4
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/dataset/DatasetFormatInfo.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.metadata.dataset;
+
+import java.io.Serializable;
+
+import org.apache.asterix.common.config.DatasetConfig.DatasetFormat;
+
+public class DatasetFormatInfo implements Serializable {
+ private static final long serialVersionUID = 7656132322813253435L;
+ public static final DatasetFormatInfo DEFAULT = new DatasetFormatInfo();
+ private final DatasetFormat format;
+ private final int maxTupleCount;
+ private final float freeSpaceTolerance;
+
+ private DatasetFormatInfo() {
+ this(DatasetFormat.ROW, -1, 0.0f);
+ }
+
+ public DatasetFormatInfo(DatasetFormat format, int maxTupleCount, float freeSpaceTolerance) {
+ this.format = format;
+ this.maxTupleCount = maxTupleCount;
+ this.freeSpaceTolerance = freeSpaceTolerance;
+ }
+
+ public DatasetFormat getFormat() {
+ return format;
+ }
+
+ public int getMaxTupleCount() {
+ return maxTupleCount;
+ }
+
+ public float getFreeSpaceTolerance() {
+ return freeSpaceTolerance;
+ }
+
+ @Override
+ public String toString() {
+ return "(format:" + format + ", maxTupleCount:" + maxTupleCount + ')';
+ }
+}
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 d119067..cafa001 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.column.ColumnManagerFactory;
import org.apache.asterix.common.config.DatasetConfig;
import org.apache.asterix.common.config.DatasetConfig.DatasetType;
import org.apache.asterix.common.context.AsterixVirtualBufferCacheProvider;
@@ -31,8 +32,10 @@
import org.apache.asterix.external.indexing.IndexingConstants;
import org.apache.asterix.formats.nontagged.NullIntrospector;
import org.apache.asterix.metadata.api.IResourceFactoryProvider;
+import org.apache.asterix.metadata.dataset.DatasetFormatInfo;
import org.apache.asterix.metadata.entities.Dataset;
import org.apache.asterix.metadata.entities.Index;
+import org.apache.asterix.metadata.entities.InternalDatasetDetails;
import org.apache.asterix.metadata.utils.IndexUtil;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.BuiltinType;
@@ -45,6 +48,8 @@
import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import org.apache.hyracks.api.dataflow.value.ITypeTraits;
import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnManagerFactory;
+import org.apache.hyracks.storage.am.lsm.btree.column.dataflow.LSMColumnBTreeLocalResourceFactory;
import org.apache.hyracks.storage.am.lsm.btree.dataflow.ExternalBTreeLocalResourceFactory;
import org.apache.hyracks.storage.am.lsm.btree.dataflow.ExternalBTreeWithBuddyLocalResourceFactory;
import org.apache.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeLocalResourceFactory;
@@ -115,13 +120,30 @@
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,
- isSecondaryNoIncrementalMaintenance);
+ DatasetFormatInfo datasetFormatInfo = dataset.getDatasetFormatInfo();
+ if (!index.isPrimaryIndex() || datasetFormatInfo.getFormat() == DatasetConfig.DatasetFormat.ROW) {
+ 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, isSecondaryNoIncrementalMaintenance);
+ } else {
+ //Column
+ List<Integer> keySourceIndicator =
+ ((InternalDatasetDetails) dataset.getDatasetDetails()).getKeySourceIndicator();
+ IColumnManagerFactory columnManagerFactory =
+ new ColumnManagerFactory(recordType, metaType, dataset.getPrimaryKeys(), keySourceIndicator,
+ mdProvider.getStorageProperties().getBufferCachePageSize(),
+ datasetFormatInfo.getMaxTupleCount(), datasetFormatInfo.getFreeSpaceTolerance());
+ return new LSMColumnBTreeLocalResourceFactory(storageManager, typeTraits, cmpFactories,
+ filterTypeTraits, filterCmpFactories, filterFields, opTrackerFactory, ioOpCallbackFactory,
+ pageWriteCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider,
+ mergePolicyFactory, mergePolicyProperties, bloomFilterFields, bloomFilterFalsePositiveRate,
+ btreeFields, compDecompFactory, typeTraitProvider.getTypeTrait(BuiltinType.ANULL),
+ NullIntrospector.INSTANCE, isSecondaryNoIncrementalMaintenance, columnManagerFactory);
+ }
default:
throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_DATASET_TYPE,
dataset.getDatasetType().toString());
@@ -156,9 +178,7 @@
secondaryTypeTraits[i] = typeTraitProvider.getTypeTrait(keyType);
}
// Add serializers and comparators for primary index fields.
- for (int i = 0; i < numPrimaryKeys; i++) {
- secondaryTypeTraits[numSecondaryKeys + i] = primaryTypeTraits[i];
- }
+ System.arraycopy(primaryTypeTraits, 0, secondaryTypeTraits, numSecondaryKeys, numPrimaryKeys);
return secondaryTypeTraits;
}
@@ -193,9 +213,7 @@
secondaryCmpFactories[i] = cmpFactoryProvider.getBinaryComparatorFactory(keyType, true);
}
// Add serializers and comparators for primary index fields.
- for (int i = 0; i < numPrimaryKeys; i++) {
- secondaryCmpFactories[numSecondaryKeys + i] = primaryCmpFactories[i];
- }
+ System.arraycopy(primaryCmpFactories, 0, secondaryCmpFactories, numSecondaryKeys, numPrimaryKeys);
return secondaryCmpFactories;
}
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 95b0906..4e6119f 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
@@ -33,7 +33,7 @@
import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSource;
import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSourcePropertiesProvider;
-import org.apache.hyracks.algebricks.core.algebra.metadata.IProjectionInfo;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IProjectionFiltrationInfo;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
import org.apache.hyracks.algebricks.core.algebra.properties.FunctionalDependency;
import org.apache.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
@@ -122,6 +122,7 @@
return datasourceType;
}
+ @Override
public Map<String, Serializable> getProperties() {
return properties;
}
@@ -165,5 +166,6 @@
List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars,
ITupleFilterFactory tupleFilterFactory, long outputLimit, IOperatorSchema opSchema,
IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig,
- IProjectionInfo<?> projectionInfo) throws AlgebricksException;
+ IProjectionFiltrationInfo<?> datasetProjectionInfo, IProjectionFiltrationInfo<?> metaProjectionInfo)
+ throws AlgebricksException;
}
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 30f5783..dbcbce0 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
@@ -37,17 +37,18 @@
import org.apache.asterix.metadata.entities.ExternalDatasetDetails;
import org.apache.asterix.metadata.entities.Index;
import org.apache.asterix.metadata.entities.InternalDatasetDetails;
+import org.apache.asterix.metadata.utils.IndexUtil;
import org.apache.asterix.metadata.utils.KeyFieldTypeUtil;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.runtime.projection.DataProjectionInfo;
+import org.apache.asterix.runtime.projection.DataProjectionFiltrationInfo;
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.metadata.IProjectionFiltrationInfo;
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;
@@ -55,7 +56,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;
+import org.apache.hyracks.storage.common.projection.ITupleProjectorFactory;
public class DatasetDataSource extends DataSource {
@@ -120,15 +121,16 @@
List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars,
ITupleFilterFactory tupleFilterFactory, long outputLimit, IOperatorSchema opSchema,
IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig,
- IProjectionInfo<?> projectionInfo) throws AlgebricksException {
+ IProjectionFiltrationInfo<?> projectionInfo, IProjectionFiltrationInfo<?> metaProjectionInfo)
+ throws AlgebricksException {
+ String itemTypeName = dataset.getItemTypeName();
+ IAType itemType = MetadataManager.INSTANCE
+ .getDatatype(metadataProvider.getMetadataTxnContext(), dataset.getItemTypeDataverseName(), itemTypeName)
+ .getDatatype();
switch (dataset.getDatasetType()) {
case EXTERNAL:
DatasetDataSource externalDataSource = (DatasetDataSource) dataSource;
Dataset externalDataset = externalDataSource.getDataset();
- String itemTypeName = externalDataset.getItemTypeName();
- IAType itemType = MetadataManager.INSTANCE.getDatatype(metadataProvider.getMetadataTxnContext(),
- externalDataset.getItemTypeDataverseName(), itemTypeName).getDatatype();
-
ExternalDatasetDetails edd = (ExternalDatasetDetails) externalDataset.getDatasetDetails();
PhysicalOptimizationConfig physicalOptimizationConfig = context.getPhysicalOptimizationConfig();
int externalScanBufferSize = physicalOptimizationConfig.getExternalScanBufferSize();
@@ -146,25 +148,38 @@
Index primaryIndex = MetadataManager.INSTANCE.getIndex(metadataProvider.getMetadataTxnContext(),
dataverseName, datasetName, datasetName);
+ ARecordType datasetType = (ARecordType) itemType;
+ ARecordType metaItemType = null;
+ if (dataset.hasMetaPart()) {
+ metaItemType = (ARecordType) MetadataManager.INSTANCE
+ .getDatatype(metadataProvider.getMetadataTxnContext(),
+ dataset.getMetaItemTypeDataverseName(), dataset.getMetaItemTypeName())
+ .getDatatype();
+ }
+ int numberOfPrimaryKeys = dataset.getPrimaryKeys().size();
+ ITupleProjectorFactory tupleProjectorFactory =
+ IndexUtil.createTupleProjectorFactory(dataset.getDatasetFormatInfo(), projectionInfo,
+ metaProjectionInfo, datasetType, metaItemType, numberOfPrimaryKeys);
+
int[] minFilterFieldIndexes = createFilterIndexes(minFilterVars, opSchema);
int[] maxFilterFieldIndexes = createFilterIndexes(maxFilterVars, opSchema);
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, DefaultTupleProjectorFactory.INSTANCE);
+ outputLimit, false, false, tupleProjectorFactory);
default:
throw new AlgebricksException("Unknown datasource type");
}
}
- private Map<String, String> addExternalProjectionInfo(IProjectionInfo<?> projectionInfo,
+ private Map<String, String> addExternalProjectionInfo(IProjectionFiltrationInfo<?> projectionInfo,
Map<String, String> properties) {
Map<String, String> propertiesCopy = properties;
if (projectionInfo != null) {
//properties could be cached and reused, so we make a copy per query
propertiesCopy = new HashMap<>(properties);
try {
- DataProjectionInfo externalProjectionInfo = (DataProjectionInfo) projectionInfo;
+ DataProjectionFiltrationInfo externalProjectionInfo = (DataProjectionFiltrationInfo) projectionInfo;
ExternalDataUtils.setExternalDataProjectionInfo(externalProjectionInfo, propertiesCopy);
} catch (IOException e) {
throw new IllegalStateException(e);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FeedDataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FeedDataSource.java
index dc7b19b..0d65083 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FeedDataSource.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FeedDataSource.java
@@ -42,7 +42,7 @@
import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSource;
-import org.apache.hyracks.algebricks.core.algebra.metadata.IProjectionInfo;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IProjectionFiltrationInfo;
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;
@@ -164,7 +164,8 @@
List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars,
ITupleFilterFactory tupleFilterFactory, long outputLimit, IOperatorSchema opSchema,
IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig,
- IProjectionInfo<?> projectionInfo) throws AlgebricksException {
+ IProjectionFiltrationInfo<?> projectionInfo, IProjectionFiltrationInfo<?> metaProjectionInfo)
+ throws AlgebricksException {
try {
if (tupleFilterFactory != null || outputLimit >= 0) {
throw CompilationException.create(ErrorCode.COMPILATION_ILLEGAL_STATE,
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FunctionDataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FunctionDataSource.java
index a7d560c..ef6b834 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FunctionDataSource.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FunctionDataSource.java
@@ -39,7 +39,7 @@
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSource;
import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSourcePropertiesProvider;
-import org.apache.hyracks.algebricks.core.algebra.metadata.IProjectionInfo;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IProjectionFiltrationInfo;
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.algebra.properties.RandomPartitioningProperty;
@@ -92,7 +92,8 @@
List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars,
ITupleFilterFactory tupleFilterFactory, long outputLimit, IOperatorSchema opSchema,
IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig,
- IProjectionInfo<?> projectionInfo) throws AlgebricksException {
+ IProjectionFiltrationInfo<?> projectionInfo, IProjectionFiltrationInfo<?> metaProjectionInfo)
+ throws AlgebricksException {
GenericAdapterFactory adapterFactory = new GenericAdapterFactory();
adapterFactory.setOutputType(RecordUtil.FULLY_OPEN_RECORD_TYPE);
IClusterStateManager csm = metadataProvider.getApplicationContext().getClusterStateManager();
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/LoadableDataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/LoadableDataSource.java
index 79a9d00..c7ccc53 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/LoadableDataSource.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/LoadableDataSource.java
@@ -40,7 +40,7 @@
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.metadata.IProjectionFiltrationInfo;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenHelper;
@@ -136,7 +136,8 @@
List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars,
ITupleFilterFactory tupleFilterFactory, long outputLimit, IOperatorSchema opSchema,
IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig,
- IProjectionInfo<?> projectionInfo) throws AlgebricksException {
+ IProjectionFiltrationInfo<?> projectionInfo, IProjectionFiltrationInfo<?> metaProjectionInfo)
+ throws AlgebricksException {
if (tupleFilterFactory != null || outputLimit >= 0) {
throw CompilationException.create(ErrorCode.COMPILATION_ILLEGAL_STATE,
"tuple filter and limit are not supported by LoadableDataSource");
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 dc2d017..c55925c 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
@@ -130,7 +130,7 @@
import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSource;
import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSourceIndex;
import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
-import org.apache.hyracks.algebricks.core.algebra.metadata.IProjectionInfo;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IProjectionFiltrationInfo;
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;
@@ -494,10 +494,11 @@
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 {
+ Object implConfig, IProjectionFiltrationInfo<?> projectionInfo,
+ IProjectionFiltrationInfo<?> metaProjectionInfo) throws AlgebricksException {
return ((DataSource) dataSource).buildDatasourceScanRuntime(this, dataSource, scanVariables, projectVariables,
projectPushed, minFilterVars, maxFilterVars, tupleFilterFactory, outputLimit, opSchema, typeEnv,
- context, jobSpec, implConfig, projectionInfo);
+ context, jobSpec, implConfig, projectionInfo, metaProjectionInfo);
}
protected Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildLoadableDatasetScan(
@@ -1505,12 +1506,8 @@
List<LogicalVariable> prevAdditionalFilteringKeys) throws AlgebricksException {
// Check the index is length-partitioned or not.
boolean isPartitioned;
- if (indexType == IndexType.LENGTH_PARTITIONED_WORD_INVIX
- || indexType == IndexType.LENGTH_PARTITIONED_NGRAM_INVIX) {
- isPartitioned = true;
- } else {
- isPartitioned = false;
- }
+ isPartitioned = indexType == IndexType.LENGTH_PARTITIONED_WORD_INVIX
+ || indexType == IndexType.LENGTH_PARTITIONED_NGRAM_INVIX;
// Sanity checks.
if (primaryKeys.size() > 1) {
@@ -1626,12 +1623,8 @@
}
boolean isPartitioned;
- if (indexType == IndexType.LENGTH_PARTITIONED_WORD_INVIX
- || indexType == IndexType.LENGTH_PARTITIONED_NGRAM_INVIX) {
- isPartitioned = true;
- } else {
- isPartitioned = false;
- }
+ isPartitioned = indexType == IndexType.LENGTH_PARTITIONED_WORD_INVIX
+ || indexType == IndexType.LENGTH_PARTITIONED_NGRAM_INVIX;
// Number of Keys that needs to be propagated
int numKeys = inputSchema.getSize();
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
index 8ed3e16..a2a0f19 100644
--- 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
@@ -29,7 +29,7 @@
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.metadata.IProjectionFiltrationInfo;
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;
@@ -60,7 +60,8 @@
List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars,
ITupleFilterFactory tupleFilterFactory, long outputLimit, IOperatorSchema opSchema,
IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig,
- IProjectionInfo<?> projectionInfo) throws AlgebricksException {
+ IProjectionFiltrationInfo<?> projectionInfo, IProjectionFiltrationInfo<?> metaProjectionInfo)
+ 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);
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 e3935a4..0774928 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
@@ -55,6 +55,7 @@
import org.apache.asterix.metadata.MetadataManager;
import org.apache.asterix.metadata.MetadataTransactionContext;
import org.apache.asterix.metadata.api.IMetadataEntity;
+import org.apache.asterix.metadata.dataset.DatasetFormatInfo;
import org.apache.asterix.metadata.declared.ArrayBTreeResourceFactoryProvider;
import org.apache.asterix.metadata.declared.BTreeResourceFactoryProvider;
import org.apache.asterix.metadata.declared.MetadataProvider;
@@ -157,6 +158,7 @@
private int pendingOp;
private final String compressionScheme;
private final DatasetFullyQualifiedName datasetFullyQualifiedName;
+ private final DatasetFormatInfo datasetFormatInfo;
public Dataset(DataverseName dataverseName, String datasetName, DataverseName recordTypeDataverseName,
String recordTypeName, String nodeGroupName, String compactionPolicy,
@@ -164,17 +166,17 @@
DatasetType datasetType, int datasetId, int pendingOp) {
this(dataverseName, datasetName, recordTypeDataverseName, recordTypeName, /*metaTypeDataverseName*/null,
/*metaTypeName*/null, nodeGroupName, compactionPolicy, compactionPolicyProperties, datasetDetails,
- hints, datasetType, datasetId, pendingOp, CompressionManager.NONE);
+ hints, datasetType, datasetId, pendingOp, CompressionManager.NONE, DatasetFormatInfo.DEFAULT);
}
public Dataset(DataverseName dataverseName, String datasetName, DataverseName itemTypeDataverseName,
String itemTypeName, DataverseName metaItemTypeDataverseName, String metaItemTypeName, String nodeGroupName,
String compactionPolicy, Map<String, String> compactionPolicyProperties, IDatasetDetails datasetDetails,
- Map<String, String> hints, DatasetType datasetType, int datasetId, int pendingOp,
- String compressionScheme) {
+ Map<String, String> hints, DatasetType datasetType, int datasetId, int pendingOp, String compressionScheme,
+ DatasetFormatInfo datasetFormatInfo) {
this(dataverseName, datasetName, itemTypeDataverseName, itemTypeName, metaItemTypeDataverseName,
metaItemTypeName, nodeGroupName, compactionPolicy, compactionPolicyProperties, datasetDetails, hints,
- datasetType, datasetId, pendingOp, 0L, compressionScheme);
+ datasetType, datasetId, pendingOp, 0L, compressionScheme, datasetFormatInfo);
}
public Dataset(Dataset dataset) {
@@ -182,14 +184,14 @@
dataset.metaTypeDataverseName, dataset.metaTypeName, dataset.nodeGroupName,
dataset.compactionPolicyFactory, dataset.compactionPolicyProperties, dataset.datasetDetails,
dataset.hints, dataset.datasetType, dataset.datasetId, dataset.pendingOp, dataset.rebalanceCount,
- dataset.compressionScheme);
+ dataset.compressionScheme, dataset.datasetFormatInfo);
}
public Dataset(DataverseName dataverseName, String datasetName, DataverseName itemTypeDataverseName,
String itemTypeName, DataverseName metaItemTypeDataverseName, String metaItemTypeName, String nodeGroupName,
String compactionPolicy, Map<String, String> compactionPolicyProperties, IDatasetDetails datasetDetails,
Map<String, String> hints, DatasetType datasetType, int datasetId, int pendingOp, long rebalanceCount,
- String compressionScheme) {
+ String compressionScheme, DatasetFormatInfo datasetFormatInfo) {
this.dataverseName = dataverseName;
this.datasetName = datasetName;
this.recordTypeName = itemTypeName;
@@ -207,6 +209,7 @@
this.rebalanceCount = rebalanceCount;
this.compressionScheme = compressionScheme;
datasetFullyQualifiedName = new DatasetFullyQualifiedName(dataverseName, datasetName);
+ this.datasetFormatInfo = datasetFormatInfo;
}
@Override
@@ -332,21 +335,14 @@
/**
* Drop this dataset
*
- * @param metadataProvider
- * metadata provider that can be used to get metadata info and runtimes
- * @param mdTxnCtx
- * the transaction context
- * @param jobsToExecute
- * a list of jobs to be executed as part of the drop operation
- * @param bActiveTxn
- * whether the metadata transaction is ongoing
- * @param progress
- * a mutable progress state used for error handling during the drop operation
- * @param hcc
- * a client connection to hyracks master for job execution
+ * @param metadataProvider metadata provider that can be used to get metadata info and runtimes
+ * @param mdTxnCtx the transaction context
+ * @param jobsToExecute a list of jobs to be executed as part of the drop operation
+ * @param bActiveTxn whether the metadata transaction is ongoing
+ * @param progress a mutable progress state used for error handling during the drop operation
+ * @param hcc a client connection to hyracks master for job execution
* @param sourceLoc
- * @throws Exception
- * if an error occur during the drop process or if the dataset can't be dropped for any reason
+ * @throws Exception if an error occur during the drop process or if the dataset can't be dropped for any reason
*/
public void drop(MetadataProvider metadataProvider, MutableObject<MetadataTransactionContext> mdTxnCtx,
List<JobSpecification> jobsToExecute, MutableBoolean bActiveTxn, MutableObject<ProgressState> progress,
@@ -370,7 +366,8 @@
new Dataset(dataverseName, datasetName, getItemTypeDataverseName(), getItemTypeName(),
getMetaItemTypeDataverseName(), getMetaItemTypeName(), getNodeGroupName(),
getCompactionPolicy(), getCompactionPolicyProperties(), getDatasetDetails(), getHints(),
- getDatasetType(), getDatasetId(), MetadataUtil.PENDING_DROP_OP, getCompressionScheme()));
+ getDatasetType(), getDatasetId(), MetadataUtil.PENDING_DROP_OP, getCompressionScheme(),
+ getDatasetFormatInfo()));
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx.getValue());
bActiveTxn.setValue(false);
@@ -450,22 +447,15 @@
/**
* Create the index dataflow helper factory for a particular index on the dataset
*
- * @param mdProvider
- * metadata provider to get metadata information, components, and runtimes
- * @param index
- * the index to get the dataflow helper factory for
- * @param recordType
- * the record type for the dataset
- * @param metaType
- * the meta type for the dataset
- * @param mergePolicyFactory
- * the merge policy factory of the dataset
- * @param mergePolicyProperties
- * the merge policy properties for the dataset
+ * @param mdProvider metadata provider to get metadata information, components, and runtimes
+ * @param index the index to get the dataflow helper factory for
+ * @param recordType the record type for the dataset
+ * @param metaType the meta type for the dataset
+ * @param mergePolicyFactory the merge policy factory of the dataset
+ * @param mergePolicyProperties the merge policy properties for the dataset
* @return indexDataflowHelperFactory
- * an instance of {@link org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory}
- * @throws AlgebricksException
- * if dataflow helper factory could not be created
+ * an instance of {@link org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory}
+ * @throws AlgebricksException if dataflow helper factory could not be created
*/
public IResourceFactory getResourceFactory(MetadataProvider mdProvider, Index index, ARecordType recordType,
ARecordType metaType, ILSMMergePolicyFactory mergePolicyFactory, Map<String, String> mergePolicyProperties)
@@ -511,13 +501,11 @@
/**
* Get the IO Operation callback factory for the index which belongs to this dataset
*
- * @param index
- * the index
+ * @param index the index
* @return ioOperationCallbackFactory
- * an instance of {@link org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory}
- * to be used with IO operations
- * @throws AlgebricksException
- * if the factory could not be created for the index/dataset combination
+ * an instance of {@link org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory}
+ * to be used with IO operations
+ * @throws AlgebricksException if the factory could not be created for the index/dataset combination
*/
@SuppressWarnings("squid:S1172")
public ILSMIOOperationCallbackFactory getIoOperationCallbackFactory(Index index) throws AlgebricksException {
@@ -531,8 +519,7 @@
/**
* get the IndexOperationTrackerFactory for a particular index on the dataset
*
- * @param index
- * the index
+ * @param index the index
* @return an instance of {@link org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory}
*/
public ILSMOperationTrackerFactory getIndexOperationTrackerFactory(Index index) {
@@ -551,22 +538,14 @@
/**
* Get search callback factory for this dataset with the passed index and operation
*
- * @param storageComponentProvider
- * storage component provider
- * @param index
- * the index
- * @param op
- * the operation this search is part of
- * @param primaryKeyFields
- * the primary key fields indexes for locking purposes
- * @param primaryKeyFieldsInSecondaryIndex
- * the primary key fields indexes in the given secondary index (used for index-only plan)
- * @param proceedIndexOnlyPlan
- * the given plan is an index-only plan? (used for index-only plan)
- * @return
- * an instance of {@link org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory}
- * @throws AlgebricksException
- * if the callback factory could not be created
+ * @param storageComponentProvider storage component provider
+ * @param index the index
+ * @param op the operation this search is part of
+ * @param primaryKeyFields the primary key fields indexes for locking purposes
+ * @param primaryKeyFieldsInSecondaryIndex the primary key fields indexes in the given secondary index (used for index-only plan)
+ * @param proceedIndexOnlyPlan the given plan is an index-only plan? (used for index-only plan)
+ * @return an instance of {@link org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory}
+ * @throws AlgebricksException if the callback factory could not be created
*/
public ISearchOperationCallbackFactory getSearchCallbackFactory(IStorageComponentProvider storageComponentProvider,
Index index, IndexOperation op, int[] primaryKeyFields, int[] primaryKeyFieldsInSecondaryIndex,
@@ -599,18 +578,12 @@
/**
* Get search callback factory for this dataset with the passed index and operation
*
- * @param storageComponentProvider
- * storage component provider
- * @param index
- * the index
- * @param op
- * the operation this search is part of
- * @param primaryKeyFields
- * the primary key fields indexes for locking purposes
- * @return
- * an instance of {@link org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory}
- * @throws AlgebricksException
- * if the callback factory could not be created
+ * @param storageComponentProvider storage component provider
+ * @param index the index
+ * @param op the operation this search is part of
+ * @param primaryKeyFields the primary key fields indexes for locking purposes
+ * @return an instance of {@link org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory}
+ * @throws AlgebricksException if the callback factory could not be created
*/
public ISearchOperationCallbackFactory getSearchCallbackFactory(IStorageComponentProvider storageComponentProvider,
Index index, IndexOperation op, int[] primaryKeyFields) throws AlgebricksException {
@@ -620,16 +593,11 @@
/**
* Get the modification callback factory associated with this dataset, the passed index, and operation.
*
- * @param index
- * the index
- * @param op
- * the operation performed for this callback
- * @param primaryKeyFields
- * the indexes of the primary keys (used for lock operations)
- * @return
- * an instance of {@link org.apache.hyracks.storage.am.common.api.IModificationOperationCallbackFactory}
- * @throws AlgebricksException
- * If the callback factory could not be created
+ * @param index the index
+ * @param op the operation performed for this callback
+ * @param primaryKeyFields the indexes of the primary keys (used for lock operations)
+ * @return an instance of {@link org.apache.hyracks.storage.am.common.api.IModificationOperationCallbackFactory}
+ * @throws AlgebricksException If the callback factory could not be created
*/
public IModificationOperationCallbackFactory getModificationCallbackFactory(
IStorageComponentProvider componentProvider, Index index, IndexOperation op, int[] primaryKeyFields)
@@ -680,6 +648,7 @@
tree.put("pendingOp", MetadataUtil.pendingOpToString(pendingOp));
tree.put("rebalanceCount", rebalanceCount);
tree.put("compressionScheme", compressionScheme);
+ tree.put("datasetFormatInfo", datasetFormatInfo.toString());
return tree;
}
@@ -691,12 +660,9 @@
/**
* Gets the commit runtime factory for inserting/upserting/deleting operations on this dataset.
*
- * @param metadataProvider,
- * the metadata provider.
- * @param primaryKeyFieldPermutation,
- * the primary key field permutation according to the input.
- * @param isSink,
- * whether this commit runtime is the last operator in the pipeline.
+ * @param metadataProvider, the metadata provider.
+ * @param primaryKeyFieldPermutation, the primary key field permutation according to the input.
+ * @param isSink, whether this commit runtime is the last operator in the pipeline.
* @return the commit runtime factory for inserting/upserting/deleting operations on this dataset.
* @throws AlgebricksException
*/
@@ -724,10 +690,10 @@
}
/**
- @return the array of type traits that contains the following type traits in order
- 1) the primary keys,
- 2) the query record type,
- 3) the metadata type trait if the dataset has metadata
+ * @return the array of type traits that contains the following type traits in order
+ * 1) the primary keys,
+ * 2) the query record type,
+ * 3) the metadata type trait if the dataset has metadata
*/
// ToDo: returning such an array can be confusing because it may contain the metadata type or not.
// instead of returning an array, create a new class that contains 1) a type trait array for the primary keys,
@@ -760,8 +726,7 @@
/**
* Gets the record descriptor for primary records of this dataset.
*
- * @param metadataProvider,
- * the metadata provider.
+ * @param metadataProvider, the metadata provider.
* @return the record descriptor for primary records of this dataset.
* @throws AlgebricksException
*/
@@ -805,8 +770,7 @@
/**
* Gets the comparator factories for the primary key fields of this dataset.
*
- * @param metadataProvider,
- * the metadata provider.
+ * @param metadataProvider, the metadata provider.
* @return the comparator factories for the primary key fields of this dataset.
* @throws AlgebricksException
*/
@@ -834,8 +798,7 @@
/**
* Gets the hash function factories for the primary key fields of this dataset.
*
- * @param metadataProvider,
- * the metadata provider.
+ * @param metadataProvider, the metadata provider.
* @return the hash function factories for the primary key fields of this dataset.
* @throws AlgebricksException
*/
@@ -873,7 +836,7 @@
this.metaTypeDataverseName, this.metaTypeName, targetNodeGroupName, this.compactionPolicyFactory,
this.compactionPolicyProperties, this.datasetDetails, this.hints, this.datasetType,
DatasetIdFactory.generateAlternatingDatasetId(this.datasetId), this.pendingOp, this.rebalanceCount + 1,
- this.compressionScheme);
+ this.compressionScheme, this.datasetFormatInfo);
}
// Gets an array of partition numbers for this dataset.
@@ -895,4 +858,7 @@
return datasetFullyQualifiedName;
}
+ public DatasetFormatInfo getDatasetFormatInfo() {
+ return datasetFormatInfo;
+ }
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/AbstractTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/AbstractTupleTranslator.java
index 3f4e63e..1ae2430 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/AbstractTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/AbstractTupleTranslator.java
@@ -35,6 +35,7 @@
import org.apache.asterix.metadata.api.IMetadataIndex;
import org.apache.asterix.metadata.bootstrap.MetadataRecordTypes;
import org.apache.asterix.om.base.ABoolean;
+import org.apache.asterix.om.base.ADouble;
import org.apache.asterix.om.base.AInt32;
import org.apache.asterix.om.base.AInt64;
import org.apache.asterix.om.base.AMutableString;
@@ -77,6 +78,9 @@
@SuppressWarnings("unchecked")
protected final ISerializerDeserializer<AInt64> int64Serde =
SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT64);
+ @SuppressWarnings("unchecked")
+ protected final ISerializerDeserializer<ADouble> doubleSerde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADOUBLE);
protected final ISerializerDeserializer<ARecord> recordSerDes;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
index 0d05d46..eafa331 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
@@ -33,6 +33,7 @@
import org.apache.asterix.builders.OrderedListBuilder;
import org.apache.asterix.builders.RecordBuilder;
import org.apache.asterix.builders.UnorderedListBuilder;
+import org.apache.asterix.common.config.DatasetConfig;
import org.apache.asterix.common.config.DatasetConfig.DatasetType;
import org.apache.asterix.common.config.DatasetConfig.TransactionState;
import org.apache.asterix.common.exceptions.AsterixException;
@@ -42,6 +43,7 @@
import org.apache.asterix.metadata.IDatasetDetails;
import org.apache.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
import org.apache.asterix.metadata.bootstrap.MetadataRecordTypes;
+import org.apache.asterix.metadata.dataset.DatasetFormatInfo;
import org.apache.asterix.metadata.entities.Dataset;
import org.apache.asterix.metadata.entities.ExternalDatasetDetails;
import org.apache.asterix.metadata.entities.InternalDatasetDetails;
@@ -51,9 +53,11 @@
import org.apache.asterix.metadata.utils.DatasetUtil;
import org.apache.asterix.om.base.ABoolean;
import org.apache.asterix.om.base.ADateTime;
+import org.apache.asterix.om.base.ADouble;
import org.apache.asterix.om.base.AInt32;
import org.apache.asterix.om.base.AInt64;
import org.apache.asterix.om.base.AInt8;
+import org.apache.asterix.om.base.AMutableDouble;
import org.apache.asterix.om.base.AMutableInt32;
import org.apache.asterix.om.base.AMutableInt64;
import org.apache.asterix.om.base.AMutableString;
@@ -88,12 +92,14 @@
protected AMutableInt32 aInt32;
protected AMutableInt64 aInt64;
+ protected AMutableDouble aDouble;
protected DatasetTupleTranslator(boolean getTuple) {
super(getTuple, MetadataPrimaryIndexes.DATASET_DATASET, DATASET_PAYLOAD_TUPLE_FIELD_INDEX);
if (getTuple) {
aInt32 = new AMutableInt32(-1);
aInt64 = new AMutableInt64(-1);
+ aDouble = new AMutableDouble(0.0);
}
}
@@ -369,10 +375,11 @@
long rebalanceCount = getRebalanceCount(datasetRecord);
String compressionScheme = getCompressionScheme(datasetRecord);
+ DatasetFormatInfo datasetFormatInfo = getDatasetFormatInfo(datasetRecord);
return new Dataset(dataverseName, datasetName, typeDataverseName, typeName, metaTypeDataverseName, metaTypeName,
nodeGroupName, compactionPolicy.first, compactionPolicy.second, datasetDetails, hints, datasetType,
- datasetId, pendingOp, rebalanceCount, compressionScheme);
+ datasetId, pendingOp, rebalanceCount, compressionScheme, datasetFormatInfo);
}
protected Pair<String, Map<String, String>> readCompactionPolicy(DatasetType datasetType, ARecord datasetRecord) {
@@ -421,6 +428,39 @@
return CompressionManager.NONE;
}
+ private DatasetFormatInfo getDatasetFormatInfo(ARecord datasetRecord) {
+ ARecordType datasetType = datasetRecord.getType();
+ int datasetFormatIndex =
+ datasetType.getFieldIndex(MetadataRecordTypes.DATASET_ARECORD_DATASET_FORMAT_FIELD_NAME);
+ if (datasetFormatIndex < 0) {
+ return DatasetFormatInfo.DEFAULT;
+ }
+
+ ARecordType datasetFormatType = (ARecordType) datasetType.getFieldTypes()[datasetFormatIndex];
+ int formatIndex =
+ datasetFormatType.getFieldIndex(MetadataRecordTypes.DATASET_ARECORD_DATASET_FORMAT_FORMAT_FIELD_NAME);
+ int maxTupleCountIndex =
+ datasetFormatType.getFieldIndex(MetadataRecordTypes.DATASET_ARECORD_DATASET_MAX_TUPLE_COUNT_FIELD_NAME);
+ int freeSpaceToleranceIndex = datasetFormatType
+ .getFieldIndex(MetadataRecordTypes.DATASET_ARECORD_DATASET_FREE_SPACE_TOLERANCE_FIELD_NAME);
+
+ ARecord datasetFormatRecord = (ARecord) datasetRecord.getValueByPos(datasetFormatIndex);
+
+ //Format
+ AString formatString = (AString) datasetFormatRecord.getValueByPos(formatIndex);
+ DatasetConfig.DatasetFormat format = DatasetConfig.DatasetFormat.valueOf(formatString.getStringValue());
+
+ //MaxTupleCount
+ AInt64 maxTupleCountInt = (AInt64) datasetFormatRecord.getValueByPos(maxTupleCountIndex);
+ int maxTupleCount = (int) maxTupleCountInt.getLongValue();
+
+ //FreeSpaceTolerance
+ ADouble freeSpaceToleranceDouble = (ADouble) datasetFormatRecord.getValueByPos(freeSpaceToleranceIndex);
+ float freeSpaceTolerance = (float) freeSpaceToleranceDouble.getDoubleValue();
+
+ return new DatasetFormatInfo(format, maxTupleCount, freeSpaceTolerance);
+ }
+
@Override
public ITupleReference getTupleFromMetadataEntity(Dataset dataset) throws HyracksDataException {
OrderedListBuilder listBuilder = new OrderedListBuilder();
@@ -575,6 +615,7 @@
writeRebalanceCount(dataset);
writeBlockLevelStorageCompression(dataset);
writeOpenDetails(dataset);
+ writeDatasetFormatInfo(dataset);
}
private void writeOpenDetails(Dataset dataset) throws HyracksDataException {
@@ -633,6 +674,47 @@
recordBuilder.addField(fieldName, fieldValue);
}
+ private void writeDatasetFormatInfo(Dataset dataset) throws HyracksDataException {
+ DatasetFormatInfo info = dataset.getDatasetFormatInfo();
+ if (DatasetFormatInfo.DEFAULT == info) {
+ return;
+ }
+
+ RecordBuilder datasetFormatObject = new RecordBuilder();
+ datasetFormatObject.reset(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
+
+ fieldName.reset();
+ aString.setValue(MetadataRecordTypes.DATASET_ARECORD_DATASET_FORMAT_FORMAT_FIELD_NAME);
+ stringSerde.serialize(aString, fieldName.getDataOutput());
+ fieldValue.reset();
+ aString.setValue(info.getFormat().toString());
+ stringSerde.serialize(aString, fieldValue.getDataOutput());
+ datasetFormatObject.addField(fieldName, fieldValue);
+
+ fieldName.reset();
+ aString.setValue(MetadataRecordTypes.DATASET_ARECORD_DATASET_MAX_TUPLE_COUNT_FIELD_NAME);
+ stringSerde.serialize(aString, fieldName.getDataOutput());
+ fieldValue.reset();
+ aInt64.setValue(info.getMaxTupleCount());
+ int64Serde.serialize(aInt64, fieldValue.getDataOutput());
+ datasetFormatObject.addField(fieldName, fieldValue);
+
+ fieldName.reset();
+ aString.setValue(MetadataRecordTypes.DATASET_ARECORD_DATASET_FREE_SPACE_TOLERANCE_FIELD_NAME);
+ stringSerde.serialize(aString, fieldName.getDataOutput());
+ fieldValue.reset();
+ aDouble.setValue(info.getFreeSpaceTolerance());
+ doubleSerde.serialize(aDouble, fieldValue.getDataOutput());
+ datasetFormatObject.addField(fieldName, fieldValue);
+
+ fieldName.reset();
+ aString.setValue(MetadataRecordTypes.DATASET_ARECORD_DATASET_FORMAT_FIELD_NAME);
+ stringSerde.serialize(aString, fieldName.getDataOutput());
+ fieldValue.reset();
+ datasetFormatObject.write(fieldValue.getDataOutput(), true);
+ recordBuilder.addField(fieldName, fieldValue);
+ }
+
private void writeRebalanceCount(Dataset dataset) throws HyracksDataException {
if (dataset.getRebalanceCount() > 0) {
// Adds the field rebalanceCount.
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ColumnFilterBuilder.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ColumnFilterBuilder.java
new file mode 100644
index 0000000..e8c2b7f
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ColumnFilterBuilder.java
@@ -0,0 +1,186 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF 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 static org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions.ComparisonKind;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.asterix.column.values.reader.filter.IColumnFilterEvaluatorFactory;
+import org.apache.asterix.column.values.reader.filter.IColumnFilterValueAccessorFactory;
+import org.apache.asterix.column.values.reader.filter.compartor.GEColumnFilterEvaluatorFactory;
+import org.apache.asterix.column.values.reader.filter.compartor.GTColumnFilterEvaluatorFactory;
+import org.apache.asterix.column.values.reader.filter.compartor.LEColumnFilterEvaluatorFactory;
+import org.apache.asterix.column.values.reader.filter.compartor.LTColumnFilterEvaluatorFactory;
+import org.apache.asterix.column.values.reader.filter.evaluator.ANDColumnFilterEvaluatorFactory;
+import org.apache.asterix.column.values.reader.filter.evaluator.NoOpColumnFilterEvaluatorFactory;
+import org.apache.asterix.column.values.reader.filter.evaluator.ORColumnFilterEvaluatorFactory;
+import org.apache.asterix.column.values.reader.filter.value.ColumnFilterValueAccessorFactory;
+import org.apache.asterix.column.values.reader.filter.value.ConstantColumnFilterValueAccessorFactory;
+import org.apache.asterix.om.base.IAObject;
+import org.apache.asterix.om.constants.AsterixConstantValue;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.runtime.projection.DataProjectionFiltrationInfo;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+
+public class ColumnFilterBuilder {
+ public static final Set<FunctionIdentifier> COMPARE_FUNCTIONS = getCompareFunctions();
+ public static final Set<FunctionIdentifier> PUSHABLE_FUNCTIONS = getPushableFunctions();
+
+ private final Map<ILogicalExpression, ARecordType> filterPaths;
+ private final ILogicalExpression filterExpression;
+
+ public ColumnFilterBuilder(DataProjectionFiltrationInfo projectionFiltrationInfo) {
+ this.filterPaths = projectionFiltrationInfo.getExpressionToPath();
+ this.filterExpression = projectionFiltrationInfo.getFilterExpression();
+ }
+
+ public IColumnFilterEvaluatorFactory build() {
+ if (filterExpression == null || filterPaths.isEmpty()) {
+ return NoOpColumnFilterEvaluatorFactory.INSTANCE;
+ }
+ return createEvaluator(filterPaths, filterExpression);
+ }
+
+ private IColumnFilterEvaluatorFactory createEvaluator(Map<ILogicalExpression, ARecordType> filterPaths,
+ ILogicalExpression filterExpression) {
+ AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) filterExpression;
+ FunctionIdentifier fid = funcExpr.getFunctionIdentifier();
+
+ if (COMPARE_FUNCTIONS.contains(fid)) {
+ return createComparator(fid, funcExpr.getArguments(), filterPaths);
+ }
+ return createEvaluatorsForArgs(funcExpr, filterPaths);
+ }
+
+ private IColumnFilterEvaluatorFactory createComparator(FunctionIdentifier fid,
+ List<Mutable<ILogicalExpression>> arguments, Map<ILogicalExpression, ARecordType> filterPaths) {
+ ILogicalExpression arg0 = arguments.get(0).getValue();
+ ILogicalExpression arg1 = arguments.get(1).getValue();
+
+ if (arg1.getExpressionTag() == LogicalExpressionTag.CONSTANT) {
+ ARecordType path = filterPaths.get(arg0);
+ IAObject constant = getConstant(arg1);
+ return createComparator(fid, path, constant, true);
+ } else {
+ ARecordType path = filterPaths.get(arg1);
+ IAObject constant = getConstant(arg0);
+ return createComparator(fid, path, constant, false);
+ }
+ }
+
+ private IColumnFilterEvaluatorFactory createEvaluatorsForArgs(AbstractFunctionCallExpression funcExpr,
+ Map<ILogicalExpression, ARecordType> filterPaths) {
+ List<Mutable<ILogicalExpression>> args = funcExpr.getArguments();
+ FunctionIdentifier fid = funcExpr.getFunctionIdentifier();
+
+ //AND/OR have at least two arguments
+ IColumnFilterEvaluatorFactory arg0 = createEvaluator(filterPaths, args.get(0).getValue());
+ IColumnFilterEvaluatorFactory arg1 = createEvaluator(filterPaths, args.get(1).getValue());
+
+ IColumnFilterEvaluatorFactory left = createEvaluator(fid, arg0, arg1);
+ for (int i = 2; i < args.size() && left != null; i++) {
+ IColumnFilterEvaluatorFactory right = createEvaluator(filterPaths, args.get(i).getValue());
+ left = createEvaluator(fid, left, right);
+ }
+ return left;
+ }
+
+ private IColumnFilterEvaluatorFactory createComparator(FunctionIdentifier fid, ARecordType path, IAObject constant,
+ boolean leftToRight) {
+ ComparisonKind comparisonKind = AlgebricksBuiltinFunctions.getComparisonType(fid);
+
+ IColumnFilterValueAccessorFactory constValue = ConstantColumnFilterValueAccessorFactory.createFactory(constant);
+ IColumnFilterValueAccessorFactory min = new ColumnFilterValueAccessorFactory(path, true);
+ IColumnFilterValueAccessorFactory max = new ColumnFilterValueAccessorFactory(path, false);
+
+ if (leftToRight) {
+ return createEvaluator(comparisonKind, min, constValue, max);
+ }
+ return createEvaluator(invert(comparisonKind), min, constValue, max);
+ }
+
+ private static IColumnFilterEvaluatorFactory createEvaluator(FunctionIdentifier fid,
+ IColumnFilterEvaluatorFactory left, IColumnFilterEvaluatorFactory right) {
+ if (right == null) {
+ return null;
+ }
+ if (BuiltinFunctions.AND.equals(fid)) {
+ return new ANDColumnFilterEvaluatorFactory(left, right);
+ }
+ return new ORColumnFilterEvaluatorFactory(left, right);
+ }
+
+ private static ComparisonKind invert(ComparisonKind comparisonKind) {
+ if (comparisonKind == ComparisonKind.LT) {
+ return ComparisonKind.GE;
+ } else if (comparisonKind == ComparisonKind.LE) {
+ return ComparisonKind.GT;
+ } else if (comparisonKind == ComparisonKind.GT) {
+ return ComparisonKind.LE;
+ }
+ //ComparisonKind.GE
+ return ComparisonKind.LT;
+ }
+
+ private static IColumnFilterEvaluatorFactory createEvaluator(ComparisonKind comparisonKind,
+ IColumnFilterValueAccessorFactory min, IColumnFilterValueAccessorFactory constVal,
+ IColumnFilterValueAccessorFactory max) {
+ if (comparisonKind == ComparisonKind.LT) {
+ return new GTColumnFilterEvaluatorFactory(constVal, min);
+ } else if (comparisonKind == ComparisonKind.LE) {
+ return new GEColumnFilterEvaluatorFactory(constVal, min);
+ } else if (comparisonKind == ComparisonKind.EQ) {
+ IColumnFilterEvaluatorFactory minComp = new GEColumnFilterEvaluatorFactory(constVal, min);
+ IColumnFilterEvaluatorFactory maxComp = new LEColumnFilterEvaluatorFactory(constVal, max);
+ return new ANDColumnFilterEvaluatorFactory(minComp, maxComp);
+ } else if (comparisonKind == ComparisonKind.GT) {
+ return new LTColumnFilterEvaluatorFactory(constVal, max);
+ }
+ //ComparisonKind.GE
+ return new LEColumnFilterEvaluatorFactory(constVal, max);
+ }
+
+ private static IAObject getConstant(ILogicalExpression expr) {
+ return ((AsterixConstantValue) ((ConstantExpression) expr).getValue()).getObject();
+ }
+
+ private static Set<FunctionIdentifier> getCompareFunctions() {
+ return Set.of(AlgebricksBuiltinFunctions.LE, AlgebricksBuiltinFunctions.GE, AlgebricksBuiltinFunctions.LT,
+ AlgebricksBuiltinFunctions.GT, AlgebricksBuiltinFunctions.EQ);
+ }
+
+ private static Set<FunctionIdentifier> getPushableFunctions() {
+ Set<FunctionIdentifier> pushableFunctions = new HashSet<>(COMPARE_FUNCTIONS);
+ pushableFunctions.add(AlgebricksBuiltinFunctions.AND);
+ pushableFunctions.add(AlgebricksBuiltinFunctions.OR);
+ return pushableFunctions;
+ }
+
+}
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 7e9cfc3..fc8fdb6 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
@@ -20,31 +20,43 @@
import static org.apache.hyracks.storage.am.common.dataflow.IndexDropOperatorDescriptor.DropOption;
+import java.util.Collections;
import java.util.EnumSet;
import java.util.List;
+import java.util.Map;
import java.util.Set;
+import org.apache.asterix.column.operation.query.QueryColumnTupleProjectorFactory;
+import org.apache.asterix.column.values.reader.filter.IColumnFilterEvaluatorFactory;
+import org.apache.asterix.column.values.reader.filter.evaluator.NoOpColumnFilterEvaluatorFactory;
import org.apache.asterix.common.config.DatasetConfig;
import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.common.transactions.TxnId;
import org.apache.asterix.external.indexing.ExternalFile;
+import org.apache.asterix.metadata.dataset.DatasetFormatInfo;
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.entities.InternalDatasetDetails;
import org.apache.asterix.om.base.AString;
import org.apache.asterix.om.base.IAObject;
+import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.IAType;
import org.apache.asterix.runtime.job.listener.JobEventListenerFactory;
+import org.apache.asterix.runtime.projection.DataProjectionFiltrationInfo;
+import org.apache.asterix.runtime.projection.FunctionCallInformation;
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.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IProjectionFiltrationInfo;
import org.apache.hyracks.api.dataflow.value.ITypeTraits;
import org.apache.hyracks.api.exceptions.SourceLocation;
import org.apache.hyracks.api.job.IJobletEventListenerFactory;
import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.storage.am.common.impls.DefaultTupleProjectorFactory;
+import org.apache.hyracks.storage.common.projection.ITupleProjectorFactory;
import org.apache.hyracks.util.OptionalBoolean;
public class IndexUtil {
@@ -179,10 +191,8 @@
/**
* Binds a job event listener to the job specification.
*
- * @param spec,
- * the job specification.
- * @param metadataProvider,
- * the metadata provider.
+ * @param spec, the job specification.
+ * @param metadataProvider, the metadata provider.
* @return the AsterixDB job id for transaction management.
*/
public static void bindJobEventListener(JobSpecification spec, MetadataProvider metadataProvider)
@@ -253,4 +263,38 @@
return new Pair<>(MetadataConstants.SAMPLE_INDEX_1_PREFIX + datasetName,
MetadataConstants.SAMPLE_INDEX_2_PREFIX + datasetName);
}
+
+ public static ITupleProjectorFactory createTupleProjectorFactory(DatasetFormatInfo datasetFormatInfo,
+ IProjectionFiltrationInfo<?> projectionInfo, IProjectionFiltrationInfo<?> metaProjectionInfo,
+ ARecordType datasetType, ARecordType metaItemType, int numberOfPrimaryKeys) {
+ if (datasetFormatInfo.getFormat() == DatasetConfig.DatasetFormat.ROW) {
+ return DefaultTupleProjectorFactory.INSTANCE;
+ }
+ DataProjectionFiltrationInfo dataProjectionInfo = (DataProjectionFiltrationInfo) projectionInfo;
+ if (dataProjectionInfo == null) {
+ // projecting pushdown is disabled
+ ARecordType metaType = metaItemType == null ? null : DataProjectionFiltrationInfo.ALL_FIELDS_TYPE;
+ return new QueryColumnTupleProjectorFactory(datasetType, metaItemType, numberOfPrimaryKeys,
+ DataProjectionFiltrationInfo.ALL_FIELDS_TYPE, Collections.emptyMap(), metaType,
+ Collections.emptyMap(), NoOpColumnFilterEvaluatorFactory.INSTANCE);
+ }
+
+ DataProjectionFiltrationInfo metaDataProjectionInfo = (DataProjectionFiltrationInfo) metaProjectionInfo;
+
+ ARecordType datasetRequestedType = dataProjectionInfo.getProjectionInfo();
+ Map<String, FunctionCallInformation> datasetFunctionCallInfo = dataProjectionInfo.getFunctionCallInfoMap();
+
+ ARecordType metaRequestedType =
+ metaDataProjectionInfo == null ? null : metaDataProjectionInfo.getProjectionInfo();
+ Map<String, FunctionCallInformation> metaFunctionCallInfo =
+ metaProjectionInfo == null ? null : metaDataProjectionInfo.getFunctionCallInfoMap();
+
+ ColumnFilterBuilder columnFilterBuilder = new ColumnFilterBuilder(dataProjectionInfo);
+ IColumnFilterEvaluatorFactory filterEvaluator = columnFilterBuilder.build();
+
+ return new QueryColumnTupleProjectorFactory(datasetType, metaItemType, numberOfPrimaryKeys,
+ datasetRequestedType, datasetFunctionCallInfo, metaRequestedType, metaFunctionCallInfo,
+ filterEvaluator);
+ }
+
}
diff --git a/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslatorTest.java b/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslatorTest.java
index 73eeae4..292ff16 100644
--- a/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslatorTest.java
+++ b/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslatorTest.java
@@ -25,6 +25,7 @@
import org.apache.asterix.common.config.DatasetConfig.DatasetType;
import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.metadata.dataset.DatasetFormatInfo;
import org.apache.asterix.metadata.entities.Dataset;
import org.apache.asterix.metadata.entities.InternalDatasetDetails;
import org.apache.asterix.metadata.entities.InternalDatasetDetails.FileStructure;
@@ -55,7 +56,8 @@
Dataset dataset = new Dataset(DataverseName.createSinglePartName("test"), "log",
DataverseName.createSinglePartName("foo"), "LogType", DataverseName.createSinglePartName("CB"),
"MetaType", "DEFAULT_NG_ALL_NODES", "prefix", compactionPolicyProperties, details,
- Collections.emptyMap(), DatasetType.INTERNAL, 115, 0, CompressionManager.NONE);
+ Collections.emptyMap(), DatasetType.INTERNAL, 115, 0, CompressionManager.NONE,
+ DatasetFormatInfo.DEFAULT);
DatasetTupleTranslator dtTranslator = new DatasetTupleTranslator(true);
ITupleReference tuple = dtTranslator.getTupleFromMetadataEntity(dataset);
Dataset deserializedDataset = dtTranslator.getMetadataEntityFromTuple(tuple);
@@ -63,7 +65,7 @@
deserializedDataset.getMetaItemTypeDataverseName());
Assert.assertEquals(dataset.getMetaItemTypeName(), deserializedDataset.getMetaItemTypeName());
if (indicator == null) {
- Assert.assertEquals(Collections.singletonList(new Integer(0)),
+ Assert.assertEquals(Collections.singletonList(Integer.valueOf(0)),
((InternalDatasetDetails) deserializedDataset.getDatasetDetails()).getKeySourceIndicator());
} else {
Assert.assertEquals(((InternalDatasetDetails) dataset.getDatasetDetails()).getKeySourceIndicator(),
diff --git a/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslatorTest.java b/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslatorTest.java
index edb360f..77c64d7 100644
--- a/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslatorTest.java
+++ b/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslatorTest.java
@@ -32,6 +32,7 @@
import org.apache.asterix.common.config.DatasetConfig.IndexType;
import org.apache.asterix.common.metadata.DataverseName;
import org.apache.asterix.metadata.MetadataNode;
+import org.apache.asterix.metadata.dataset.DatasetFormatInfo;
import org.apache.asterix.metadata.entities.Dataset;
import org.apache.asterix.metadata.entities.Datatype;
import org.apache.asterix.metadata.entities.Index;
@@ -69,7 +70,7 @@
DataverseName dvCB = DataverseName.createSinglePartName("CB");
Dataset dataset = new Dataset(dvTest, "d1", dvFoo, "LogType", dvCB, "MetaType", "DEFAULT_NG_ALL_NODES",
"prefix", compactionPolicyProperties, details, Collections.emptyMap(), DatasetType.INTERNAL, 115, 0,
- CompressionManager.NONE);
+ CompressionManager.NONE, DatasetFormatInfo.DEFAULT);
Index index = new Index(dvTest, "d1", "i1", IndexType.BTREE,
Collections.singletonList(Collections.singletonList("row_id")),
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/object/base/AdmObjectNode.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/object/base/AdmObjectNode.java
index 294f48b..966b9ba 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/object/base/AdmObjectNode.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/object/base/AdmObjectNode.java
@@ -162,6 +162,22 @@
return ((AdmStringNode) node).get();
}
+ public int getOptionalInt(String field, int defaultValue) {
+ final IAdmNode node = get(field);
+ if (node == null) {
+ return defaultValue;
+ }
+ return (int) ((AdmBigIntNode) node).get();
+ }
+
+ public float getOptionalFloat(String field, float defaultValue) {
+ final IAdmNode node = get(field);
+ if (node == null) {
+ return defaultValue;
+ }
+ return (float) ((AdmDoubleNode) node).get();
+ }
+
public static String getString(AdmObjectNode openFields, String field) throws HyracksDataException {
IAdmNode node = openFields.get(field);
if (node == null) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/projection/DataProjectionInfo.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/projection/DataProjectionFiltrationInfo.java
similarity index 78%
rename from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/projection/DataProjectionInfo.java
rename to asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/projection/DataProjectionFiltrationInfo.java
index de402ec..e625163 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/projection/DataProjectionInfo.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/projection/DataProjectionFiltrationInfo.java
@@ -29,9 +29,10 @@
import org.apache.asterix.om.types.IAType;
import org.apache.asterix.om.types.visitor.SimpleStringBuilderForIATypeVisitor;
import org.apache.commons.lang3.SerializationUtils;
-import org.apache.hyracks.algebricks.core.algebra.metadata.IProjectionInfo;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IProjectionFiltrationInfo;
-public class DataProjectionInfo implements IProjectionInfo<ARecordType> {
+public class DataProjectionFiltrationInfo implements IProjectionFiltrationInfo<ARecordType> {
//Default open record type when requesting the entire fields
public static final ARecordType ALL_FIELDS_TYPE = createType("");
//Default open record type when requesting none of the fields
@@ -39,13 +40,18 @@
private final ARecordType root;
private final Map<String, FunctionCallInformation> functionCallInfoMap;
+ private final Map<ILogicalExpression, ARecordType> expressionToPath;
+ private final ILogicalExpression filterExpression;
- public DataProjectionInfo(ARecordType root, Map<String, FunctionCallInformation> sourceInformationMap) {
+ public DataProjectionFiltrationInfo(ARecordType root, Map<String, FunctionCallInformation> sourceInformationMap,
+ Map<ILogicalExpression, ARecordType> expressionToPath, ILogicalExpression filterExpression) {
this.root = root;
this.functionCallInfoMap = sourceInformationMap;
+ this.expressionToPath = expressionToPath;
+ this.filterExpression = filterExpression;
}
- private DataProjectionInfo(DataProjectionInfo other) {
+ private DataProjectionFiltrationInfo(DataProjectionFiltrationInfo other) {
if (other.root == ALL_FIELDS_TYPE) {
root = ALL_FIELDS_TYPE;
} else if (other.root == EMPTY_TYPE) {
@@ -54,6 +60,8 @@
root = other.root.deepCopy(other.root);
}
functionCallInfoMap = new HashMap<>(other.functionCallInfoMap);
+ expressionToPath = new HashMap<>(other.expressionToPath);
+ filterExpression = other.filterExpression;
}
@Override
@@ -62,14 +70,23 @@
}
@Override
- public DataProjectionInfo createCopy() {
- return new DataProjectionInfo(this);
+ public DataProjectionFiltrationInfo createCopy() {
+ return new DataProjectionFiltrationInfo(this);
+ }
+
+ @Override
+ public ILogicalExpression getFilterExpression() {
+ return filterExpression;
}
public Map<String, FunctionCallInformation> getFunctionCallInfoMap() {
return functionCallInfoMap;
}
+ public Map<ILogicalExpression, ARecordType> getExpressionToPath() {
+ return expressionToPath;
+ }
+
@Override
public boolean equals(Object o) {
if (this == o) {
@@ -78,8 +95,10 @@
if (o == null || getClass() != o.getClass()) {
return false;
}
- DataProjectionInfo otherInfo = (DataProjectionInfo) o;
- return root.deepEqual(otherInfo.root) && Objects.equals(functionCallInfoMap, otherInfo.functionCallInfoMap);
+ DataProjectionFiltrationInfo otherInfo = (DataProjectionFiltrationInfo) o;
+ return root.deepEqual(otherInfo.root) && Objects.equals(functionCallInfoMap, otherInfo.functionCallInfoMap)
+ && Objects.equals(filterExpression, otherInfo.filterExpression)
+ && Objects.equals(expressionToPath, otherInfo.expressionToPath);
}
@Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/projection/FunctionCallInformation.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/projection/FunctionCallInformation.java
index be96358..65c5cd5 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/projection/FunctionCallInformation.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/projection/FunctionCallInformation.java
@@ -80,7 +80,6 @@
output.writeUTF(functionName);
SourceLocation.writeFields(sourceLocation, output);
output.writeInt(typeMismatches.size());
- output.writeInt(warningFactory.getErrorCode().intValue());
for (ATypeTag typeTag : typeMismatches) {
output.write(typeTag.serialize());
}
@@ -90,13 +89,13 @@
String functionName = in.readUTF();
SourceLocation sourceLocation = SourceLocation.create(in);
int typeMismatchesLength = in.readInt();
- Set<ATypeTag> typeMismatches = EnumSet.noneOf(ATypeTag.class);
+ Set<ATypeTag> warnings = EnumSet.noneOf(ATypeTag.class);
IProjectionFiltrationWarningFactory warningFactory =
- ProjectionFiltrationWarningFactoryProvider.getWarningFactory(in.readInt());
+ ProjectionFiltrationWarningFactoryProvider.TYPE_MISMATCH_FACTORY;
for (int i = 0; i < typeMismatchesLength; i++) {
- typeMismatches.add(ATypeTag.VALUE_TYPE_MAPPING[in.readByte()]);
+ warnings.add(ATypeTag.VALUE_TYPE_MAPPING[in.readByte()]);
}
- return new FunctionCallInformation(functionName, sourceLocation, typeMismatches, warningFactory);
+ return new FunctionCallInformation(functionName, sourceLocation, warnings, warningFactory);
}
@Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/projection/ProjectionFiltrationWarningFactoryProvider.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/projection/ProjectionFiltrationWarningFactoryProvider.java
index 7d548ce..96de2cf 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/projection/ProjectionFiltrationWarningFactoryProvider.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/projection/ProjectionFiltrationWarningFactoryProvider.java
@@ -28,15 +28,6 @@
private ProjectionFiltrationWarningFactoryProvider() {
}
- public static IProjectionFiltrationWarningFactory getWarningFactory(int errorCode) {
- if (errorCode == ErrorCode.TYPE_MISMATCH_FUNCTION.intValue()) {
- return TYPE_MISMATCH_FACTORY;
- } else if (errorCode == ErrorCode.INCOMPARABLE_TYPES.intValue()) {
- return INCOMPARABLE_TYPES_FACTORY;
- }
- throw new UnsupportedOperationException("Unsupported error code " + errorCode);
- }
-
public static final IProjectionFiltrationWarningFactory TYPE_MISMATCH_FACTORY =
new IProjectionFiltrationWarningFactory() {
private static final long serialVersionUID = 4263556611813387010L;
@@ -54,7 +45,11 @@
}
};
- public static final IProjectionFiltrationWarningFactory INCOMPARABLE_TYPES_FACTORY =
+ public static IProjectionFiltrationWarningFactory getIncomparableTypesFactory(boolean leftConstant) {
+ return leftConstant ? LEFT_CONSTANT_INCOMPARABLE_TYPES_FACTORY : RIGHT_CONSTANT_INCOMPARABLE_TYPES_FACTORY;
+ }
+
+ private static final IProjectionFiltrationWarningFactory LEFT_CONSTANT_INCOMPARABLE_TYPES_FACTORY =
new IProjectionFiltrationWarningFactory() {
private static final long serialVersionUID = -7447187099851545763L;
@@ -69,4 +64,20 @@
return ErrorCode.INCOMPARABLE_TYPES;
}
};
+
+ private static final IProjectionFiltrationWarningFactory RIGHT_CONSTANT_INCOMPARABLE_TYPES_FACTORY =
+ new IProjectionFiltrationWarningFactory() {
+ private static final long serialVersionUID = 2818081955008928378L;
+
+ @Override
+ public Warning createWarning(SourceLocation sourceLocation, String functionName, String position,
+ ATypeTag expectedType, ATypeTag actualType) {
+ return Warning.of(sourceLocation, ErrorCode.INCOMPARABLE_TYPES, actualType, expectedType);
+ }
+
+ @Override
+ public ErrorCode getErrorCode() {
+ return ErrorCode.INCOMPARABLE_TYPES;
+ }
+ };
}
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 d350789..4596393 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
@@ -43,41 +43,42 @@
import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
public interface IMetadataProvider<S, I> {
- public IDataSource<S> findDataSource(S id) throws AlgebricksException;
+ IDataSource<S> findDataSource(S id) throws AlgebricksException;
/**
* Obs: A scanner may choose to contribute a null
* AlgebricksPartitionConstraint and implement
* contributeSchedulingConstraints instead.
*/
- public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getScannerRuntime(IDataSource<S> dataSource,
+ Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getScannerRuntime(IDataSource<S> 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;
+ IProjectionFiltrationInfo<?> projectionInfo, IProjectionFiltrationInfo<?> metaProjectionInfo)
+ throws AlgebricksException;
- public Pair<IPushRuntimeFactory, AlgebricksPartitionConstraint> getWriteFileRuntime(IDataSink sink,
- int[] printColumns, IPrinterFactory[] printerFactories, IAWriterFactory writerFactory,
- RecordDescriptor inputDesc) throws AlgebricksException;
+ Pair<IPushRuntimeFactory, AlgebricksPartitionConstraint> getWriteFileRuntime(IDataSink sink, int[] printColumns,
+ IPrinterFactory[] printerFactories, IAWriterFactory writerFactory, RecordDescriptor inputDesc)
+ throws AlgebricksException;
- public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getResultHandleRuntime(IDataSink sink,
- int[] printColumns, IPrinterFactory[] printerFactories, IAWriterFactory writerFactory,
+ Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getResultHandleRuntime(IDataSink sink, int[] printColumns,
+ IPrinterFactory[] printerFactories, IAWriterFactory writerFactory,
IResultSerializerFactoryProvider resultSerializerFactoryProvider, RecordDescriptor inputDesc,
IResultMetadata metadata, JobSpecification spec) throws AlgebricksException;
- public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getWriteResultRuntime(IDataSource<S> dataSource,
+ Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getWriteResultRuntime(IDataSource<S> dataSource,
IOperatorSchema propagatedSchema, List<LogicalVariable> keys, LogicalVariable payLoadVar,
List<LogicalVariable> additionalNonKeyFields, JobGenContext context, JobSpecification jobSpec)
throws AlgebricksException;
- public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getInsertRuntime(IDataSource<S> dataSource,
+ Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getInsertRuntime(IDataSource<S> dataSource,
IOperatorSchema propagatedSchema, IVariableTypeEnvironment typeEnv, List<LogicalVariable> keys,
LogicalVariable payLoadVar, List<LogicalVariable> additionalFilterKeyFields,
List<LogicalVariable> additionalNonFilteringFields, RecordDescriptor inputRecordDesc, JobGenContext context,
JobSpecification jobSpec, boolean bulkload) throws AlgebricksException;
- public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getDeleteRuntime(IDataSource<S> dataSource,
+ Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getDeleteRuntime(IDataSource<S> dataSource,
IOperatorSchema propagatedSchema, IVariableTypeEnvironment typeEnv, List<LogicalVariable> keys,
LogicalVariable payLoadVar, List<LogicalVariable> additionalNonKeyFields,
List<LogicalVariable> additionalNonFilteringFields, RecordDescriptor inputRecordDesc, JobGenContext context,
@@ -87,42 +88,28 @@
* Creates the insert runtime of IndexInsertDeletePOperator, which models
* insert/delete operations into a secondary index.
*
- * @param dataSource
- * Target secondary index.
- * @param propagatedSchema
- * Output schema of the insert/delete operator to be created.
- * @param inputSchemas
- * Output schemas of the insert/delete operator to be created.
- * @param typeEnv
- * Type environment of the original IndexInsertDeleteOperator operator.
- * @param primaryKeys
- * Variables for the dataset's primary keys that the dataSource secondary index belongs to.
- * @param secondaryKeys
- * Variables for the secondary-index keys.
- * @param additionalNonKeyFields
- * Additional variables that can be passed to the secondary index as payload.
- * This can be useful when creating a second filter on a non-primary and non-secondary
- * fields for additional pruning power.
- * @param filterExpr
- * Filtering expression to be pushed inside the runtime op.
- * Such a filter may, e.g., exclude NULLs from being inserted/deleted.
- * @param recordDesc
- * Output record descriptor of the runtime op to be created.
- * @param context
- * Job generation context.
- * @param spec
- * Target job specification.
- * @param secondaryKeysPipelines
- * Nested plans to extract secondary keys.
- * @param pipelineTopSchema
- * Schema of the primary pipeline for secondary keys.
- * @return
- * A Hyracks IOperatorDescriptor and its partition constraint.
+ * @param dataSource Target secondary index.
+ * @param propagatedSchema Output schema of the insert/delete operator to be created.
+ * @param inputSchemas Output schemas of the insert/delete operator to be created.
+ * @param typeEnv Type environment of the original IndexInsertDeleteOperator operator.
+ * @param primaryKeys Variables for the dataset's primary keys that the dataSource secondary index belongs to.
+ * @param secondaryKeys Variables for the secondary-index keys.
+ * @param additionalNonKeyFields Additional variables that can be passed to the secondary index as payload.
+ * This can be useful when creating a second filter on a non-primary and non-secondary
+ * fields for additional pruning power.
+ * @param filterExpr Filtering expression to be pushed inside the runtime op.
+ * Such a filter may, e.g., exclude NULLs from being inserted/deleted.
+ * @param recordDesc Output record descriptor of the runtime op to be created.
+ * @param context Job generation context.
+ * @param spec Target job specification.
+ * @param secondaryKeysPipelines Nested plans to extract secondary keys.
+ * @param pipelineTopSchema Schema of the primary pipeline for secondary keys.
+ * @return A Hyracks IOperatorDescriptor and its partition constraint.
* @throws AlgebricksException
*/
- public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getIndexInsertRuntime(
- IDataSourceIndex<I, S> dataSource, IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas,
- IVariableTypeEnvironment typeEnv, List<LogicalVariable> primaryKeys, List<LogicalVariable> secondaryKeys,
+ Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getIndexInsertRuntime(IDataSourceIndex<I, S> dataSource,
+ IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IVariableTypeEnvironment typeEnv,
+ List<LogicalVariable> primaryKeys, List<LogicalVariable> secondaryKeys,
List<LogicalVariable> additionalNonKeyFields, ILogicalExpression filterExpr, RecordDescriptor recordDesc,
JobGenContext context, JobSpecification spec, boolean bulkload,
List<List<AlgebricksPipeline>> secondaryKeysPipelines, IOperatorSchema pipelineTopSchema)
@@ -132,42 +119,28 @@
* Creates the delete runtime of IndexInsertDeletePOperator, which models
* insert/delete operations into a secondary index.
*
- * @param dataSource
- * Target secondary index.
- * @param propagatedSchema
- * Output schema of the insert/delete operator to be created.
- * @param inputSchemas
- * Output schemas of the insert/delete operator to be created.
- * @param typeEnv
- * Type environment of the original IndexInsertDeleteOperator operator.
- * @param primaryKeys
- * Variables for the dataset's primary keys that the dataSource secondary index belongs to.
- * @param secondaryKeys
- * Variables for the secondary-index keys.
- * @param additionalNonKeyFields
- * Additional variables that can be passed to the secondary index as payload.
- * This can be useful when creating a second filter on a non-primary and non-secondary
- * fields for additional pruning power.
- * @param filterExpr
- * Filtering expression to be pushed inside the runtime op.
- * Such a filter may, e.g., exclude NULLs from being inserted/deleted.
- * @param recordDesc
- * Output record descriptor of the runtime op to be created.
- * @param context
- * Job generation context.
- * @param spec
- * Target job specification.
- * @param secondaryKeysPipelines
- * Nested plan to extract secondary keys.
- * @param pipelineTopSchema
- * Schema of the primary pipeline for secondary keys.
- * @return
- * A Hyracks IOperatorDescriptor and its partition constraint.
+ * @param dataSource Target secondary index.
+ * @param propagatedSchema Output schema of the insert/delete operator to be created.
+ * @param inputSchemas Output schemas of the insert/delete operator to be created.
+ * @param typeEnv Type environment of the original IndexInsertDeleteOperator operator.
+ * @param primaryKeys Variables for the dataset's primary keys that the dataSource secondary index belongs to.
+ * @param secondaryKeys Variables for the secondary-index keys.
+ * @param additionalNonKeyFields Additional variables that can be passed to the secondary index as payload.
+ * This can be useful when creating a second filter on a non-primary and non-secondary
+ * fields for additional pruning power.
+ * @param filterExpr Filtering expression to be pushed inside the runtime op.
+ * Such a filter may, e.g., exclude NULLs from being inserted/deleted.
+ * @param recordDesc Output record descriptor of the runtime op to be created.
+ * @param context Job generation context.
+ * @param spec Target job specification.
+ * @param secondaryKeysPipelines Nested plan to extract secondary keys.
+ * @param pipelineTopSchema Schema of the primary pipeline for secondary keys.
+ * @return A Hyracks IOperatorDescriptor and its partition constraint.
* @throws AlgebricksException
*/
- public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getIndexDeleteRuntime(
- IDataSourceIndex<I, S> dataSource, IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas,
- IVariableTypeEnvironment typeEnv, List<LogicalVariable> primaryKeys, List<LogicalVariable> secondaryKeys,
+ Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getIndexDeleteRuntime(IDataSourceIndex<I, S> dataSource,
+ IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IVariableTypeEnvironment typeEnv,
+ List<LogicalVariable> primaryKeys, List<LogicalVariable> secondaryKeys,
List<LogicalVariable> additionalNonKeyFields, ILogicalExpression filterExpr, RecordDescriptor recordDesc,
JobGenContext context, JobSpecification spec, List<List<AlgebricksPipeline>> secondaryKeysPipelines,
IOperatorSchema pipelineTopSchema) throws AlgebricksException;
@@ -177,48 +150,37 @@
* secondary key into [token, number of token] pair in a length-partitioned index.
* In case of non length-partitioned index, it tokenizes secondary key into [token].
*
- * @param dataSource
- * Target secondary index.
- * @param propagatedSchema
- * Output schema of the insert/delete operator to be created.
- * @param inputSchemas
- * Output schemas of the insert/delete operator to be created.
- * @param typeEnv
- * Type environment of the original IndexInsertDeleteOperator operator.
- * @param primaryKeys
- * Variables for the dataset's primary keys that the dataSource secondary index belongs to.
- * @param secondaryKeys
- * Variables for the secondary-index keys.
- * @param filterExpr
- * Filtering expression to be pushed inside the runtime op.
- * Such a filter may, e.g., exclude NULLs from being inserted/deleted.
- * @param recordDesc
- * Output record descriptor of the runtime op to be created.
- * @param context
- * Job generation context.
- * @param spec
- * Target job specification.
- * @return
- * A Hyracks IOperatorDescriptor and its partition constraint.
+ * @param dataSource Target secondary index.
+ * @param propagatedSchema Output schema of the insert/delete operator to be created.
+ * @param inputSchemas Output schemas of the insert/delete operator to be created.
+ * @param typeEnv Type environment of the original IndexInsertDeleteOperator operator.
+ * @param primaryKeys Variables for the dataset's primary keys that the dataSource secondary index belongs to.
+ * @param secondaryKeys Variables for the secondary-index keys.
+ * @param filterExpr Filtering expression to be pushed inside the runtime op.
+ * Such a filter may, e.g., exclude NULLs from being inserted/deleted.
+ * @param recordDesc Output record descriptor of the runtime op to be created.
+ * @param context Job generation context.
+ * @param spec Target job specification.
+ * @return A Hyracks IOperatorDescriptor and its partition constraint.
* @throws AlgebricksException
*/
- public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getTokenizerRuntime(
- IDataSourceIndex<I, S> dataSource, IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas,
- IVariableTypeEnvironment typeEnv, List<LogicalVariable> primaryKeys, List<LogicalVariable> secondaryKeys,
- ILogicalExpression filterExpr, RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec,
- boolean bulkload) throws AlgebricksException;
+ Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getTokenizerRuntime(IDataSourceIndex<I, S> dataSource,
+ IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IVariableTypeEnvironment typeEnv,
+ List<LogicalVariable> primaryKeys, List<LogicalVariable> secondaryKeys, ILogicalExpression filterExpr,
+ RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec, boolean bulkload)
+ throws AlgebricksException;
- public IDataSourceIndex<I, S> findDataSourceIndex(I indexId, S dataSourceId) throws AlgebricksException;
+ IDataSourceIndex<I, S> findDataSourceIndex(I indexId, S dataSourceId) throws AlgebricksException;
- public IFunctionInfo lookupFunction(FunctionIdentifier fid);
+ IFunctionInfo lookupFunction(FunctionIdentifier fid);
- public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getUpsertRuntime(IDataSource<S> dataSource,
+ Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getUpsertRuntime(IDataSource<S> dataSource,
IOperatorSchema inputSchema, IVariableTypeEnvironment typeEnv, List<LogicalVariable> keys,
LogicalVariable payLoadVar, List<LogicalVariable> additionalFilterFields,
List<LogicalVariable> additionalNonFilteringFields, RecordDescriptor recordDesc, JobGenContext context,
JobSpecification jobSpec) throws AlgebricksException;
- public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getIndexUpsertRuntime(
+ Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getIndexUpsertRuntime(
IDataSourceIndex<I, S> dataSourceIndex, IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas,
IVariableTypeEnvironment typeEnv, List<LogicalVariable> primaryKeys, List<LogicalVariable> secondaryKeys,
List<LogicalVariable> additionalFilteringKeys, ILogicalExpression filterExpr,
@@ -226,12 +188,11 @@
LogicalVariable prevAdditionalFilteringKeys, RecordDescriptor inputDesc, JobGenContext context,
JobSpecification spec, List<List<AlgebricksPipeline>> secondaryKeysPipelines) throws AlgebricksException;
- public ITupleFilterFactory createTupleFilterFactory(IOperatorSchema[] inputSchemas,
- IVariableTypeEnvironment typeEnv, ILogicalExpression filterExpr, JobGenContext context)
- throws AlgebricksException;
+ ITupleFilterFactory createTupleFilterFactory(IOperatorSchema[] inputSchemas, IVariableTypeEnvironment typeEnv,
+ ILogicalExpression filterExpr, JobGenContext context) throws AlgebricksException;
- public Map<String, Object> getConfig();
+ Map<String, Object> getConfig();
- public boolean isBlockingOperatorDisabled();
+ boolean isBlockingOperatorDisabled();
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IProjectionInfo.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IProjectionFiltrationInfo.java
similarity index 80%
rename from hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IProjectionInfo.java
rename to hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IProjectionFiltrationInfo.java
index 3c1a24d..9973d08 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IProjectionInfo.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IProjectionFiltrationInfo.java
@@ -18,18 +18,22 @@
*/
package org.apache.hyracks.algebricks.core.algebra.metadata;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+
/**
* Generic interface to include the projection information for
* {@link org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator}
*/
-public interface IProjectionInfo<T> {
+public interface IProjectionFiltrationInfo<T> {
/**
* @return projected values' information
*/
T getProjectionInfo();
+ ILogicalExpression getFilterExpression();
+
/**
- * @return a copy of the {@link IProjectionInfo}
+ * @return a copy of the {@link IProjectionFiltrationInfo}
*/
- IProjectionInfo<T> createCopy();
+ IProjectionFiltrationInfo<T> createCopy();
}
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 9f73113..e3ce82d 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
@@ -29,7 +29,7 @@
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.metadata.IProjectionFiltrationInfo;
import org.apache.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
import org.apache.hyracks.algebricks.core.algebra.typing.ITypingContext;
import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
@@ -50,19 +50,22 @@
// the maximum of number of results output by this operator
private long outputLimit = -1;
- private IProjectionInfo<?> projectionInfo;
+ private IProjectionFiltrationInfo<?> datasetProjectionInfo;
+ private IProjectionFiltrationInfo<?> metaProjectionInfo;
public DataSourceScanOperator(List<LogicalVariable> variables, IDataSource<?> dataSource) {
- this(variables, dataSource, null, -1, null);
+ this(variables, dataSource, null, -1, null, null);
}
public DataSourceScanOperator(List<LogicalVariable> variables, IDataSource<?> dataSource,
- Mutable<ILogicalExpression> selectCondition, long outputLimit, IProjectionInfo<?> projectionInfo) {
+ Mutable<ILogicalExpression> selectCondition, long outputLimit,
+ IProjectionFiltrationInfo<?> datasetProjectionInfo, IProjectionFiltrationInfo<?> metaProjectionInfo) {
super(variables, dataSource);
projectVars = new ArrayList<>();
this.selectCondition = selectCondition;
this.outputLimit = outputLimit;
- this.projectionInfo = projectionInfo;
+ this.datasetProjectionInfo = datasetProjectionInfo;
+ this.metaProjectionInfo = metaProjectionInfo;
}
@Override
@@ -173,11 +176,19 @@
this.outputLimit = outputLimit;
}
- public void setProjectionInfo(IProjectionInfo<?> projectionInfo) {
- this.projectionInfo = projectionInfo;
+ public void setDatasetProjectionInfo(IProjectionFiltrationInfo<?> datasetProjectionInfo) {
+ this.datasetProjectionInfo = datasetProjectionInfo;
}
- public IProjectionInfo<?> getProjectionInfo() {
- return projectionInfo;
+ public IProjectionFiltrationInfo<?> getDatasetProjectionInfo() {
+ return datasetProjectionInfo;
+ }
+
+ public void setMetaProjectionInfo(IProjectionFiltrationInfo<?> metaProjectionInfo) {
+ this.metaProjectionInfo = metaProjectionInfo;
+ }
+
+ public IProjectionFiltrationInfo<?> getMetaProjectionInfo() {
+ return metaProjectionInfo;
}
}
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 f8d07b8..6d11931 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
@@ -26,6 +26,7 @@
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.IVariableTypeEnvironment;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IProjectionFiltrationInfo;
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;
@@ -39,14 +40,18 @@
// the maximum of number of results output by this operator
private long outputLimit = -1;
+ private IProjectionFiltrationInfo<?> datasetProjectionInfo;
+ private IProjectionFiltrationInfo<?> metaProjectionInfo;
+
public UnnestMapOperator(List<LogicalVariable> variables, Mutable<ILogicalExpression> expression,
List<Object> variableTypes, boolean propagateInput) {
- this(variables, expression, variableTypes, propagateInput, null, -1);
+ this(variables, expression, variableTypes, propagateInput, null, -1, null, null);
}
public UnnestMapOperator(List<LogicalVariable> variables, Mutable<ILogicalExpression> expression,
List<Object> variableTypes, boolean propagateInput, Mutable<ILogicalExpression> selectCondition,
- long outputLimit) {
+ long outputLimit, IProjectionFiltrationInfo<?> datasetProjectionInfo,
+ IProjectionFiltrationInfo<?> metaProjectionInfo) {
super(variables, expression, variableTypes, propagateInput);
this.selectCondition = selectCondition;
this.outputLimit = outputLimit;
@@ -101,4 +106,20 @@
this.outputLimit = outputLimit;
}
+ public void setDatasetProjectionInfo(IProjectionFiltrationInfo<?> projectionInfo) {
+ this.datasetProjectionInfo = projectionInfo;
+ }
+
+ public IProjectionFiltrationInfo<?> getDatasetProjectionInfo() {
+ return datasetProjectionInfo;
+ }
+
+ public void setMetaProjectionInfo(IProjectionFiltrationInfo<?> metaProjectionInfo) {
+ this.metaProjectionInfo = metaProjectionInfo;
+ }
+
+ public IProjectionFiltrationInfo<?> getMetaProjectionInfo() {
+ return metaProjectionInfo;
+ }
+
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
index cfae695..5a6574a 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
@@ -447,7 +447,9 @@
if (!isomorphic) {
return Boolean.FALSE;
}
- isomorphic = op.getExpressionRef().getValue().equals(unnestOpArg.getExpressionRef().getValue());
+ isomorphic = op.getExpressionRef().getValue().equals(unnestOpArg.getExpressionRef().getValue())
+ && Objects.equals(op.getDatasetProjectionInfo(), unnestOpArg.getDatasetProjectionInfo())
+ && Objects.equals(op.getMetaProjectionInfo(), unnestOpArg.getMetaProjectionInfo());
return isomorphic;
}
@@ -480,7 +482,8 @@
DataSourceScanOperator argScan = (DataSourceScanOperator) arg;
boolean isomorphic = op.getDataSource().getId().equals(argScan.getDataSource().getId())
&& op.getOutputLimit() == argScan.getOutputLimit()
- && Objects.equals(op.getProjectionInfo(), argScan.getProjectionInfo());
+ && Objects.equals(op.getDatasetProjectionInfo(), argScan.getDatasetProjectionInfo())
+ && Objects.equals(op.getMetaProjectionInfo(), argScan.getMetaProjectionInfo());
if (!isomorphic) {
return Boolean.FALSE;
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java
index 08acf13..71a659b 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java
@@ -37,7 +37,7 @@
import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
import org.apache.hyracks.algebricks.core.algebra.base.IVariableContext;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
-import org.apache.hyracks.algebricks.core.algebra.metadata.IProjectionInfo;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IProjectionFiltrationInfo;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
@@ -104,22 +104,17 @@
private final boolean reuseFreeVars;
/**
- * @param varContext
- * , the variable context.
- * @param typeContext
- * the type context.
+ * @param varContext , the variable context.
+ * @param typeContext the type context.
*/
public LogicalOperatorDeepCopyWithNewVariablesVisitor(IVariableContext varContext, ITypingContext typeContext) {
this(varContext, typeContext, new LinkedHashMap<>(), false);
}
/**
- * @param varContext
- * , the variable context.
- * @param typeContext
- * the type context.
- * @param reuseFreeVars
- * whether free variables in the given plan tree should be reused.
+ * @param varContext , the variable context.
+ * @param typeContext the type context.
+ * @param reuseFreeVars whether free variables in the given plan tree should be reused.
*/
public LogicalOperatorDeepCopyWithNewVariablesVisitor(IVariableContext varContext, ITypingContext typeContext,
boolean reuseFreeVars) {
@@ -127,16 +122,12 @@
}
/**
- * @param varContext
- * , the variable context.
- * @param typeContext
- * the type context.
- * @param inVarMapping
- * Variable mapping keyed by variables in the original plan.
- * Those variables are replaced by their corresponding value in
- * the map in the copied plan.
- * @param reuseFreeVars
- * whether free variables in the given plan tree should be reused.
+ * @param varContext , the variable context.
+ * @param typeContext the type context.
+ * @param inVarMapping Variable mapping keyed by variables in the original plan.
+ * Those variables are replaced by their corresponding value in
+ * the map in the copied plan.
+ * @param reuseFreeVars whether free variables in the given plan tree should be reused.
*/
public LogicalOperatorDeepCopyWithNewVariablesVisitor(IVariableContext varContext, ITypingContext typeContext,
LinkedHashMap<LogicalVariable, LogicalVariable> inVarMapping, boolean reuseFreeVars) {
@@ -326,9 +317,12 @@
throws AlgebricksException {
Mutable<ILogicalExpression> newSelectCondition = op.getSelectCondition() != null
? exprDeepCopyVisitor.deepCopyExpressionReference(op.getSelectCondition()) : null;
- IProjectionInfo<?> projectionInfo = op.getProjectionInfo() != null ? op.getProjectionInfo().createCopy() : null;
+ IProjectionFiltrationInfo<?> datasetProjectionInfo =
+ op.getDatasetProjectionInfo() != null ? op.getDatasetProjectionInfo().createCopy() : null;
+ IProjectionFiltrationInfo<?> metaProjectionInfo =
+ op.getMetaProjectionInfo() != null ? op.getMetaProjectionInfo().createCopy() : null;
DataSourceScanOperator opCopy = new DataSourceScanOperator(deepCopyVariableList(op.getVariables()),
- op.getDataSource(), newSelectCondition, op.getOutputLimit(), projectionInfo);
+ op.getDataSource(), newSelectCondition, op.getOutputLimit(), datasetProjectionInfo, metaProjectionInfo);
deepCopyInputsAnnotationsAndExecutionMode(op, arg, opCopy);
return opCopy;
}
@@ -540,9 +534,14 @@
throws AlgebricksException {
Mutable<ILogicalExpression> newSelectCondition = op.getSelectCondition() != null
? exprDeepCopyVisitor.deepCopyExpressionReference(op.getSelectCondition()) : null;
+ IProjectionFiltrationInfo<?> datasetProjectionInfo =
+ op.getDatasetProjectionInfo() != null ? op.getDatasetProjectionInfo().createCopy() : null;
+ IProjectionFiltrationInfo<?> metaProjectionInfo =
+ op.getMetaProjectionInfo() != null ? op.getMetaProjectionInfo().createCopy() : null;
UnnestMapOperator opCopy = new UnnestMapOperator(deepCopyVariableList(op.getVariables()),
exprDeepCopyVisitor.deepCopyExpressionReference(op.getExpressionRef()), op.getVariableTypes(),
- op.propagatesInput(), newSelectCondition, op.getOutputLimit());
+ op.propagatesInput(), newSelectCondition, op.getOutputLimit(), datasetProjectionInfo,
+ metaProjectionInfo);
deepCopyInputsAnnotationsAndExecutionMode(op, arg, opCopy);
return opCopy;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java
index 8ef0b5b..b7029d1 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java
@@ -33,7 +33,7 @@
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalPlan;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractLogicalExpression;
-import org.apache.hyracks.algebricks.core.algebra.metadata.IProjectionInfo;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IProjectionFiltrationInfo;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
@@ -245,8 +245,13 @@
newInputList.addAll(op.getVariables());
Mutable<ILogicalExpression> newSelectCondition =
op.getSelectCondition() != null ? deepCopyExpressionRef(op.getSelectCondition()) : null;
+ IProjectionFiltrationInfo<?> datasetProjectionInfo =
+ op.getDatasetProjectionInfo() != null ? op.getDatasetProjectionInfo().createCopy() : null;
+ IProjectionFiltrationInfo<?> metaProjectionInfo =
+ op.getMetaProjectionInfo() != null ? op.getMetaProjectionInfo().createCopy() : null;
return new UnnestMapOperator(newInputList, deepCopyExpressionRef(op.getExpressionRef()),
- new ArrayList<>(op.getVariableTypes()), op.propagatesInput(), newSelectCondition, op.getOutputLimit());
+ new ArrayList<>(op.getVariableTypes()), op.propagatesInput(), newSelectCondition, op.getOutputLimit(),
+ datasetProjectionInfo, metaProjectionInfo);
}
@Override
@@ -264,10 +269,13 @@
newInputList.addAll(op.getVariables());
Mutable<ILogicalExpression> newSelectCondition =
op.getSelectCondition() != null ? deepCopyExpressionRef(op.getSelectCondition()) : null;
- IProjectionInfo<?> projectionInfo = op.getProjectionInfo() != null ? op.getProjectionInfo().createCopy() : null;
+ IProjectionFiltrationInfo<?> datasetProjectionInfo =
+ op.getDatasetProjectionInfo() != null ? op.getDatasetProjectionInfo().createCopy() : null;
+ IProjectionFiltrationInfo<?> metaProjectionInfo =
+ op.getMetaProjectionInfo() != null ? op.getMetaProjectionInfo().createCopy() : null;
return new DataSourceScanOperator(newInputList, op.getDataSource(), newSelectCondition, op.getOutputLimit(),
- projectionInfo);
+ datasetProjectionInfo, metaProjectionInfo);
}
@Override
@@ -379,7 +387,7 @@
private void deepCopyExpressionRefs(List<Mutable<ILogicalExpression>> newExprs,
List<Mutable<ILogicalExpression>> oldExprs) {
for (Mutable<ILogicalExpression> oldExpr : oldExprs) {
- newExprs.add(new MutableObject<>(((AbstractLogicalExpression) oldExpr.getValue()).cloneExpression()));
+ newExprs.add(new MutableObject<>(oldExpr.getValue().cloneExpression()));
}
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/DataSourceScanPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/DataSourceScanPOperator.java
index 48dc607..866334a 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/DataSourceScanPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/DataSourceScanPOperator.java
@@ -116,10 +116,10 @@
scan.getSelectCondition().getValue(), context);
}
- Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> p =
- mp.getScannerRuntime(dataSource, vars, projectVars, scan.isProjectPushed(), scan.getMinFilterVars(),
- scan.getMaxFilterVars(), tupleFilterFactory, scan.getOutputLimit(), opSchema, typeEnv, context,
- builder.getJobSpec(), implConfig, scan.getProjectionInfo());
+ Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> p = mp.getScannerRuntime(dataSource, vars, projectVars,
+ scan.isProjectPushed(), scan.getMinFilterVars(), scan.getMaxFilterVars(), tupleFilterFactory,
+ scan.getOutputLimit(), opSchema, typeEnv, context, builder.getJobSpec(), implConfig,
+ scan.getDatasetProjectionInfo(), scan.getMetaProjectionInfo());
IOperatorDescriptor opDesc = p.first;
opDesc.setSourceLocation(scan.getSourceLocation());
builder.contributeHyracksOperator(scan, opDesc);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
index 7aabbef..069012b 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
@@ -33,7 +33,7 @@
import org.apache.hyracks.algebricks.core.algebra.base.IPhysicalOperator;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
-import org.apache.hyracks.algebricks.core.algebra.metadata.IProjectionInfo;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IProjectionFiltrationInfo;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractUnnestMapOperator;
@@ -359,6 +359,9 @@
AlgebricksStringBuilderWriter plan = printAbstractUnnestMapOperator(op, indent, "unnest-map", null);
appendSelectConditionInformation(plan, op.getSelectCondition(), indent);
appendLimitInformation(plan, op.getOutputLimit());
+ appendProjectInformation(plan, "project", op.getDatasetProjectionInfo());
+ appendProjectInformation(plan, "project-meta", op.getMetaProjectionInfo());
+ appendFilterExpression(plan, op.getDatasetProjectionInfo());
return null;
}
@@ -386,7 +389,9 @@
appendFilterInformation(plan, op.getMinFilterVars(), op.getMaxFilterVars());
appendSelectConditionInformation(plan, op.getSelectCondition(), indent);
appendLimitInformation(plan, op.getOutputLimit());
- appendProjectInformation(plan, op.getProjectionInfo());
+ appendProjectInformation(plan, "project", op.getDatasetProjectionInfo());
+ appendProjectInformation(plan, "project-meta", op.getMetaProjectionInfo());
+ appendFilterExpression(plan, op.getDatasetProjectionInfo());
return null;
}
@@ -417,15 +422,30 @@
}
}
- private void appendProjectInformation(AlgebricksStringBuilderWriter plan, IProjectionInfo<?> projectionInfo) {
+ private void appendProjectInformation(AlgebricksStringBuilderWriter plan, String projectionSource,
+ IProjectionFiltrationInfo<?> projectionInfo) {
final String projectedFields = projectionInfo == null ? "" : projectionInfo.toString();
if (!projectedFields.isEmpty()) {
- plan.append(" project (");
+ plan.append(" ");
+ plan.append(projectionSource);
+ plan.append(" (");
plan.append(projectedFields);
plan.append(")");
}
}
+ private void appendFilterExpression(AlgebricksStringBuilderWriter plan,
+ IProjectionFiltrationInfo<?> projectionInfo) {
+ final String filterExpr = projectionInfo == null || projectionInfo.getFilterExpression() == null ? ""
+ : projectionInfo.getFilterExpression().toString();
+ if (!filterExpr.isEmpty()) {
+ plan.append(" filter on ");
+ plan.append("(");
+ plan.append(filterExpr);
+ plan.append(")");
+ }
+ }
+
@Override
public Void visitLimitOperator(LimitOperator op, Integer indent) throws AlgebricksException {
addIndent(indent).append("limit");
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java
index e54ef02..115448e 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java
@@ -38,7 +38,7 @@
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
import org.apache.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
-import org.apache.hyracks.algebricks.core.algebra.metadata.IProjectionInfo;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IProjectionFiltrationInfo;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
@@ -545,6 +545,9 @@
try {
writeUnnestMapOperator(op, indent, "unnest-map", null);
writeSelectLimitInformation(op.getSelectCondition(), op.getOutputLimit(), indent);
+ writeProjectInformation("project", op.getDatasetProjectionInfo());
+ writeProjectInformation("project-meta", op.getMetaProjectionInfo());
+ writeFilterInformation(op.getDatasetProjectionInfo());
return null;
} catch (IOException e) {
throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
@@ -574,7 +577,9 @@
}
writeFilterInformation(op.getMinFilterVars(), op.getMaxFilterVars());
writeSelectLimitInformation(op.getSelectCondition(), op.getOutputLimit(), indent);
- writeProjectInformation(op.getProjectionInfo());
+ writeProjectInformation("project", op.getDatasetProjectionInfo());
+ writeProjectInformation("project-meta", op.getMetaProjectionInfo());
+ writeFilterInformation(op.getDatasetProjectionInfo());
return null;
} catch (IOException e) {
throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
@@ -903,10 +908,19 @@
}
}
- private void writeProjectInformation(IProjectionInfo<?> projectionInfo) throws IOException {
+ private void writeProjectInformation(String projectionSource, IProjectionFiltrationInfo<?> projectionInfo)
+ throws IOException {
final String projectedFields = projectionInfo == null ? "" : projectionInfo.toString();
if (!projectedFields.isEmpty()) {
- jsonGenerator.writeStringField("project", projectedFields);
+ jsonGenerator.writeStringField(projectionSource, projectedFields);
+ }
+ }
+
+ private void writeFilterInformation(IProjectionFiltrationInfo<?> projectionInfo) throws IOException {
+ final String filterExpr = projectionInfo == null || projectionInfo.getFilterExpression() == null ? ""
+ : projectionInfo.getFilterExpression().toString();
+ if (!filterExpr.isEmpty()) {
+ jsonGenerator.writeStringField("filter-on", filterExpr);
}
}
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 c94d72a..0d02203 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
@@ -45,4 +45,5 @@
public static final int EXTERNAL_SCAN_BUFFER_SIZE =
StorageUtil.getIntSizeInBytes(8, StorageUtil.StorageUnit.KILOBYTE);
public static final boolean BATCH_LOOKUP_DEFAULT = false;
+ public static final boolean COLUMN_FILTER_DEFAULT = false;
}
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 86be6d0..3e51a80 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
@@ -52,6 +52,7 @@
private static final String CBO_TEST = "CBO_TEST";
private static final String FORCE_JOIN_ORDER = "FORCE_JOIN_ORDER";
private static final String QUERY_PLAN_SHAPE = "QUERY_PLAN_SHAPE";
+ private static final String COLUMN_FILTER = "COLUMN_FILTER";
private final Properties properties = new Properties();
@@ -294,6 +295,14 @@
setInt(EXTERNAL_SCAN_BUFFER_SIZE, bufferSize);
}
+ public void setColumnFilter(boolean columnFilter) {
+ setBoolean(COLUMN_FILTER, columnFilter);
+ }
+
+ public boolean isColumnFilterEnabled() {
+ return getBoolean(COLUMN_FILTER, AlgebricksConfig.COLUMN_FILTER_DEFAULT);
+ }
+
private void setInt(String property, int value) {
properties.setProperty(property, Integer.toString(value));
}
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 d555b31..4fc8057 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
@@ -243,12 +243,6 @@
cursor.next();
matchingTupleCount++;
ITupleReference tuple = cursor.getTuple();
- if (tupleFilter != null) {
- referenceFilterTuple.reset(tuple);
- if (!tupleFilter.accept(referenceFilterTuple)) {
- continue;
- }
- }
tb.reset();
if (retainInput) {
@@ -258,7 +252,17 @@
tb.addFieldEndOffset();
}
}
- writeTupleToOutput(tuple);
+
+ // tuple must be written first before the filter is applied to
+ // assemble columnar tuples
+ tuple = writeTupleToOutput(tuple);
+ if (tupleFilter != null) {
+ referenceFilterTuple.reset(tuple);
+ if (!tupleFilter.accept(referenceFilterTuple)) {
+ continue;
+ }
+ }
+
if (appendSearchCallbackProceedResult) {
writeSearchCallbackProceedResult(tb,
((ILSMIndexCursor) cursor).getSearchOperationCallbackProceedResult());
@@ -355,9 +359,9 @@
}
}
- protected void writeTupleToOutput(ITupleReference tuple) throws IOException {
+ protected ITupleReference writeTupleToOutput(ITupleReference tuple) throws IOException {
try {
- tupleProjector.project(tuple, dos, tb);
+ return tupleProjector.project(tuple, dos, tb);
} catch (Exception e) {
throw e;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/pom.xml b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/pom.xml
index cadc714..8e0bc0c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/pom.xml
@@ -91,5 +91,9 @@
<groupId>org.apache.logging.log4j</groupId>
<artifactId>log4j-api</artifactId>
</dependency>
+ <dependency>
+ <groupId>com.fasterxml.jackson.core</groupId>
+ <artifactId>jackson-databind</artifactId>
+ </dependency>
</dependencies>
</project>
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/dataflow/LSMColumnBTreeLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/dataflow/LSMColumnBTreeLocalResource.java
new file mode 100644
index 0000000..56090bb
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/dataflow/LSMColumnBTreeLocalResource.java
@@ -0,0 +1,125 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.lsm.btree.column.dataflow;
+
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hyracks.api.application.INCServiceContext;
+import org.apache.hyracks.api.compression.ICompressorDecompressorFactory;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.api.io.IIOManager;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
+import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnManagerFactory;
+import org.apache.hyracks.storage.am.lsm.btree.column.utils.LSMColumnBTreeUtil;
+import org.apache.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeLocalResource;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMPageWriteCallbackFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
+import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCacheProvider;
+import org.apache.hyracks.storage.common.IStorageManager;
+import org.apache.hyracks.storage.common.compression.NoOpCompressorDecompressorFactory;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
+public class LSMColumnBTreeLocalResource extends LSMBTreeLocalResource {
+ private final IColumnManagerFactory columnManagerFactory;
+
+ public LSMColumnBTreeLocalResource(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories,
+ int[] bloomFilterKeyFields, double bloomFilterFalsePositiveRate, String path,
+ IStorageManager storageManager, ILSMMergePolicyFactory mergePolicyFactory,
+ Map<String, String> mergePolicyProperties, int[] btreeFields, ILSMOperationTrackerFactory opTrackerProvider,
+ ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
+ IMetadataPageManagerFactory metadataPageManagerFactory, IVirtualBufferCacheProvider vbcProvider,
+ ILSMIOOperationSchedulerProvider ioSchedulerProvider,
+ ICompressorDecompressorFactory compressorDecompressorFactory, ITypeTraits nullTypeTraits,
+ INullIntrospector nullIntrospector, boolean isSecondaryNoIncrementalMaintenance,
+ IColumnManagerFactory columnManagerFactory) {
+ super(typeTraits, cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate, true, path, storageManager,
+ mergePolicyFactory, mergePolicyProperties, null, null, btreeFields, null, opTrackerProvider,
+ ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory, vbcProvider,
+ ioSchedulerProvider, true, compressorDecompressorFactory, true, nullTypeTraits, nullIntrospector,
+ isSecondaryNoIncrementalMaintenance);
+ this.columnManagerFactory = columnManagerFactory;
+ }
+
+ private LSMColumnBTreeLocalResource(IPersistedResourceRegistry registry, JsonNode json, int[] bloomFilterKeyFields,
+ double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields,
+ ICompressorDecompressorFactory compressorDecompressorFactory, boolean hasBloomFilter,
+ boolean isSecondaryNoIncrementalMaintenance, IColumnManagerFactory columnManagerFactory)
+ throws HyracksDataException {
+ super(registry, json, bloomFilterKeyFields, bloomFilterFalsePositiveRate, isPrimary, btreeFields,
+ compressorDecompressorFactory, hasBloomFilter, isSecondaryNoIncrementalMaintenance);
+ this.columnManagerFactory = columnManagerFactory;
+ }
+
+ @Override
+ public ILSMIndex createInstance(INCServiceContext serviceCtx) throws HyracksDataException {
+ IIOManager ioManager = serviceCtx.getIoManager();
+ FileReference file = ioManager.resolve(path);
+ List<IVirtualBufferCache> vbcs = vbcProvider.getVirtualBufferCaches(serviceCtx, file);
+ ioOpCallbackFactory.initialize(serviceCtx, this);
+ pageWriteCallbackFactory.initialize(serviceCtx, this);
+ return LSMColumnBTreeUtil.createLSMTree(ioManager, vbcs, file, storageManager.getBufferCache(serviceCtx),
+ typeTraits, cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate,
+ mergePolicyFactory.createMergePolicy(mergePolicyProperties, serviceCtx),
+ opTrackerProvider.getOperationTracker(serviceCtx, this), ioSchedulerProvider.getIoScheduler(serviceCtx),
+ ioOpCallbackFactory, pageWriteCallbackFactory, btreeFields, metadataPageManagerFactory, false,
+ serviceCtx.getTracer(), compressorDecompressorFactory, nullTypeTraits, nullIntrospector,
+ columnManagerFactory);
+ }
+
+ public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json)
+ throws HyracksDataException {
+ int[] bloomFilterKeyFields = OBJECT_MAPPER.convertValue(json.get("bloomFilterKeyFields"), int[].class);
+ double bloomFilterFalsePositiveRate = json.get("bloomFilterFalsePositiveRate").asDouble();
+ boolean isPrimary = json.get("isPrimary").asBoolean();
+ boolean hasBloomFilter = getOrDefaultHasBloomFilter(json, isPrimary);
+ int[] btreeFields = OBJECT_MAPPER.convertValue(json.get("btreeFields"), int[].class);
+ JsonNode compressorDecompressorNode = json.get("compressorDecompressorFactory");
+ ICompressorDecompressorFactory compDecompFactory = (ICompressorDecompressorFactory) registry
+ .deserializeOrDefault(compressorDecompressorNode, NoOpCompressorDecompressorFactory.class);
+ JsonNode columnManagerFactoryNode = json.get("columnManagerFactory");
+ boolean isSecondaryNoIncrementalMaintenance =
+ getOrDefaultBoolean(json, "isSecondaryNoIncrementalMaintenance", false);
+ IColumnManagerFactory columnManagerFactory =
+ (IColumnManagerFactory) registry.deserialize(columnManagerFactoryNode);
+ return new LSMColumnBTreeLocalResource(registry, json, bloomFilterKeyFields, bloomFilterFalsePositiveRate,
+ isPrimary, btreeFields, compDecompFactory, hasBloomFilter, isSecondaryNoIncrementalMaintenance,
+ columnManagerFactory);
+ }
+
+ @Override
+ protected void appendToJson(final ObjectNode json, IPersistedResourceRegistry registry)
+ throws HyracksDataException {
+ super.appendToJson(json, registry);
+ json.putPOJO("columnManagerFactory", columnManagerFactory.toJson(registry));
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/dataflow/LSMColumnBTreeLocalResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/dataflow/LSMColumnBTreeLocalResourceFactory.java
new file mode 100644
index 0000000..eccb7c2
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/dataflow/LSMColumnBTreeLocalResourceFactory.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.lsm.btree.column.dataflow;
+
+import java.util.Map;
+
+import org.apache.hyracks.api.compression.ICompressorDecompressorFactory;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnManagerFactory;
+import org.apache.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeLocalResourceFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMPageWriteCallbackFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCacheProvider;
+import org.apache.hyracks.storage.am.lsm.common.dataflow.LsmResource;
+import org.apache.hyracks.storage.common.IStorageManager;
+
+public class LSMColumnBTreeLocalResourceFactory extends LSMBTreeLocalResourceFactory {
+ private static final long serialVersionUID = -676367767925618165L;
+ private final IColumnManagerFactory columnManagerFactory;
+
+ public LSMColumnBTreeLocalResourceFactory(IStorageManager storageManager, ITypeTraits[] typeTraits,
+ IBinaryComparatorFactory[] cmpFactories, ITypeTraits[] filterTypeTraits,
+ IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields,
+ ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
+ ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
+ IMetadataPageManagerFactory metadataPageManagerFactory, IVirtualBufferCacheProvider vbcProvider,
+ ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMMergePolicyFactory mergePolicyFactory,
+ Map<String, String> mergePolicyProperties, int[] bloomFilterKeyFields, double bloomFilterFalsePositiveRate,
+ int[] btreeFields, ICompressorDecompressorFactory compressorDecompressorFactory, ITypeTraits nullTypeTraits,
+ INullIntrospector nullIntrospector, boolean isSecondaryNoIncrementalMaintenance,
+ IColumnManagerFactory columnManagerFactory) {
+ super(storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
+ opTrackerFactory, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
+ vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, true, bloomFilterKeyFields,
+ bloomFilterFalsePositiveRate, true, btreeFields, compressorDecompressorFactory, true, nullTypeTraits,
+ nullIntrospector, isSecondaryNoIncrementalMaintenance);
+ this.columnManagerFactory = columnManagerFactory;
+ }
+
+ @Override
+ public LsmResource createResource(FileReference fileRef) {
+ return new LSMColumnBTreeLocalResource(typeTraits, cmpFactories, bloomFilterKeyFields,
+ bloomFilterFalsePositiveRate, fileRef.getRelativePath(), storageManager, mergePolicyFactory,
+ mergePolicyProperties, btreeFields, opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory,
+ metadataPageManagerFactory, vbcProvider, ioSchedulerProvider, compressorDecompressorFactory,
+ nullTypeTraits, nullIntrospector, isSecondaryNoIncrementalMaintenance, columnManagerFactory);
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/tuples/AbstractColumnTupleReference.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/tuples/AbstractColumnTupleReference.java
index d39f94e..d0e1e1d 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/tuples/AbstractColumnTupleReference.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/tuples/AbstractColumnTupleReference.java
@@ -40,8 +40,8 @@
private final IColumnBufferProvider[] primaryKeyBufferProviders;
private final IColumnBufferProvider[] buffersProviders;
private final int numberOfPrimaryKeys;
- private int totalNumberOfPages;
- private int numOfSkippedPages;
+ private int totalNumberOfMegaLeafNodes;
+ private int numOfSkippedMegaLeafNodes;
protected int tupleIndex;
/**
@@ -73,8 +73,8 @@
buffersProviders[i] = new ColumnSingleBufferProvider(columnIndex);
}
}
- totalNumberOfPages = 0;
- numOfSkippedPages = 0;
+ totalNumberOfMegaLeafNodes = 0;
+ numOfSkippedMegaLeafNodes = 0;
}
@Override
@@ -104,9 +104,9 @@
startColumn(provider, tupleIndex, i, numberOfTuples);
}
} else {
- numOfSkippedPages++;
+ numOfSkippedMegaLeafNodes++;
}
- totalNumberOfPages++;
+ totalNumberOfMegaLeafNodes++;
}
protected abstract boolean startNewPage(ByteBuffer pageZero, int numberOfColumns, int numberOfTuples);
@@ -149,8 +149,9 @@
@Override
public final void close() {
- if (LOGGER.isInfoEnabled()) {
- LOGGER.info("Skipped {} pages out of {} in total", numOfSkippedPages, totalNumberOfPages);
+ if (LOGGER.isInfoEnabled() && numOfSkippedMegaLeafNodes > 0) {
+ LOGGER.info("Filtered {} disk mega-leaf nodes out of {} in total", numOfSkippedMegaLeafNodes,
+ totalNumberOfMegaLeafNodes);
}
}
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 a7e433c..21c818c 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
@@ -158,12 +158,12 @@
json.put("isSecondaryNoIncrementalMaintenance", isSecondaryNoIncrementalMaintenance);
}
- private static boolean getOrDefaultHasBloomFilter(JsonNode json, boolean isPrimary) {
+ protected static boolean getOrDefaultHasBloomFilter(JsonNode json, boolean 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) {
+ protected static boolean getOrDefaultBoolean(JsonNode jsonNode, String fieldName, boolean defaultValue) {
return jsonNode.has(fieldName) ? jsonNode.get(fieldName).asBoolean() : defaultValue;
}