[NO ISSUE][COMP] Make LeftOuterJoin produce NULL in SQL-compat mode
- user model changes: no
- storage format changes: no
- interface changes: no
Details:
- SQL-compat mode: make Left Outer Join emit NULL instead of
MISSING for non-matching values
- SQL-compat mode: make Left Outer Unnest emit NULL instead of
MISSING if there are no items in the input list
- Fix comparison of AsterixConstantValue with
IAlgebricksConstantValue which is NULL/MISSING or BOOLEAN
- Add testcases
Change-Id: Ib39bd4e704015bbd38a2bf0b8d3a54bdbd7cd7e1
Reviewed-on: https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/14244
Integration-Tests: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: Ali Alsuliman <ali.al.solaiman@gmail.com>
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 ed8e9bf..06a6687 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,6 +21,8 @@
import java.util.ArrayList;
import java.util.List;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.metadata.declared.DataSourceId;
import org.apache.asterix.metadata.declared.DataSourceIndex;
import org.apache.asterix.metadata.declared.MetadataProvider;
@@ -37,7 +39,6 @@
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
-import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
import org.apache.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
@@ -46,6 +47,7 @@
import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSourceIndex;
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;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder.OrderKind;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
import org.apache.hyracks.algebricks.core.algebra.properties.BroadcastPartitioningProperty;
@@ -61,6 +63,7 @@
import org.apache.hyracks.algebricks.core.algebra.properties.UnorderedPartitionedProperty;
import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
+import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
/**
@@ -116,31 +119,45 @@
int[] lowKeyIndexes = getKeyIndexes(jobGenParams.getLowKeyVarList(), inputSchemas);
int[] highKeyIndexes = getKeyIndexes(jobGenParams.getHighKeyVarList(), inputSchemas);
+ boolean propagateFilter = unnestMap.propagateIndexFilter();
+ IMissingWriterFactory nonFilterWriterFactory = getNonFilterWriterFactory(propagateFilter, context);
int[] minFilterFieldIndexes = getKeyIndexes(unnestMap.getMinFilterVars(), inputSchemas);
int[] maxFilterFieldIndexes = getKeyIndexes(unnestMap.getMaxFilterVars(), inputSchemas);
- boolean propagateFilter = unnestMap.propagateIndexFilter();
MetadataProvider metadataProvider = (MetadataProvider) context.getMetadataProvider();
Dataset dataset = metadataProvider.findDataset(jobGenParams.getDataverseName(), jobGenParams.getDatasetName());
IVariableTypeEnvironment typeEnv = context.getTypeEnvironment(op);
ITupleFilterFactory tupleFilterFactory = null;
long outputLimit = -1;
- if (unnestMap.getOperatorTag() == LogicalOperatorTag.UNNEST_MAP) {
- UnnestMapOperator unnestMapOp = (UnnestMapOperator) unnestMap;
- outputLimit = unnestMapOp.getOutputLimit();
- if (unnestMapOp.getSelectCondition() != null) {
- tupleFilterFactory = metadataProvider.createTupleFilterFactory(new IOperatorSchema[] { opSchema },
- typeEnv, unnestMapOp.getSelectCondition().getValue(), context);
- }
+ boolean retainMissing = false;
+ IMissingWriterFactory nonMatchWriterFactory = null;
+ switch (unnestMap.getOperatorTag()) {
+ case UNNEST_MAP:
+ UnnestMapOperator unnestMapOp = (UnnestMapOperator) unnestMap;
+ outputLimit = unnestMapOp.getOutputLimit();
+ if (unnestMapOp.getSelectCondition() != null) {
+ tupleFilterFactory = metadataProvider.createTupleFilterFactory(new IOperatorSchema[] { opSchema },
+ typeEnv, unnestMapOp.getSelectCondition().getValue(), context);
+ }
+ break;
+ case LEFT_OUTER_UNNEST_MAP:
+ // By nature, LEFT_OUTER_UNNEST_MAP should generate missing (or null) values for non-matching tuples.
+ retainMissing = true;
+ nonMatchWriterFactory =
+ getNonMatchWriterFactory(((LeftOuterUnnestMapOperator) unnestMap).getMissingValue(), context,
+ unnestMap.getSourceLocation());
+ break;
+ default:
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, unnestMap.getSourceLocation(),
+ String.valueOf(unnestMap.getOperatorTag()));
}
- // By nature, LEFT_OUTER_UNNEST_MAP should generate null values for non-matching tuples.
- boolean retainMissing = op.getOperatorTag() == LogicalOperatorTag.LEFT_OUTER_UNNEST_MAP;
+
Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> btreeSearch = metadataProvider.buildBtreeRuntime(
- builder.getJobSpec(), opSchema, typeEnv, context, jobGenParams.getRetainInput(), retainMissing, dataset,
- jobGenParams.getIndexName(), lowKeyIndexes, highKeyIndexes, jobGenParams.isLowKeyInclusive(),
- jobGenParams.isHighKeyInclusive(), propagateFilter, minFilterFieldIndexes, maxFilterFieldIndexes,
- tupleFilterFactory, outputLimit, unnestMap.getGenerateCallBackProceedResultVar(),
- isPrimaryIndexPointSearch(op));
+ builder.getJobSpec(), opSchema, typeEnv, context, jobGenParams.getRetainInput(), retainMissing,
+ nonMatchWriterFactory, dataset, jobGenParams.getIndexName(), lowKeyIndexes, highKeyIndexes,
+ jobGenParams.isLowKeyInclusive(), jobGenParams.isHighKeyInclusive(), propagateFilter,
+ nonFilterWriterFactory, minFilterFieldIndexes, maxFilterFieldIndexes, tupleFilterFactory, outputLimit,
+ unnestMap.getGenerateCallBackProceedResultVar(), isPrimaryIndexPointSearch(op));
IOperatorDescriptor opDesc = btreeSearch.first;
opDesc.setSourceLocation(unnestMap.getSourceLocation());
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/IndexSearchPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/IndexSearchPOperator.java
index 73fcf92..4de30ce 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/IndexSearchPOperator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/IndexSearchPOperator.java
@@ -20,10 +20,13 @@
import java.util.List;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.metadata.declared.DataSourceId;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
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.IDataSourcePropertiesProvider;
@@ -36,6 +39,9 @@
import org.apache.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
import org.apache.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
import org.apache.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
+import org.apache.hyracks.api.exceptions.SourceLocation;
/**
* Class that embodies the commonalities between access method physical operators.
@@ -102,4 +108,21 @@
}
return ss;
}
+
+ protected static IMissingWriterFactory getNonMatchWriterFactory(IAlgebricksConstantValue missingValue,
+ JobGenContext context, SourceLocation sourceLoc) throws CompilationException {
+ IMissingWriterFactory nonMatchWriterFactory;
+ if (missingValue.isMissing()) {
+ nonMatchWriterFactory = context.getMissingWriterFactory();
+ } else if (missingValue.isNull()) {
+ nonMatchWriterFactory = context.getNullWriterFactory();
+ } else {
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc, missingValue.toString());
+ }
+ return nonMatchWriterFactory;
+ }
+
+ protected static IMissingWriterFactory getNonFilterWriterFactory(boolean propagateFilter, JobGenContext context) {
+ return propagateFilter ? context.getMissingWriterFactory() : null;
+ }
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java
index 532fb43..00eef69 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java
@@ -49,11 +49,13 @@
import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSourceIndex;
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;
import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
import org.apache.hyracks.algebricks.core.algebra.properties.LocalMemoryRequirements;
import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenHelper;
import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
+import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
import org.apache.hyracks.api.job.JobSpecification;
import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
@@ -117,13 +119,18 @@
Dataset dataset = metadataProvider.findDataset(jobGenParams.getDataverseName(), jobGenParams.getDatasetName());
int[] keyIndexes = getKeyIndexes(jobGenParams.getKeyVarList(), inputSchemas);
+ boolean propagateIndexFilter = unnestMapOp.propagateIndexFilter();
+ IMissingWriterFactory nonFilterWriterFactory = getNonFilterWriterFactory(propagateIndexFilter, context);
int[] minFilterFieldIndexes = getKeyIndexes(unnestMapOp.getMinFilterVars(), inputSchemas);
int[] maxFilterFieldIndexes = getKeyIndexes(unnestMapOp.getMaxFilterVars(), inputSchemas);
- boolean retainNull = false;
+ boolean retainMissing = false;
+ IMissingWriterFactory nonMatchWriterFactory = null;
if (op.getOperatorTag() == LogicalOperatorTag.LEFT_OUTER_UNNEST_MAP) {
- // By nature, LEFT_OUTER_UNNEST_MAP should generate null values for non-matching
- // tuples.
- retainNull = true;
+ // By nature, LEFT_OUTER_UNNEST_MAP should generate null values for non-matching tuples
+ retainMissing = true;
+ nonMatchWriterFactory =
+ getNonMatchWriterFactory(((LeftOuterUnnestMapOperator) unnestMapOp).getMissingValue(), context,
+ unnestMapOp.getSourceLocation());
}
// In-memory budget (frame limit) for inverted-index search operations
int frameLimit = localMemoryRequirements.getMemoryBudgetInFrames();
@@ -131,9 +138,10 @@
// Build runtime.
Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> invIndexSearch =
buildInvertedIndexRuntime(metadataProvider, context, builder.getJobSpec(), unnestMapOp, opSchema,
- jobGenParams.getRetainInput(), retainNull, jobGenParams.getDatasetName(), dataset,
- jobGenParams.getIndexName(), jobGenParams.getSearchKeyType(), keyIndexes,
- jobGenParams.getSearchModifierType(), jobGenParams.getSimilarityThreshold(),
+ jobGenParams.getRetainInput(), retainMissing, nonMatchWriterFactory,
+ jobGenParams.getDatasetName(), dataset, jobGenParams.getIndexName(),
+ jobGenParams.getSearchKeyType(), keyIndexes, jobGenParams.getSearchModifierType(),
+ jobGenParams.getSimilarityThreshold(), propagateIndexFilter, nonFilterWriterFactory,
minFilterFieldIndexes, maxFilterFieldIndexes, jobGenParams.getIsFullTextSearch(), frameLimit);
IOperatorDescriptor opDesc = invIndexSearch.first;
opDesc.setSourceLocation(unnestMapOp.getSourceLocation());
@@ -148,11 +156,11 @@
public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildInvertedIndexRuntime(
MetadataProvider metadataProvider, JobGenContext context, JobSpecification jobSpec,
AbstractUnnestMapOperator unnestMap, IOperatorSchema opSchema, boolean retainInput, boolean retainMissing,
- String datasetName, Dataset dataset, String indexName, ATypeTag searchKeyType, int[] keyFields,
- SearchModifierType searchModifierType, IAlgebricksConstantValue similarityThreshold,
- int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, boolean isFullTextSearchQuery, int frameLimit)
- throws AlgebricksException {
- boolean propagateIndexFilter = unnestMap.propagateIndexFilter();
+ IMissingWriterFactory nonMatchWriterFactory, String datasetName, Dataset dataset, String indexName,
+ ATypeTag searchKeyType, int[] keyFields, SearchModifierType searchModifierType,
+ IAlgebricksConstantValue similarityThreshold, boolean propagateIndexFilter,
+ IMissingWriterFactory nonFilterWriterFactory, int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes,
+ boolean isFullTextSearchQuery, int frameLimit) throws AlgebricksException {
IAObject simThresh = ((AsterixConstantValue) similarityThreshold).getObject();
int numPrimaryKeys = dataset.getPrimaryKeys().size();
Index secondaryIndex = MetadataManager.INSTANCE.getIndex(metadataProvider.getMetadataTxnContext(),
@@ -178,14 +186,14 @@
IIndexDataflowHelperFactory dataflowHelperFactory = new IndexDataflowHelperFactory(
metadataProvider.getStorageComponentProvider().getStorageManager(), secondarySplitsAndConstraint.first);
- LSMInvertedIndexSearchOperatorDescriptor invIndexSearchOp =
- new LSMInvertedIndexSearchOperatorDescriptor(jobSpec, outputRecDesc, queryField, dataflowHelperFactory,
- queryTokenizerFactory, fullTextConfigEvaluatorFactory, searchModifierFactory, retainInput,
- retainMissing, context.getMissingWriterFactory(),
- dataset.getSearchCallbackFactory(metadataProvider.getStorageComponentProvider(), secondaryIndex,
- IndexOperation.SEARCH, null),
- minFilterFieldIndexes, maxFilterFieldIndexes, isFullTextSearchQuery, numPrimaryKeys,
- propagateIndexFilter, frameLimit);
+ LSMInvertedIndexSearchOperatorDescriptor invIndexSearchOp = new LSMInvertedIndexSearchOperatorDescriptor(
+ jobSpec, outputRecDesc, queryField, dataflowHelperFactory, queryTokenizerFactory,
+ fullTextConfigEvaluatorFactory, searchModifierFactory, retainInput, retainMissing,
+ nonMatchWriterFactory,
+ dataset.getSearchCallbackFactory(metadataProvider.getStorageComponentProvider(), secondaryIndex,
+ IndexOperation.SEARCH, null),
+ minFilterFieldIndexes, maxFilterFieldIndexes, isFullTextSearchQuery, numPrimaryKeys,
+ propagateIndexFilter, nonFilterWriterFactory, frameLimit);
return new Pair<>(invIndexSearchOp, secondarySplitsAndConstraint.second);
}
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/RTreeSearchPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/RTreeSearchPOperator.java
index a80e5b3..6534ebe 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/RTreeSearchPOperator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/RTreeSearchPOperator.java
@@ -42,10 +42,12 @@
import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSourceIndex;
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;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
+import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
/**
* Contributes the runtime operator for an unnest-map representing a RTree
@@ -82,6 +84,7 @@
int[] keyIndexes = getKeyIndexes(jobGenParams.getKeyVarList(), inputSchemas);
boolean propagateIndexFilter = unnestMap.propagateIndexFilter();
+ IMissingWriterFactory nonFilterWriterFactory = getNonFilterWriterFactory(propagateIndexFilter, context);
int[] minFilterFieldIndexes = getKeyIndexes(unnestMap.getMinFilterVars(), inputSchemas);
int[] maxFilterFieldIndexes = getKeyIndexes(unnestMap.getMaxFilterVars(), inputSchemas);
@@ -93,15 +96,20 @@
outputVars = new ArrayList<LogicalVariable>();
VariableUtilities.getLiveVariables(unnestMap, outputVars);
}
- boolean retainNull = false;
+ boolean retainMissing = false;
+ IMissingWriterFactory nonMatchWriterFactory = null;
if (op.getOperatorTag() == LogicalOperatorTag.LEFT_OUTER_UNNEST_MAP) {
// By nature, LEFT_OUTER_UNNEST_MAP should generate null values for non-matching tuples.
- retainNull = true;
+ retainMissing = true;
+ nonMatchWriterFactory = getNonMatchWriterFactory(((LeftOuterUnnestMapOperator) unnestMap).getMissingValue(),
+ context, unnestMap.getSourceLocation());
}
- Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> rtreeSearch = mp.buildRtreeRuntime(
- builder.getJobSpec(), outputVars, opSchema, typeEnv, context, jobGenParams.getRetainInput(), retainNull,
- dataset, jobGenParams.getIndexName(), keyIndexes, propagateIndexFilter, minFilterFieldIndexes,
- maxFilterFieldIndexes, unnestMap.getGenerateCallBackProceedResultVar());
+
+ Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> rtreeSearch =
+ mp.buildRtreeRuntime(builder.getJobSpec(), outputVars, opSchema, typeEnv, context,
+ jobGenParams.getRetainInput(), retainMissing, nonMatchWriterFactory, dataset,
+ jobGenParams.getIndexName(), keyIndexes, propagateIndexFilter, nonFilterWriterFactory,
+ minFilterFieldIndexes, maxFilterFieldIndexes, unnestMap.getGenerateCallBackProceedResultVar());
IOperatorDescriptor opDesc = rtreeSearch.first;
opDesc.setSourceLocation(unnestMap.getSourceLocation());
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 1641665..5493dd1 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
@@ -151,7 +151,6 @@
import org.apache.hyracks.algebricks.rewriter.rules.subplan.EliminateSubplanWithInputCardinalityOneRule;
import org.apache.hyracks.algebricks.rewriter.rules.subplan.NestedSubplanToJoinRule;
import org.apache.hyracks.algebricks.rewriter.rules.subplan.PushSubplanIntoGroupByRule;
-import org.apache.hyracks.algebricks.rewriter.rules.subplan.SubplanOutOfGroupRule;
public final class RuleCollections {
@@ -241,7 +240,6 @@
condPushDownAndJoinInference.add(new PushSubplanWithAggregateDownThroughProductRule());
condPushDownAndJoinInference
.add(new AsterixPushMapOperatorThroughUnionRule(LogicalOperatorTag.ASSIGN, LogicalOperatorTag.SELECT));
- condPushDownAndJoinInference.add(new SubplanOutOfGroupRule());
condPushDownAndJoinInference.add(new AsterixExtractFunctionsFromJoinConditionRule());
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/AsterixIntroduceGroupByCombinerRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/AsterixIntroduceGroupByCombinerRule.java
index 83e62b8..df2e1ff 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/AsterixIntroduceGroupByCombinerRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/AsterixIntroduceGroupByCombinerRule.java
@@ -58,7 +58,7 @@
ScalarFunctionCallExpression nonSystemNullTest =
new ScalarFunctionCallExpression(finfoNot, new MutableObject<>(isSystemNullTest));
nonSystemNullTest.setSourceLocation(sourceLoc);
- selectNonSystemNull = new SelectOperator(new MutableObject<>(nonSystemNullTest), false, null);
+ selectNonSystemNull = new SelectOperator(new MutableObject<>(nonSystemNullTest));
selectNonSystemNull.setSourceLocation(sourceLoc);
} else {
List<Mutable<ILogicalExpression>> isSystemNullTestList = new ArrayList<>();
@@ -76,7 +76,7 @@
}
IFunctionInfo finfoAnd = context.getMetadataProvider().lookupFunction(AlgebricksBuiltinFunctions.AND);
selectNonSystemNull = new SelectOperator(
- new MutableObject<>(new ScalarFunctionCallExpression(finfoAnd, isSystemNullTestList)), false, null);
+ new MutableObject<>(new ScalarFunctionCallExpression(finfoAnd, isSystemNullTestList)));
selectNonSystemNull.setSourceLocation(sourceLoc);
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
index f8e84ad..03d112d 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
@@ -33,6 +33,7 @@
import org.apache.asterix.common.exceptions.WarningCollector;
import org.apache.asterix.dataflow.data.common.ExpressionTypeComputer;
import org.apache.asterix.dataflow.data.nontagged.MissingWriterFactory;
+import org.apache.asterix.dataflow.data.nontagged.NullWriterFactory;
import org.apache.asterix.formats.nontagged.ADMPrinterFactoryProvider;
import org.apache.asterix.formats.nontagged.BinaryBooleanInspector;
import org.apache.asterix.formats.nontagged.BinaryComparatorFactoryProvider;
@@ -125,7 +126,8 @@
}
@Override
- public Object getVarType(LogicalVariable var, List<LogicalVariable> nonNullVariables,
+ public Object getVarType(LogicalVariable var, List<LogicalVariable> nonMissableVariables,
+ List<List<LogicalVariable>> correlatedMissableVariableLists, List<LogicalVariable> nonNullableVariables,
List<List<LogicalVariable>> correlatedNullableVariableLists) {
throw new IllegalStateException();
}
@@ -149,7 +151,7 @@
BinaryHashFunctionFactoryProvider.INSTANCE, BinaryHashFunctionFamilyProvider.INSTANCE,
BinaryComparatorFactoryProvider.INSTANCE, TypeTraitProvider.INSTANCE, BinaryBooleanInspector.FACTORY,
BinaryIntegerInspector.FACTORY, ADMPrinterFactoryProvider.INSTANCE, MissingWriterFactory.INSTANCE,
- UnnestingPositionWriterFactory.INSTANCE, null,
+ NullWriterFactory.INSTANCE, UnnestingPositionWriterFactory.INSTANCE, null,
new ExpressionRuntimeProvider(new QueryLogicalExpressionJobGen(metadataProvider.getFunctionManager())),
ExpressionTypeComputer.INSTANCE, null, null, null, null, GlobalConfig.DEFAULT_FRAME_SIZE, null,
NoOpWarningCollector.INSTANCE, 0, new PhysicalOptimizationConfig());
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
index 01f93a4..0969fd8 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
@@ -1132,7 +1132,7 @@
}
public void applyFilteringExpression(Mutable<ILogicalExpression> filterExpression) throws AlgebricksException {
- SelectOperator selectOperator = new SelectOperator(filterExpression, false, null);
+ SelectOperator selectOperator = new SelectOperator(filterExpression);
selectOperator.setSourceLocation(sourceLoc);
this.currentTop = introduceNewOp(currentTop, selectOperator, true);
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PullSelectOutOfSpatialJoin.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PullSelectOutOfSpatialJoin.java
index e395549..507906b 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PullSelectOutOfSpatialJoin.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PullSelectOutOfSpatialJoin.java
@@ -106,7 +106,7 @@
}
// pull up
ILogicalExpression pulledCond = makeCondition(otherPredicates, context, op);
- SelectOperator select = new SelectOperator(new MutableObject<ILogicalExpression>(pulledCond), false, null);
+ SelectOperator select = new SelectOperator(new MutableObject<ILogicalExpression>(pulledCond));
select.setSourceLocation(op.getSourceLocation());
ILogicalExpression newJoinCond = makeCondition(spatialVarVarComps, context, op);
join.getCondition().setValue(newJoinCond);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveLeftOuterUnnestForLeftOuterJoinRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveLeftOuterUnnestForLeftOuterJoinRule.java
index 775a1df..f9f9a4c 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveLeftOuterUnnestForLeftOuterJoinRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveLeftOuterUnnestForLeftOuterJoinRule.java
@@ -22,6 +22,7 @@
import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
+import java.util.Objects;
import java.util.Set;
import org.apache.asterix.om.functions.BuiltinFunctions;
@@ -38,7 +39,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.IAlgebricksConstantValue;
import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
@@ -46,6 +49,7 @@
import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterUnnestOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
/**
@@ -55,7 +59,7 @@
* left-outer-unnest $x <- scan-collection($y)
* group-by($k){
* aggregate $y <- listify($z1)
- * select not(is-missing($z2))
+ * select not(is-missing/null($z2))
* NTS
* }
* left outer join ($a=$b)
@@ -89,7 +93,7 @@
// Checks whether the left outer unnest and the group-by operator are qualified for rewriting.
Triple<Boolean, ILogicalExpression, ILogicalExpression> checkGbyResult =
- checkUnnestAndGby(outerUnnest, gbyOperator);
+ checkUnnestAndGby(outerUnnest, gbyOperator, lojOperator);
// The argument for listify and not(is-missing(...)) check should be variables.
if (!checkGbyResult.first || checkGbyResult.second == null || !isVariableReference(checkGbyResult.second)
|| checkGbyResult.third == null || !isVariableReference(checkGbyResult.third)) {
@@ -127,7 +131,7 @@
// Checks left outer unnest and gby.
private Triple<Boolean, ILogicalExpression, ILogicalExpression> checkUnnestAndGby(
- LeftOuterUnnestOperator outerUnnest, GroupByOperator gbyOperator) {
+ LeftOuterUnnestOperator outerUnnest, GroupByOperator gbyOperator, LeftOuterJoinOperator lojOperator) {
// Checks left outer unnest.
Pair<Boolean, LogicalVariable> checkUnnestResult = checkUnnest(outerUnnest);
if (!checkUnnestResult.first) {
@@ -136,7 +140,8 @@
// Checks group-by.
LogicalVariable varToUnnest = checkUnnestResult.second;
- Triple<Boolean, ILogicalExpression, ILogicalExpression> checkGbyResult = checkGroupBy(gbyOperator, varToUnnest);
+ Triple<Boolean, ILogicalExpression, ILogicalExpression> checkGbyResult =
+ checkGroupBy(gbyOperator, varToUnnest, lojOperator.getMissingValue());
if (!checkGbyResult.first) {
return new Triple<>(false, null, null);
}
@@ -159,7 +164,7 @@
// Checks the group-by operator on top of the left outer join operator.
private Triple<Boolean, ILogicalExpression, ILogicalExpression> checkGroupBy(GroupByOperator gbyOperator,
- LogicalVariable varToUnnest) {
+ LogicalVariable varToUnnest, IAlgebricksConstantValue leftOuterMissingValue) {
Pair<Boolean, ILogicalOperator> checkNestedPlanResult = checkNestedPlan(gbyOperator);
if (!checkNestedPlanResult.first) {
return new Triple<>(false, null, null);
@@ -182,7 +187,7 @@
return new Triple<>(false, null, null);
}
SelectOperator select = (SelectOperator) rootInputOp;
- Pair<Boolean, ILogicalExpression> conditionArgPair = checkSelect(select);
+ Pair<Boolean, ILogicalExpression> conditionArgPair = checkSelect(select, leftOuterMissingValue);
return new Triple<>(true, listifyArgPair.second, conditionArgPair.second);
}
@@ -222,7 +227,8 @@
}
// Checks the expression for the nested select operator inside the group-by operator.
- private Pair<Boolean, ILogicalExpression> checkSelect(SelectOperator select) {
+ private Pair<Boolean, ILogicalExpression> checkSelect(SelectOperator select,
+ IAlgebricksConstantValue leftOuterMissingValue) {
ILogicalExpression condition = select.getCondition().getValue();
if (condition.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
return new Pair<>(false, null);
@@ -236,7 +242,9 @@
return new Pair<>(false, null);
}
conditionFunc = (AbstractFunctionCallExpression) condition;
- if (!conditionFunc.getFunctionIdentifier().equals(BuiltinFunctions.IS_MISSING)) {
+ FunctionIdentifier isMissingNullFuncId =
+ Objects.requireNonNull(OperatorPropertiesUtil.getIsMissingNullFunction(leftOuterMissingValue));
+ if (!conditionFunc.getFunctionIdentifier().equals(isMissingNullFuncId)) {
return new Pair<>(false, null);
}
ILogicalExpression conditionArg = conditionFunc.getArguments().get(0).getValue();
@@ -265,7 +273,7 @@
lhs.add(outerUnnest.getVariable());
VariableReferenceExpression listifyVarRef = new VariableReferenceExpression(listifyVar);
listifyVarRef.setSourceLocation(gbyOperator.getSourceLocation());
- rhs.add(new MutableObject<ILogicalExpression>(listifyVarRef));
+ rhs.add(new MutableObject<>(listifyVarRef));
List<Pair<LogicalVariable, Mutable<ILogicalExpression>>> gbyList = gbyOperator.getGroupByList();
for (Pair<LogicalVariable, Mutable<ILogicalExpression>> gbyPair : gbyList) {
lhs.add(gbyPair.first);
@@ -273,7 +281,9 @@
}
AssignOperator assignOp = new AssignOperator(lhs, rhs);
assignOp.setSourceLocation(outerUnnest.getSourceLocation());
- assignOp.getInputs().add(new MutableObject<ILogicalOperator>(lojOperator));
+ assignOp.getInputs().add(new MutableObject<>(lojOperator));
+ lojOperator.setMissingValue(outerUnnest.getMissingValue());
+ context.computeAndSetTypeEnvironmentForOperator(lojOperator);
context.computeAndSetTypeEnvironmentForOperator(assignOp);
opRef.setValue(assignOp);
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
index 819922b..7f1ff4a 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
@@ -397,7 +397,10 @@
}
@Override
- public Object getVarType(LogicalVariable var, List<LogicalVariable> nonNullVariables,
+ public Object getVarType(LogicalVariable var,
+ List<LogicalVariable> nonMissableVariables,
+ List<List<LogicalVariable>> correlatedMissableVariableLists,
+ List<LogicalVariable> nonNullableVariables,
List<List<LogicalVariable>> correlatedNullableVariableLists)
throws AlgebricksException {
if (var.equals(optFuncExpr.getSourceVar(exprAndVarIdx.second))) {
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodAnalysisContext.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodAnalysisContext.java
index b066339..3d4986d 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodAnalysisContext.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodAnalysisContext.java
@@ -57,7 +57,7 @@
// See AccessMethodUtils#removeUnjoinedDuplicatesInLOJ() for a definition of a special GroupBy
// and extra output processing steps needed when it's not available.
private Mutable<ILogicalOperator> lojSpecialGroupByOpRef = null;
- private ScalarFunctionCallExpression lojIsMissingFuncInSpecialGroupBy = null;
+ private ScalarFunctionCallExpression lojIsMissingNullFuncInSpecialGroupBy = null;
// For a secondary index, if we use only PK and secondary key field in a plan, it is an index-only plan.
// Contains information about index-only plan
@@ -146,12 +146,12 @@
return lojSpecialGroupByOpRef;
}
- public void setLOJIsMissingFuncInSpecialGroupBy(ScalarFunctionCallExpression isMissingFunc) {
- lojIsMissingFuncInSpecialGroupBy = isMissingFunc;
+ public void setLOJIsMissingNullFuncInSpecialGroupBy(ScalarFunctionCallExpression isMissingNullFunc) {
+ lojIsMissingNullFuncInSpecialGroupBy = isMissingNullFunc;
}
- public ScalarFunctionCallExpression getLOJIsMissingFuncInSpecialGroupBy() {
- return lojIsMissingFuncInSpecialGroupBy;
+ public ScalarFunctionCallExpression getLOJIsMissingNullFuncInSpecialGroupBy() {
+ return lojIsMissingNullFuncInSpecialGroupBy;
}
public Dataset getDatasetFromIndexDatasetMap(Index idx) {
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
index 063d55f..67e3784 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
@@ -83,6 +83,7 @@
import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractLogicalExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.UnnestingFunctionCallExpression;
@@ -827,26 +828,28 @@
Mutable<ILogicalOperator> joinRef, OptimizableOperatorSubTree indexSubTree,
OptimizableOperatorSubTree probeSubTree, AccessMethodAnalysisContext analysisCtx,
IOptimizationContext context, boolean isLeftOuterJoin, boolean isLeftOuterJoinWithSpecialGroupBy,
- ILogicalOperator indexSearchOp, LogicalVariable newNullPlaceHolderVar,
- Mutable<ILogicalExpression> conditionRef, Dataset dataset, Index chosenIndex) throws AlgebricksException {
+ IAlgebricksConstantValue leftOuterMissingValue, ILogicalOperator indexSearchOp,
+ LogicalVariable newMissingNullPlaceHolderVar, Mutable<ILogicalExpression> conditionRef, Dataset dataset,
+ Index chosenIndex) throws AlgebricksException {
boolean isIndexOnlyPlan = analysisCtx.getIndexOnlyPlanInfo().getFirst();
List<LogicalVariable> probePKVars = null;
ILogicalOperator finalIndexSearchOp = indexSearchOp;
if (isLeftOuterJoin) {
if (isLeftOuterJoinWithSpecialGroupBy) {
- ScalarFunctionCallExpression lojFuncExprs = analysisCtx.getLOJIsMissingFuncInSpecialGroupBy();
- List<LogicalVariable> lojMissingVariables = new ArrayList<>();
- lojFuncExprs.getUsedVariables(lojMissingVariables);
- boolean lojMissingVarExist = !lojMissingVariables.isEmpty();
+ ScalarFunctionCallExpression lojFuncExprs = analysisCtx.getLOJIsMissingNullFuncInSpecialGroupBy();
+ List<LogicalVariable> lojMissingNullVariables = new ArrayList<>();
+ lojFuncExprs.getUsedVariables(lojMissingNullVariables);
+ boolean lojMissingNullVarExist = !lojMissingNullVariables.isEmpty();
// Resets the missing place holder variable.
- AccessMethodUtils.resetLOJMissingPlaceholderVarInGroupByOp(analysisCtx, newNullPlaceHolderVar, context);
+ AccessMethodUtils.resetLOJMissingNullPlaceholderVarInGroupByOp(analysisCtx,
+ newMissingNullPlaceHolderVar, context);
- // For the index-only plan, if newNullPlaceHolderVar is not in the variable map of the union operator
+ // For the index-only plan, if newMissingNullPlaceHolderVar is not in the variable map of the union operator
// or if the variable is removed during the above method, we need to refresh the variable mapping in UNION.
if (isIndexOnlyPlan) {
finalIndexSearchOp = AccessMethodUtils.resetVariableMappingInUnionOpInIndexOnlyPlan(
- lojMissingVarExist, lojMissingVariables, indexSearchOp, afterJoinRefs, context);
+ lojMissingNullVarExist, lojMissingNullVariables, indexSearchOp, afterJoinRefs, context);
}
} else {
// We'll need to remove unjoined duplicates after the left outer join if there is no special GroupBy,
@@ -919,7 +922,9 @@
} else {
// Non-index only plan case
indexSubTree.getDataSourceRef().setValue(finalIndexSearchOp);
- SelectOperator topSelectOp = new SelectOperator(conditionRef, isLeftOuterJoin, newNullPlaceHolderVar);
+ SelectOperator topSelectOp = isLeftOuterJoin
+ ? new SelectOperator(conditionRef, leftOuterMissingValue, newMissingNullPlaceHolderVar)
+ : new SelectOperator(conditionRef);
topSelectOp.setSourceLocation(sourceLoc);
topSelectOp.getInputs().add(indexSubTree.getRootRef());
topSelectOp.setExecutionMode(ExecutionMode.LOCAL);
@@ -935,7 +940,8 @@
}
if (isLeftOuterJoin && !isLeftOuterJoinWithSpecialGroupBy) {
- finalOp = removeUnjoinedDuplicatesInLOJ(finalOp, probePKVars, newNullPlaceHolderVar, context, sourceLoc);
+ finalOp = removeUnjoinedDuplicatesInLOJ(finalOp, probePKVars, newMissingNullPlaceHolderVar,
+ leftOuterMissingValue, context, sourceLoc);
}
joinRef.setValue(finalOp);
@@ -947,12 +953,12 @@
* (see {@link IntroduceJoinAccessMethodRule#checkAndApplyJoinTransformation(Mutable, IOptimizationContext)}.
* A "Special GroupBy" is a GroupBy that eliminates unjoined duplicates that might be produced by the secondary
* index probe. We probe secondary indexes on each index partition and return a tuple with a right branch variable
- * set to MISSING if there's no match on that partition. Therefore if there's more than one partition where
- * nothing joined then the index probe plan will produce several such MISSING tuples, however the join result
- * must have a single MISSING tuple for each unjoined left branch tuple. If the special GroupBy is available then
- * it'll eliminate those MISSING duplicates, otherwise this method is called to produce additional operators to
- * achieve that. The special GroupBy operators are introduced by the optimizer when rewriting FROM-LET or
- * equivalent patterns into a left outer join with parent a group by.
+ * set to MISSING (or NULL) if there's no match on that partition. Therefore if there's more than one partition
+ * where nothing joined then the index probe plan will produce several such MISSING (or NULL) tuples, however the
+ * join result must have a single MISSING (or NULL) tuple for each unjoined left branch tuple. If the special
+ * GroupBy is available then it'll eliminate those MISSING (or NULL) duplicates, otherwise this method is
+ * called to produce additional operators to achieve that. The special GroupBy operators are introduced by
+ * the optimizer when rewriting FROM-LET or equivalent patterns into a left outer join with parent a group by.
* <p>
* The plan generated by this method to eliminate unjoined duplicates is as follows:
* <ul>
@@ -970,8 +976,9 @@
* the Select operator eliminates those unjoined duplicate tuples.
*/
private static SelectOperator removeUnjoinedDuplicatesInLOJ(ILogicalOperator inputOp,
- List<LogicalVariable> probePKVars, LogicalVariable newNullPlaceHolderVar, IOptimizationContext context,
- SourceLocation sourceLoc) throws AlgebricksException {
+ List<LogicalVariable> probePKVars, LogicalVariable newNullPlaceHolderVar,
+ IAlgebricksConstantValue lojMissingValue, IOptimizationContext context, SourceLocation sourceLoc)
+ throws AlgebricksException {
if (probePKVars == null || probePKVars.isEmpty()) {
throw new IllegalArgumentException();
}
@@ -984,16 +991,17 @@
VariableReferenceExpression winOrderByVarRef = new VariableReferenceExpression(newNullPlaceHolderVar);
winOrderByVarRef.setSourceLocation(sourceLoc);
- /* Sort in DESC order, so all MISSING values are at the end */
+ /* Sort in DESC order, so all MISSING (or NULL) values are at the end */
Pair<OrderOperator.IOrder, Mutable<ILogicalExpression>> winOrderByPair =
new Pair<>(OrderOperator.DESC_ORDER, new MutableObject<>(winOrderByVarRef));
LogicalVariable winVar = context.newVar();
VariableReferenceExpression winOrderByVarRef2 = new VariableReferenceExpression(newNullPlaceHolderVar);
winOrderByVarRef2.setSourceLocation(sourceLoc);
- AbstractFunctionCallExpression winExpr =
- BuiltinFunctions.makeWindowFunctionExpression(BuiltinFunctions.WIN_MARK_FIRST_MISSING_IMPL,
- Collections.singletonList(new MutableObject<>(winOrderByVarRef2)));
+ FunctionIdentifier winMarkFirstUnknownValueFn = lojMissingValue.isNull()
+ ? BuiltinFunctions.WIN_MARK_FIRST_NULL_IMPL : BuiltinFunctions.WIN_MARK_FIRST_MISSING_IMPL;
+ AbstractFunctionCallExpression winExpr = BuiltinFunctions.makeWindowFunctionExpression(
+ winMarkFirstUnknownValueFn, Collections.singletonList(new MutableObject<>(winOrderByVarRef2)));
WindowOperator winOp = new WindowOperator(winPartitionByList, Collections.singletonList(winOrderByPair));
winOp.getVariables().add(winVar);
@@ -1005,7 +1013,7 @@
VariableReferenceExpression winVarRef = new VariableReferenceExpression(winVar);
winVarRef.setSourceLocation(sourceLoc);
- SelectOperator selectOp = new SelectOperator(new MutableObject<>(winVarRef), false, null);
+ SelectOperator selectOp = new SelectOperator(new MutableObject<>(winVarRef));
selectOp.getInputs().add(new MutableObject<>(winOp));
selectOp.setExecutionMode(ExecutionMode.LOCAL);
selectOp.setSourceLocation(sourceLoc);
@@ -1017,7 +1025,8 @@
public static ILogicalOperator createSecondaryIndexUnnestMap(Dataset dataset, ARecordType recordType,
ARecordType metaRecordType, Index index, ILogicalOperator inputOp, AccessMethodJobGenParams jobGenParams,
IOptimizationContext context, boolean retainInput, boolean retainNull,
- boolean generateInstantTrylockResultFromIndexSearch) throws AlgebricksException {
+ boolean generateInstantTrylockResultFromIndexSearch, IAlgebricksConstantValue leftOuterMissingValue)
+ throws AlgebricksException {
SourceLocation sourceLoc = inputOp.getSourceLocation();
// The job gen parameters are transferred to the actual job gen via the UnnestMapOperator's function arguments.
ArrayList<Mutable<ILogicalExpression>> secondaryIndexFuncArgs = new ArrayList<>();
@@ -1047,7 +1056,7 @@
if (retainInput) {
LeftOuterUnnestMapOperator secondaryIndexLeftOuterUnnestOp = new LeftOuterUnnestMapOperator(
secondaryIndexUnnestVars, new MutableObject<ILogicalExpression>(secondaryIndexSearchFunc),
- secondaryIndexOutputTypes, true);
+ secondaryIndexOutputTypes, leftOuterMissingValue);
secondaryIndexLeftOuterUnnestOp.setSourceLocation(sourceLoc);
secondaryIndexLeftOuterUnnestOp
.setGenerateCallBackProceedResultVar(generateInstantTrylockResultFromIndexSearch);
@@ -1078,7 +1087,8 @@
ILogicalOperator inputOp, IOptimizationContext context, boolean sortPrimaryKeys, boolean retainInput,
boolean retainMissing, boolean requiresBroadcast, boolean requiresDistinct,
List<LogicalVariable> primaryKeyVars, List<LogicalVariable> primaryIndexUnnestVars,
- List<LogicalVariable> auxDistinctVars, List<Object> primaryIndexOutputTypes) throws AlgebricksException {
+ List<LogicalVariable> auxDistinctVars, List<Object> primaryIndexOutputTypes,
+ IAlgebricksConstantValue leftOuterMissingValue) throws AlgebricksException {
SourceLocation sourceLoc = inputOp.getSourceLocation();
// Sanity check: requiresDistinct and sortPrimaryKeys are mutually exclusive.
@@ -1130,11 +1140,11 @@
// Creates the primary-index search unnest-map operator.
AbstractUnnestMapOperator primaryIndexUnnestMapOp =
createPrimaryIndexUnnestMapOp(dataset, retainInput, retainMissing, requiresBroadcast, primaryKeyVars,
- primaryIndexUnnestVars, primaryIndexOutputTypes, sourceLoc);
+ primaryIndexUnnestVars, primaryIndexOutputTypes, sourceLoc, leftOuterMissingValue);
if (requiresDistinct) {
- primaryIndexUnnestMapOp.getInputs().add(new MutableObject<ILogicalOperator>(distinct));
+ primaryIndexUnnestMapOp.getInputs().add(new MutableObject<>(distinct));
} else if (sortPrimaryKeys) {
- primaryIndexUnnestMapOp.getInputs().add(new MutableObject<ILogicalOperator>(order));
+ primaryIndexUnnestMapOp.getInputs().add(new MutableObject<>(order));
} else {
primaryIndexUnnestMapOp.getInputs().add(new MutableObject<>(inputOp));
}
@@ -1149,9 +1159,9 @@
ARecordType metaRecordType, ILogicalOperator inputOp, IOptimizationContext context, boolean retainInput,
boolean retainMissing, boolean requiresBroadcast, Index secondaryIndex,
AccessMethodAnalysisContext analysisCtx, OptimizableOperatorSubTree subTree,
- LogicalVariable newMissingPlaceHolderForLOJ, List<LogicalVariable> pkVarsFromSIdxUnnestMapOp,
- List<LogicalVariable> primaryIndexUnnestVars, List<Object> primaryIndexOutputTypes)
- throws AlgebricksException {
+ LogicalVariable newMissingPlaceHolderForLOJ, IAlgebricksConstantValue leftOuterMissingValue,
+ List<LogicalVariable> pkVarsFromSIdxUnnestMapOp, List<LogicalVariable> primaryIndexUnnestVars,
+ List<Object> primaryIndexOutputTypes) throws AlgebricksException {
SourceLocation sourceLoc = inputOp.getSourceLocation();
Quadruple<Boolean, Boolean, Boolean, Boolean> indexOnlyPlanInfo = analysisCtx.getIndexOnlyPlanInfo();
// From now on, we deal with the index-only plan.
@@ -1527,7 +1537,7 @@
// The job gen parameters are transferred to the actual job gen via the UnnestMapOperator's function arguments.
AbstractUnnestMapOperator primaryIndexUnnestMapOp = createPrimaryIndexUnnestMapOp(dataset, retainInput,
retainMissing, requiresBroadcast, pkVarsInLeftPathFromSIdxSearchBeforeSplit,
- pkVarsFromPIdxSearchInLeftPath, primaryIndexOutputTypes, sourceLoc);
+ pkVarsFromPIdxSearchInLeftPath, primaryIndexOutputTypes, sourceLoc, leftOuterMissingValue);
primaryIndexUnnestMapOp.setSourceLocation(sourceLoc);
primaryIndexUnnestMapOp.getInputs().add(new MutableObject<ILogicalOperator>(origVarsToLeftPathVarsAssignOp));
context.computeAndSetTypeEnvironmentForOperator(primaryIndexUnnestMapOp);
@@ -1544,8 +1554,9 @@
ILogicalExpression conditionRefExpr = conditionRef.getValue().cloneExpression();
// The retainMissing variable contains the information whether we are optimizing a left-outer join or not.
LogicalVariable newMissingPlaceHolderVar = retainMissing ? newMissingPlaceHolderForLOJ : null;
- newSelectOpInLeftPath = new SelectOperator(new MutableObject<ILogicalExpression>(conditionRefExpr),
- retainMissing, newMissingPlaceHolderVar);
+ newSelectOpInLeftPath =
+ retainMissing ? new SelectOperator(new MutableObject<>(conditionRefExpr), leftOuterMissingValue,
+ newMissingPlaceHolderVar) : new SelectOperator(new MutableObject<>(conditionRefExpr));
newSelectOpInLeftPath.setSourceLocation(conditionRefExpr.getSourceLocation());
VariableUtilities.substituteVariables(newSelectOpInLeftPath, origVarToNewVarInLeftPathMap, context);
@@ -1604,8 +1615,11 @@
// since we need to change the variable reference in the SELECT operator.
// For the index-nested-loop join case, we copy the condition of the join operator.
ILogicalExpression conditionRefExpr2 = conditionRef.getValue().cloneExpression();
- newSelectOpInRightPath = new SelectOperator(new MutableObject<ILogicalExpression>(conditionRefExpr2),
- retainMissing, newMissingPlaceHolderVar);
+ newSelectOpInRightPath =
+ retainMissing
+ ? new SelectOperator(new MutableObject<>(conditionRefExpr2), leftOuterMissingValue,
+ newMissingPlaceHolderVar)
+ : new SelectOperator(new MutableObject<>(conditionRefExpr2));
newSelectOpInRightPath.setSourceLocation(conditionRefExpr2.getSourceLocation());
newSelectOpInRightPath.getInputs().add(new MutableObject<ILogicalOperator>(currentTopOpInRightPath));
VariableUtilities.substituteVariables(newSelectOpInRightPath, origVarToSIdxUnnestMapOpVarMap, context);
@@ -1655,7 +1669,7 @@
private static AbstractUnnestMapOperator createPrimaryIndexUnnestMapOp(Dataset dataset, boolean retainInput,
boolean retainMissing, boolean requiresBroadcast, List<LogicalVariable> primaryKeyVars,
List<LogicalVariable> primaryIndexUnnestVars, List<Object> primaryIndexOutputTypes,
- SourceLocation sourceLoc) throws AlgebricksException {
+ SourceLocation sourceLoc, IAlgebricksConstantValue leftOuterMissingValue) throws AlgebricksException {
// The job gen parameters are transferred to the actual job gen via the UnnestMapOperator's function arguments.
List<Mutable<ILogicalExpression>> primaryIndexFuncArgs = new ArrayList<>();
BTreeJobGenParams jobGenParams = new BTreeJobGenParams(dataset.getDatasetName(), IndexType.BTREE,
@@ -1675,12 +1689,11 @@
// This is the operator that jobgen will be looking for. It contains an unnest function that has
// all necessary arguments to determine which index to use, which variables contain the index-search keys,
// what is the original dataset, etc.
- AbstractUnnestMapOperator primaryIndexUnnestMapOp = null;
+ AbstractUnnestMapOperator primaryIndexUnnestMapOp;
if (retainMissing) {
if (retainInput) {
primaryIndexUnnestMapOp = new LeftOuterUnnestMapOperator(primaryIndexUnnestVars,
- new MutableObject<ILogicalExpression>(primaryIndexSearchFunc), primaryIndexOutputTypes,
- retainInput);
+ new MutableObject<>(primaryIndexSearchFunc), primaryIndexOutputTypes, leftOuterMissingValue);
primaryIndexUnnestMapOp.setSourceLocation(sourceLoc);
} else {
// Left-outer-join without retainNull and retainInput doesn't make sense.
@@ -1689,8 +1702,7 @@
}
} else {
primaryIndexUnnestMapOp = new UnnestMapOperator(primaryIndexUnnestVars,
- new MutableObject<ILogicalExpression>(primaryIndexSearchFunc), primaryIndexOutputTypes,
- retainInput);
+ new MutableObject<>(primaryIndexSearchFunc), primaryIndexOutputTypes, retainInput);
primaryIndexUnnestMapOp.setSourceLocation(sourceLoc);
}
return primaryIndexUnnestMapOp;
@@ -1721,7 +1733,8 @@
IOptimizationContext context, boolean sortPrimaryKeys, boolean retainInput, boolean retainMissing,
boolean requiresBroadcast, Index secondaryIndex, AccessMethodAnalysisContext analysisCtx,
OptimizableOperatorSubTree indexSubTree, OptimizableOperatorSubTree probeSubTree,
- LogicalVariable newMissingPlaceHolderForLOJ) throws AlgebricksException {
+ LogicalVariable newMissingPlaceHolderForLOJ, IAlgebricksConstantValue leftOuterMissingValue)
+ throws AlgebricksException {
// Common part for the non-index-only plan and index-only plan
// Variables and types for the primary-index search.
List<LogicalVariable> primaryIndexUnnestVars = new ArrayList<>();
@@ -1767,14 +1780,14 @@
return createFinalNonIndexOnlySearchPlan(dataset, inputOp, context, !isArrayIndex && sortPrimaryKeys,
retainInput, retainMissing, requiresBroadcast, isArrayIndex, pkVarsFromSIdxUnnestMapOp,
- primaryIndexUnnestVars, joinPKVars, primaryIndexOutputTypes);
+ primaryIndexUnnestVars, joinPKVars, primaryIndexOutputTypes, leftOuterMissingValue);
} else if (!isArrayIndex) {
// Index-only plan case: creates a UNIONALL operator that has two paths after the secondary unnest-map op,
// and returns it.
return createFinalIndexOnlySearchPlan(afterTopOpRefs, topOpRef, conditionRef, assignsBeforeTopOpRef,
dataset, recordType, metaRecordType, inputOp, context, retainInput, retainMissing,
requiresBroadcast, secondaryIndex, analysisCtx, indexSubTree, newMissingPlaceHolderForLOJ,
- pkVarsFromSIdxUnnestMapOp, primaryIndexUnnestVars, primaryIndexOutputTypes);
+ leftOuterMissingValue, pkVarsFromSIdxUnnestMapOp, primaryIndexUnnestVars, primaryIndexOutputTypes);
} else {
throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, inputOp.getSourceLocation(),
"Cannot use index-only plan with array indexes.");
@@ -1809,23 +1822,23 @@
return createRectangleExpr;
}
- private static ScalarFunctionCallExpression getNestedIsMissingCall(AbstractFunctionCallExpression call,
- OptimizableOperatorSubTree rightSubTree) throws AlgebricksException {
- ScalarFunctionCallExpression isMissingFuncExpr;
+ private static ScalarFunctionCallExpression getNestedIsMissingNullCall(AbstractFunctionCallExpression call,
+ OptimizableOperatorSubTree rightSubTree, FunctionIdentifier funId) throws AlgebricksException {
+ ScalarFunctionCallExpression isMissingNullFuncExpr;
if (call.getFunctionIdentifier().equals(AlgebricksBuiltinFunctions.NOT)) {
if (call.getArguments().get(0).getValue().getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
if (((AbstractFunctionCallExpression) call.getArguments().get(0).getValue()).getFunctionIdentifier()
- .equals(AlgebricksBuiltinFunctions.IS_MISSING)) {
- isMissingFuncExpr = (ScalarFunctionCallExpression) call.getArguments().get(0).getValue();
- if (isMissingFuncExpr.getArguments().get(0).getValue()
+ .equals(funId)) {
+ isMissingNullFuncExpr = (ScalarFunctionCallExpression) call.getArguments().get(0).getValue();
+ if (isMissingNullFuncExpr.getArguments().get(0).getValue()
.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
LogicalVariable var =
- ((VariableReferenceExpression) isMissingFuncExpr.getArguments().get(0).getValue())
+ ((VariableReferenceExpression) isMissingNullFuncExpr.getArguments().get(0).getValue())
.getVariableReference();
List<LogicalVariable> liveSubplanVars = new ArrayList<>();
VariableUtilities.getSubplanLocalLiveVariables(rightSubTree.getRoot(), liveSubplanVars);
if (liveSubplanVars.contains(var)) {
- return isMissingFuncExpr;
+ return isMissingNullFuncExpr;
}
}
}
@@ -1834,9 +1847,9 @@
return null;
}
- public static ScalarFunctionCallExpression findIsMissingInSubplan(AbstractLogicalOperator inputOp,
- OptimizableOperatorSubTree rightSubTree) throws AlgebricksException {
- ScalarFunctionCallExpression isMissingFuncExpr = null;
+ public static ScalarFunctionCallExpression findIsMissingNullInSubplan(AbstractLogicalOperator inputOp,
+ OptimizableOperatorSubTree rightSubTree, FunctionIdentifier funId) throws AlgebricksException {
+ ScalarFunctionCallExpression isMissingNullFuncExpr = null;
AbstractLogicalOperator currentOp = inputOp;
while (currentOp != null) {
if (currentOp.getOperatorTag() == LogicalOperatorTag.SELECT) {
@@ -1847,27 +1860,27 @@
if (call.getFunctionIdentifier().equals(AlgebricksBuiltinFunctions.AND)) {
for (Mutable<ILogicalExpression> mexpr : call.getArguments()) {
if (mexpr.getValue().getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
- isMissingFuncExpr = getNestedIsMissingCall(
- (AbstractFunctionCallExpression) mexpr.getValue(), rightSubTree);
- if (isMissingFuncExpr != null) {
- return isMissingFuncExpr;
+ isMissingNullFuncExpr = getNestedIsMissingNullCall(
+ (AbstractFunctionCallExpression) mexpr.getValue(), rightSubTree, funId);
+ if (isMissingNullFuncExpr != null) {
+ return isMissingNullFuncExpr;
}
}
}
}
- isMissingFuncExpr = getNestedIsMissingCall(call, rightSubTree);
- if (isMissingFuncExpr != null) {
- return isMissingFuncExpr;
+ isMissingNullFuncExpr = getNestedIsMissingNullCall(call, rightSubTree, funId);
+ if (isMissingNullFuncExpr != null) {
+ return isMissingNullFuncExpr;
}
}
} else if (currentOp.hasNestedPlans()) {
AbstractOperatorWithNestedPlans nestedPlanOp = (AbstractOperatorWithNestedPlans) currentOp;
for (ILogicalPlan nestedPlan : nestedPlanOp.getNestedPlans()) {
for (Mutable<ILogicalOperator> root : nestedPlan.getRoots()) {
- isMissingFuncExpr =
- findIsMissingInSubplan((AbstractLogicalOperator) root.getValue(), rightSubTree);
- if (isMissingFuncExpr != null) {
- return isMissingFuncExpr;
+ isMissingNullFuncExpr = findIsMissingNullInSubplan((AbstractLogicalOperator) root.getValue(),
+ rightSubTree, funId);
+ if (isMissingNullFuncExpr != null) {
+ return isMissingNullFuncExpr;
}
}
}
@@ -1875,27 +1888,29 @@
currentOp = currentOp.getInputs().isEmpty() ? null
: (AbstractLogicalOperator) currentOp.getInputs().get(0).getValue();
}
- return isMissingFuncExpr;
+ return isMissingNullFuncExpr;
}
- public static ScalarFunctionCallExpression findLOJIsMissingFuncInGroupBy(GroupByOperator lojGroupbyOp,
- OptimizableOperatorSubTree rightSubTree) throws AlgebricksException {
- //find IS_MISSING function of which argument has the nullPlaceholder variable in the nested plan of groupby.
+ public static ScalarFunctionCallExpression findLOJIsMissingNullFuncInGroupBy(GroupByOperator lojGroupbyOp,
+ OptimizableOperatorSubTree rightSubTree, FunctionIdentifier funId) throws AlgebricksException {
+ //find IS_MISSING or IS_NULL function of which argument has the nullPlaceholder variable in the nested plan of groupby.
ALogicalPlanImpl subPlan = (ALogicalPlanImpl) lojGroupbyOp.getNestedPlans().get(0);
Mutable<ILogicalOperator> subPlanRootOpRef = subPlan.getRoots().get(0);
AbstractLogicalOperator subPlanRootOp = (AbstractLogicalOperator) subPlanRootOpRef.getValue();
- return findIsMissingInSubplan(subPlanRootOp, rightSubTree);
+ return findIsMissingNullInSubplan(subPlanRootOp, rightSubTree, funId);
}
- public static void resetLOJMissingPlaceholderVarInGroupByOp(AccessMethodAnalysisContext analysisCtx,
- LogicalVariable newMissingPlaceholderVaraible, IOptimizationContext context) throws AlgebricksException {
+ public static void resetLOJMissingNullPlaceholderVarInGroupByOp(AccessMethodAnalysisContext analysisCtx,
+ LogicalVariable newMissingNullPlaceholderVaraible, IOptimizationContext context)
+ throws AlgebricksException {
//reset the missing placeholder variable in groupby operator
- ScalarFunctionCallExpression isMissingFuncExpr = analysisCtx.getLOJIsMissingFuncInSpecialGroupBy();
- isMissingFuncExpr.getArguments().clear();
- VariableReferenceExpression newMissingVarRef = new VariableReferenceExpression(newMissingPlaceholderVaraible);
- newMissingVarRef.setSourceLocation(isMissingFuncExpr.getSourceLocation());
- isMissingFuncExpr.getArguments().add(new MutableObject<ILogicalExpression>(newMissingVarRef));
+ ScalarFunctionCallExpression isMissingNullFuncExpr = analysisCtx.getLOJIsMissingNullFuncInSpecialGroupBy();
+ isMissingNullFuncExpr.getArguments().clear();
+ VariableReferenceExpression newMissingNullVarRef =
+ new VariableReferenceExpression(newMissingNullPlaceholderVaraible);
+ newMissingNullVarRef.setSourceLocation(isMissingNullFuncExpr.getSourceLocation());
+ isMissingNullFuncExpr.getArguments().add(new MutableObject<ILogicalExpression>(newMissingNullVarRef));
//recompute type environment.
OperatorPropertiesUtil.typeOpRec(analysisCtx.getLOJSpecialGroupByOpRef(), context);
@@ -2754,10 +2769,10 @@
*
* @throws AlgebricksException
*/
- public static ILogicalOperator resetVariableMappingInUnionOpInIndexOnlyPlan(boolean LOJVarExist,
- List<LogicalVariable> LOJMissingVariables, ILogicalOperator unionAllOp,
+ public static ILogicalOperator resetVariableMappingInUnionOpInIndexOnlyPlan(boolean lojVarExist,
+ List<LogicalVariable> lojMissingNullVariables, ILogicalOperator unionAllOp,
List<Mutable<ILogicalOperator>> aboveTopRefs, IOptimizationContext context) throws AlgebricksException {
- // For an index-only plan, if newNullPlaceHolderVar is not in the variable map of the UNIONALL operator,
+ // For an index-only plan, if newMissingNullPlaceHolderVar is not in the variable map of the UNIONALL operator,
// we need to add this variable to the map.
// Also, we need to delete replaced variables in the map if it was used only in the group-by operator.
if (unionAllOp.getOperatorTag() != LogicalOperatorTag.UNIONALL) {
@@ -2766,15 +2781,15 @@
// First, check whether the given old variable can be deleted. If it is used somewhere else
// except the group-by operator, we can't delete it since we need to propagate it.
- boolean LOJVarCanBeDeleted = true;
- if (LOJVarExist) {
+ boolean lojVarCanBeDeleted = true;
+ if (lojVarExist) {
List<LogicalVariable> usedVars = new ArrayList<>();
for (int i = 0; i < aboveTopRefs.size(); i++) {
usedVars.clear();
ILogicalOperator lOp = aboveTopRefs.get(i).getValue();
VariableUtilities.getUsedVariables(lOp, usedVars);
- if (usedVars.containsAll(LOJMissingVariables) && lOp.getOperatorTag() != LogicalOperatorTag.GROUP) {
- LOJVarCanBeDeleted = false;
+ if (usedVars.containsAll(lojMissingNullVariables) && lOp.getOperatorTag() != LogicalOperatorTag.GROUP) {
+ lojVarCanBeDeleted = false;
break;
}
}
@@ -2783,20 +2798,20 @@
List<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> varMap =
((UnionAllOperator) unionAllOp).getVariableMappings();
- if (LOJVarExist && LOJVarCanBeDeleted) {
+ if (lojVarExist && lojVarCanBeDeleted) {
// Delete old variables from the map.
for (Iterator<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> it = varMap.iterator(); it
.hasNext();) {
Triple<LogicalVariable, LogicalVariable, LogicalVariable> tripleVars = it.next();
- if (tripleVars.first.equals(LOJMissingVariables.get(0))
- || tripleVars.second.equals(LOJMissingVariables.get(0))
- || tripleVars.third.equals(LOJMissingVariables.get(0))) {
+ if (tripleVars.first.equals(lojMissingNullVariables.get(0))
+ || tripleVars.second.equals(lojMissingNullVariables.get(0))
+ || tripleVars.third.equals(lojMissingNullVariables.get(0))) {
it.remove();
}
}
}
- if (LOJVarExist && LOJVarCanBeDeleted) {
+ if (lojVarExist && lojVarCanBeDeleted) {
UnionAllOperator newUnionAllOp = new UnionAllOperator(varMap);
newUnionAllOp.getInputs()
.add(new MutableObject<ILogicalOperator>(unionAllOp.getInputs().get(0).getValue()));
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/ArrayBTreeAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/ArrayBTreeAccessMethod.java
index c633559..ce25d4d 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/ArrayBTreeAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/ArrayBTreeAccessMethod.java
@@ -43,6 +43,8 @@
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.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterUnnestOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
@@ -85,8 +87,8 @@
public boolean applyJoinPlanTransformation(List<Mutable<ILogicalOperator>> afterJoinRefs,
Mutable<ILogicalOperator> joinRef, OptimizableOperatorSubTree leftSubTree,
OptimizableOperatorSubTree rightSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
- IOptimizationContext context, boolean isLeftOuterJoin, boolean isLeftOuterJoinWithSpecialGroupBy)
- throws AlgebricksException {
+ IOptimizationContext context, boolean isLeftOuterJoin, boolean isLeftOuterJoinWithSpecialGroupBy,
+ IAlgebricksConstantValue leftOuterMissingValue) throws AlgebricksException {
AbstractBinaryJoinOperator joinOp = (AbstractBinaryJoinOperator) joinRef.getValue();
Mutable<ILogicalExpression> conditionRef = joinOp.getCondition();
Dataset dataset = analysisCtx.getIndexDatasetMap().get(chosenIndex);
@@ -159,7 +161,8 @@
if (workingOp.getOperatorTag() == LogicalOperatorTag.UNNEST) {
UnnestOperator oldUnnest = (UnnestOperator) workingOp;
LeftOuterUnnestOperator newUnnest = new LeftOuterUnnestOperator(oldUnnest.getVariable(),
- new MutableObject<>(oldUnnest.getExpressionRef().getValue()));
+ new MutableObject<>(oldUnnest.getExpressionRef().getValue()),
+ ConstantExpression.MISSING.getValue());
newUnnest.setSourceLocation(oldUnnest.getSourceLocation());
newUnnest.getInputs().addAll(oldUnnest.getInputs());
newUnnest.setExecutionMode(oldUnnest.getExecutionMode());
@@ -185,14 +188,14 @@
ILogicalOperator indexSearchOp = createIndexSearchPlan(afterJoinRefs, joinRef, conditionRef,
indexSubTree.getAssignsAndUnnestsRefs(), indexSubTree, probeSubTree, chosenIndex, analysisCtx, true,
- isLeftOuterJoin, true, context, newNullPlaceHolderVar);
+ isLeftOuterJoin, true, context, newNullPlaceHolderVar, leftOuterMissingValue);
if (indexSearchOp == null) {
return false;
}
return AccessMethodUtils.finalizeJoinPlanTransformation(afterJoinRefs, joinRef, indexSubTree, probeSubTree,
- analysisCtx, context, isLeftOuterJoin, isLeftOuterJoinWithSpecialGroupBy, indexSearchOp,
- newNullPlaceHolderVar, conditionRef, dataset, chosenIndex);
+ analysisCtx, context, isLeftOuterJoin, isLeftOuterJoinWithSpecialGroupBy, leftOuterMissingValue,
+ indexSearchOp, newNullPlaceHolderVar, conditionRef, dataset, chosenIndex);
}
@Override
@@ -201,7 +204,8 @@
List<Mutable<ILogicalOperator>> assignBeforeTheOpRefs, OptimizableOperatorSubTree indexSubTree,
OptimizableOperatorSubTree probeSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
boolean retainInput, boolean retainMissing, boolean requiresBroadcast, IOptimizationContext context,
- LogicalVariable newMissingPlaceHolderForLOJ) throws AlgebricksException {
+ LogicalVariable newMissingNullPlaceHolderForLOJ, IAlgebricksConstantValue leftOuterMissingValue)
+ throws AlgebricksException {
Index.ArrayIndexDetails chosenIndexDetails = (Index.ArrayIndexDetails) chosenIndex.getIndexDetails();
List<List<String>> chosenIndexKeyFieldNames = new ArrayList<>();
@@ -218,8 +222,8 @@
return createBTreeIndexSearchPlan(afterTopOpRefs, topOpRef, conditionRef, assignBeforeTheOpRefs, indexSubTree,
probeSubTree, chosenIndex, analysisCtx, retainInput, retainMissing, requiresBroadcast, context,
- newMissingPlaceHolderForLOJ, chosenIndexKeyFieldNames, chosenIndexKeyFieldTypes,
- chosenIndexKeyFieldSourceIndicators);
+ newMissingNullPlaceHolderForLOJ, leftOuterMissingValue, chosenIndexKeyFieldNames,
+ chosenIndexKeyFieldTypes, chosenIndexKeyFieldSourceIndicators);
}
@Override
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
index 1acbd89..60dd53b 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
@@ -64,6 +64,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.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.UnnestingFunctionCallExpression;
@@ -199,7 +200,7 @@
afterSelectRefs),
false, subTree.getDataSourceRef().getValue().getInputs().get(0).getValue()
.getExecutionMode() == ExecutionMode.UNPARTITIONED,
- context, null);
+ context, null, null);
if (primaryIndexUnnestOp == null) {
return false;
@@ -252,8 +253,8 @@
public boolean applyJoinPlanTransformation(List<Mutable<ILogicalOperator>> afterJoinRefs,
Mutable<ILogicalOperator> joinRef, OptimizableOperatorSubTree leftSubTree,
OptimizableOperatorSubTree rightSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
- IOptimizationContext context, boolean isLeftOuterJoin, boolean isLeftOuterJoinWithSpecialGroupBy)
- throws AlgebricksException {
+ IOptimizationContext context, boolean isLeftOuterJoin, boolean isLeftOuterJoinWithSpecialGroupBy,
+ IAlgebricksConstantValue leftOuterMissingValue) throws AlgebricksException {
AbstractBinaryJoinOperator joinOp = (AbstractBinaryJoinOperator) joinRef.getValue();
Mutable<ILogicalExpression> conditionRef = joinOp.getCondition();
@@ -278,12 +279,12 @@
return false;
}
- LogicalVariable newNullPlaceHolderVar = null;
+ LogicalVariable newMissingNullPlaceHolderVar = null;
if (isLeftOuterJoin) {
- // Gets a new null place holder variable that is the first field variable of the primary key
+ // Gets a new missing/null place holder variable that is the first field variable of the primary key
// from the indexSubTree's datasourceScanOp.
// We need this for all left outer joins, even those that do not have a special GroupBy
- newNullPlaceHolderVar = indexSubTree.getDataSourceVariables().get(0);
+ newMissingNullPlaceHolderVar = indexSubTree.getDataSourceVariables().get(0);
}
boolean canContinue = AccessMethodUtils.setIndexOnlyPlanInfo(afterJoinRefs, joinRef, probeSubTree, indexSubTree,
@@ -294,15 +295,15 @@
ILogicalOperator indexSearchOp = createIndexSearchPlan(afterJoinRefs, joinRef, conditionRef,
indexSubTree.getAssignsAndUnnestsRefs(), indexSubTree, probeSubTree, chosenIndex, analysisCtx, true,
- isLeftOuterJoin, true, context, newNullPlaceHolderVar);
+ isLeftOuterJoin, true, context, newMissingNullPlaceHolderVar, leftOuterMissingValue);
if (indexSearchOp == null) {
return false;
}
return AccessMethodUtils.finalizeJoinPlanTransformation(afterJoinRefs, joinRef, indexSubTree, probeSubTree,
- analysisCtx, context, isLeftOuterJoin, isLeftOuterJoinWithSpecialGroupBy, indexSearchOp,
- newNullPlaceHolderVar, conditionRef, dataset, chosenIndex);
+ analysisCtx, context, isLeftOuterJoin, isLeftOuterJoinWithSpecialGroupBy, leftOuterMissingValue,
+ indexSearchOp, newMissingNullPlaceHolderVar, conditionRef, dataset, chosenIndex);
}
/**
@@ -316,7 +317,8 @@
List<Mutable<ILogicalOperator>> assignBeforeTheOpRefs, OptimizableOperatorSubTree indexSubTree,
OptimizableOperatorSubTree probeSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
boolean retainInput, boolean retainMissing, boolean requiresBroadcast, IOptimizationContext context,
- LogicalVariable newMissingPlaceHolderForLOJ) throws AlgebricksException {
+ LogicalVariable newMissingNullPlaceHolderForLOJ, IAlgebricksConstantValue leftOuterMissingValue)
+ throws AlgebricksException {
Index.ValueIndexDetails chosenIndexDetails = (Index.ValueIndexDetails) chosenIndex.getIndexDetails();
List<List<String>> chosenIndexKeyFieldNames = chosenIndexDetails.getKeyFieldNames();
@@ -325,8 +327,8 @@
return createBTreeIndexSearchPlan(afterTopOpRefs, topOpRef, conditionRef, assignBeforeTheOpRefs, indexSubTree,
probeSubTree, chosenIndex, analysisCtx, retainInput, retainMissing, requiresBroadcast, context,
- newMissingPlaceHolderForLOJ, chosenIndexKeyFieldNames, chosenIndexKeyFieldTypes,
- chosenIndexKeyFieldSourceIndicators);
+ newMissingNullPlaceHolderForLOJ, leftOuterMissingValue, chosenIndexKeyFieldNames,
+ chosenIndexKeyFieldTypes, chosenIndexKeyFieldSourceIndicators);
}
protected ILogicalOperator createBTreeIndexSearchPlan(List<Mutable<ILogicalOperator>> afterTopOpRefs,
@@ -334,9 +336,9 @@
List<Mutable<ILogicalOperator>> assignBeforeTheOpRefs, OptimizableOperatorSubTree indexSubTree,
OptimizableOperatorSubTree probeSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
boolean retainInput, boolean retainMissing, boolean requiresBroadcast, IOptimizationContext context,
- LogicalVariable newMissingPlaceHolderForLOJ, List<List<String>> chosenIndexKeyFieldNames,
- List<IAType> chosenIndexKeyFieldTypes, List<Integer> chosenIndexKeyFieldSourceIndicators)
- throws AlgebricksException {
+ LogicalVariable newMissingNullPlaceHolderForLOJ, IAlgebricksConstantValue leftOuterMissingValue,
+ List<List<String>> chosenIndexKeyFieldNames, List<IAType> chosenIndexKeyFieldTypes,
+ List<Integer> chosenIndexKeyFieldSourceIndicators) throws AlgebricksException {
Dataset dataset = indexSubTree.getDataset();
ARecordType recordType = indexSubTree.getRecordType();
ARecordType metaRecordType = indexSubTree.getMetaRecordType();
@@ -669,7 +671,7 @@
// The result: SK, PK, [Optional - the result of an instantTrylock on PK]
ILogicalOperator secondaryIndexUnnestOp = AccessMethodUtils.createSecondaryIndexUnnestMap(dataset, recordType,
metaRecordType, chosenIndex, inputOp, jobGenParams, context, retainInput, retainMissing,
- generateInstantTrylockResultFromIndexSearch);
+ generateInstantTrylockResultFromIndexSearch, leftOuterMissingValue);
// Generate the rest of the upstream plan which feeds the search results into the primary index.
ILogicalOperator indexSearchOp = null;
@@ -686,7 +688,7 @@
indexSearchOp = AccessMethodUtils.createRestOfIndexSearchPlan(afterTopOpRefs, topOpRef, conditionRef,
assignBeforeTheOpRefs, dataSourceOp, dataset, recordType, metaRecordType, secondaryIndexUnnestOp,
context, true, retainInput, retainMissing, false, chosenIndex, analysisCtx, indexSubTree,
- probeSubTree, newMissingPlaceHolderForLOJ);
+ probeSubTree, newMissingNullPlaceHolderForLOJ, leftOuterMissingValue);
// Replaces the datasource scan with the new plan rooted at
// Get dataSourceRef operator -
@@ -763,7 +765,7 @@
} else {
unnestMapOp = new LeftOuterUnnestMapOperator(scanVariables,
new MutableObject<ILogicalExpression>(primaryIndexSearchFunc), primaryIndexOutputTypes,
- true);
+ leftOuterMissingValue);
}
unnestMapOp.setSourceLocation(dataSourceOp.getSourceLocation());
indexSearchOp = unnestMapOp;
@@ -776,7 +778,7 @@
} else {
unnestMapOp = new LeftOuterUnnestMapOperator(scanVariables,
((LeftOuterUnnestMapOperator) secondaryIndexUnnestOp).getExpressionRef(),
- primaryIndexOutputTypes, true);
+ primaryIndexOutputTypes, leftOuterMissingValue);
}
unnestMapOp.setSourceLocation(dataSourceOp.getSourceLocation());
indexSearchOp = unnestMapOp;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IAccessMethod.java
index e7becfa..5688a70 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IAccessMethod.java
@@ -32,6 +32,7 @@
import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
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.AbstractLogicalOperator;
@@ -107,7 +108,8 @@
List<Mutable<ILogicalOperator>> assignBeforeTheOpRefs, OptimizableOperatorSubTree indexSubTree,
OptimizableOperatorSubTree probeSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
boolean retainInput, boolean retainNull, boolean requiresBroadcast, IOptimizationContext context,
- LogicalVariable newNullPlaceHolderForLOJ) throws AlgebricksException;
+ LogicalVariable newMissingNullPlaceHolderForLOJ, IAlgebricksConstantValue leftOuterMissingValue)
+ throws AlgebricksException;
/**
* Applies the plan transformation to use chosenIndex to optimize a join query.
@@ -117,8 +119,8 @@
public boolean applyJoinPlanTransformation(List<Mutable<ILogicalOperator>> afterJoinRefs,
Mutable<ILogicalOperator> joinRef, OptimizableOperatorSubTree leftSubTree,
OptimizableOperatorSubTree rightSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
- IOptimizationContext context, boolean isLeftOuterJoin, boolean isLeftOuterJoinWithSpecialGroupBy)
- throws AlgebricksException;
+ IOptimizationContext context, boolean isLeftOuterJoin, boolean isLeftOuterJoinWithSpecialGroupBy,
+ IAlgebricksConstantValue leftOuterMissingValue) throws AlgebricksException;
/**
* Analyzes expr to see whether it is optimizable by the given concrete index.
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
index bc6d21c..6f53219 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
@@ -23,6 +23,7 @@
import java.util.Iterator;
import java.util.List;
import java.util.Map;
+import java.util.Objects;
import org.apache.asterix.metadata.entities.Dataset;
import org.apache.asterix.metadata.entities.Index;
@@ -38,6 +39,7 @@
import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
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.functions.FunctionIdentifier;
@@ -389,6 +391,9 @@
// Applies the plan transformation using chosen index.
AccessMethodAnalysisContext analysisCtx = analyzedAMs.get(chosenIndex.first);
+ IAlgebricksConstantValue leftOuterMissingValue =
+ isLeftOuterJoin ? ((LeftOuterJoinOperator) joinOp).getMissingValue() : null;
+
// For a left outer join with a special GroupBy, prepare objects to reset LOJ's
// nullPlaceHolderVariable in that GroupBy's nested plan.
// See AccessMethodUtils#removeUnjoinedDuplicatesInLOJ() for a definition of a special GroupBy
@@ -396,14 +401,16 @@
boolean isLeftOuterJoinWithSpecialGroupBy;
if (isLeftOuterJoin && op.getOperatorTag() == LogicalOperatorTag.GROUP) {
GroupByOperator groupByOp = (GroupByOperator) opRef.getValue();
- ScalarFunctionCallExpression isNullFuncExpr =
- AccessMethodUtils.findLOJIsMissingFuncInGroupBy(groupByOp, rightSubTree);
+ FunctionIdentifier isMissingNullFuncId = Objects
+ .requireNonNull(OperatorPropertiesUtil.getIsMissingNullFunction(leftOuterMissingValue));
+ ScalarFunctionCallExpression isMissingNullFuncExpr = AccessMethodUtils
+ .findLOJIsMissingNullFuncInGroupBy(groupByOp, rightSubTree, isMissingNullFuncId);
// TODO:(dmitry) do we need additional checks to ensure that this is a special GroupBy,
// i.e. that this GroupBy will eliminate unjoined duplicates?
- isLeftOuterJoinWithSpecialGroupBy = isNullFuncExpr != null;
+ isLeftOuterJoinWithSpecialGroupBy = isMissingNullFuncExpr != null;
if (isLeftOuterJoinWithSpecialGroupBy) {
analysisCtx.setLOJSpecialGroupByOpRef(opRef);
- analysisCtx.setLOJIsMissingFuncInSpecialGroupBy(isNullFuncExpr);
+ analysisCtx.setLOJIsMissingNullFuncInSpecialGroupBy(isMissingNullFuncExpr);
}
} else {
isLeftOuterJoinWithSpecialGroupBy = false;
@@ -422,7 +429,7 @@
// Finally, tries to apply plan transformation using the chosen index.
boolean res = chosenIndex.first.applyJoinPlanTransformation(afterJoinRefs, joinRef, leftSubTree,
rightSubTree, chosenIndex.second, analysisCtx, context, isLeftOuterJoin,
- isLeftOuterJoinWithSpecialGroupBy);
+ isLeftOuterJoinWithSpecialGroupBy, leftOuterMissingValue);
// If the plan transformation is successful, we don't need to traverse the plan
// any more, since if there are more JOIN operators, the next trigger on this plan
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroducePrimaryIndexForAggregationRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroducePrimaryIndexForAggregationRule.java
index d221fd1..cdeee0f 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroducePrimaryIndexForAggregationRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroducePrimaryIndexForAggregationRule.java
@@ -44,6 +44,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.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractScanOperator;
@@ -235,7 +236,8 @@
AbstractUnnestMapOperator primaryIndexUnnestOperator =
(AbstractUnnestMapOperator) AccessMethodUtils.createSecondaryIndexUnnestMap(dataset, recordType,
metaRecordType, primaryIndex, scanOperator.getInputs().get(0).getValue(),
- newBTreeParameters, context, retainInput, false, false);
+ newBTreeParameters, context, retainInput, false, false,
+ ConstantExpression.MISSING.getValue());
// re-use the PK variables of the original scan operator
primaryIndexUnnestOperator.getVariables().clear();
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
index 1027e65..d9b5da9 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
@@ -238,7 +238,7 @@
.getExecutionMode() == ExecutionMode.UNPARTITIONED;
ILogicalOperator subRoot = pair.first.createIndexSearchPlan(afterSelectRefs, selectRef, conditionRef,
subTree.getAssignsAndUnnestsRefs(), subTree, null, pair.second, analysisCtx, retainInput, false,
- requiresBroadcast, context, null);
+ requiresBroadcast, context, null, null);
if (subRoot == null) {
return false;
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
index a5a2907..ff3d887 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
@@ -65,6 +65,7 @@
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
@@ -400,7 +401,8 @@
List<Mutable<ILogicalOperator>> assignBeforeTopOpRefs, OptimizableOperatorSubTree indexSubTree,
OptimizableOperatorSubTree probeSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
boolean retainInput, boolean retainNull, boolean requiresBroadcast, IOptimizationContext context,
- LogicalVariable newNullPlaceHolderForLOJ) throws AlgebricksException {
+ LogicalVariable newMissingNullPlaceHolderForLOJ, IAlgebricksConstantValue leftOuterMissingValue)
+ throws AlgebricksException {
// TODO: we currently do not support the index-only plan for the inverted index searches since
// there can be many <SK, PK> pairs for the same PK and we may see two different records with the same PK
// (e.g., the record is deleted and inserted with the same PK). The reason is that there are
@@ -453,13 +455,13 @@
// since it doesn't contain a field value, only part of it.
ILogicalOperator secondaryIndexUnnestOp = AccessMethodUtils.createSecondaryIndexUnnestMap(dataset, recordType,
metaRecordType, chosenIndex, inputOp, jobGenParams, context, retainInput, retainNull,
- generateInstantTrylockResultFromIndexSearch);
+ generateInstantTrylockResultFromIndexSearch, leftOuterMissingValue);
// Generates the rest of the upstream plan which feeds the search results into the primary index.
ILogicalOperator primaryIndexUnnestOp = AccessMethodUtils.createRestOfIndexSearchPlan(afterTopOpRefs, topOpRef,
conditionRef, assignBeforeTopOpRefs, dataSourceScan, dataset, recordType, metaRecordType,
secondaryIndexUnnestOp, context, true, retainInput, retainNull, false, chosenIndex, analysisCtx,
- indexSubTree, null, newNullPlaceHolderForLOJ);
+ indexSubTree, null, newMissingNullPlaceHolderForLOJ, leftOuterMissingValue);
return primaryIndexUnnestOp;
}
@@ -487,10 +489,10 @@
SelectOperator selectOp = (SelectOperator) selectRef.getValue();
ILogicalOperator indexPlanRootOp =
createIndexSearchPlan(afterSelectRefs, selectRef, selectOp.getCondition(),
- subTree.getAssignsAndUnnestsRefs(),
- subTree, null, chosenIndex, analysisCtx, false, false, subTree.getDataSourceRef().getValue()
- .getInputs().get(0).getValue().getExecutionMode() == ExecutionMode.UNPARTITIONED,
- context, null);
+ subTree.getAssignsAndUnnestsRefs(), subTree,
+ null, chosenIndex, analysisCtx, false, false, subTree.getDataSourceRef().getValue().getInputs()
+ .get(0).getValue().getExecutionMode() == ExecutionMode.UNPARTITIONED,
+ context, null, null);
// Replace the datasource scan with the new plan rooted at primaryIndexUnnestMap.
subTree.getDataSourceRef().setValue(indexPlanRootOp);
@@ -501,8 +503,8 @@
public boolean applyJoinPlanTransformation(List<Mutable<ILogicalOperator>> afterJoinRefs,
Mutable<ILogicalOperator> joinRef, OptimizableOperatorSubTree leftSubTree,
OptimizableOperatorSubTree rightSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
- IOptimizationContext context, boolean isLeftOuterJoin, boolean isLeftOuterJoinWithSpecialGroupBy)
- throws AlgebricksException {
+ IOptimizationContext context, boolean isLeftOuterJoin, boolean isLeftOuterJoinWithSpecialGroupBy,
+ IAlgebricksConstantValue leftOuterMissingValue) throws AlgebricksException {
Dataset dataset = analysisCtx.getDatasetFromIndexDatasetMap(chosenIndex);
OptimizableOperatorSubTree indexSubTree;
OptimizableOperatorSubTree probeSubTree;
@@ -528,16 +530,17 @@
}
//if LOJ, reset null place holder variable
- LogicalVariable newNullPlaceHolderVar = null;
+ LogicalVariable newMissingNullPlaceHolderVar = null;
if (isLeftOuterJoin) {
//get a new null place holder variable that is the first field variable of the primary key
//from the indexSubTree's datasourceScanOp
// We need this for all left outer joins, even those that do not have a special GroupBy
- newNullPlaceHolderVar = indexSubTree.getDataSourceVariables().get(0);
+ newMissingNullPlaceHolderVar = indexSubTree.getDataSourceVariables().get(0);
if (isLeftOuterJoinWithSpecialGroupBy) {
//reset the null place holder variable
- AccessMethodUtils.resetLOJMissingPlaceholderVarInGroupByOp(analysisCtx, newNullPlaceHolderVar, context);
+ AccessMethodUtils.resetLOJMissingNullPlaceholderVarInGroupByOp(analysisCtx,
+ newMissingNullPlaceHolderVar, context);
}
}
@@ -573,14 +576,16 @@
probeSubTree.setRoot(newProbeRootRef.getValue());
}
// Create regular indexed-nested loop join path.
- ILogicalOperator indexPlanRootOp = createIndexSearchPlan(afterJoinRefs, joinRef,
- new MutableObject<ILogicalExpression>(joinCond), indexSubTree.getAssignsAndUnnestsRefs(), indexSubTree,
- probeSubTree, chosenIndex, analysisCtx, true, isLeftOuterJoin, true, context, newNullPlaceHolderVar);
+ ILogicalOperator indexPlanRootOp =
+ createIndexSearchPlan(afterJoinRefs, joinRef, new MutableObject<ILogicalExpression>(joinCond),
+ indexSubTree.getAssignsAndUnnestsRefs(), indexSubTree, probeSubTree, chosenIndex, analysisCtx,
+ true, isLeftOuterJoin, true, context, newMissingNullPlaceHolderVar, leftOuterMissingValue);
indexSubTree.getDataSourceRef().setValue(indexPlanRootOp);
// Change join into a select with the same condition.
- SelectOperator topSelect = new SelectOperator(new MutableObject<ILogicalExpression>(joinCond), isLeftOuterJoin,
- newNullPlaceHolderVar);
+ SelectOperator topSelect = isLeftOuterJoin
+ ? new SelectOperator(new MutableObject<>(joinCond), leftOuterMissingValue, newMissingNullPlaceHolderVar)
+ : new SelectOperator(new MutableObject<>(joinCond));
topSelect.setSourceLocation(indexPlanRootOp.getSourceLocation());
topSelect.getInputs().add(indexSubTree.getRootRef());
topSelect.setExecutionMode(ExecutionMode.LOCAL);
@@ -832,7 +837,7 @@
}
SelectOperator isFilterableSelectOp =
- new SelectOperator(new MutableObject<ILogicalExpression>(isFilterableExpr), false, null);
+ new SelectOperator(new MutableObject<ILogicalExpression>(isFilterableExpr));
isFilterableSelectOp.setSourceLocation(sourceLoc);
isFilterableSelectOp.getInputs().add(new MutableObject<ILogicalOperator>(inputOp));
isFilterableSelectOp.setExecutionMode(ExecutionMode.LOCAL);
@@ -845,7 +850,7 @@
FunctionUtil.getFunctionInfo(BuiltinFunctions.NOT), isNotFilterableArgs);
isNotFilterableExpr.setSourceLocation(sourceLoc);
SelectOperator isNotFilterableSelectOp =
- new SelectOperator(new MutableObject<ILogicalExpression>(isNotFilterableExpr), false, null);
+ new SelectOperator(new MutableObject<ILogicalExpression>(isNotFilterableExpr));
isNotFilterableSelectOp.setSourceLocation(sourceLoc);
isNotFilterableSelectOp.getInputs().add(new MutableObject<ILogicalOperator>(inputOp));
isNotFilterableSelectOp.setExecutionMode(ExecutionMode.LOCAL);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeAccessMethod.java
index 73b331c..8d4b83a 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeAccessMethod.java
@@ -52,6 +52,7 @@
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
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.functions.FunctionIdentifier;
@@ -190,11 +191,12 @@
analysisCtx.setIndexOnlyPlanInfo(indexOnlyPlanInfo);
- ILogicalOperator primaryIndexUnnestOp = createIndexSearchPlan(afterSelectRefs, selectRef,
- selectOp.getCondition(), subTree.getAssignsAndUnnestsRefs(), subTree, null, chosenIndex, analysisCtx,
- AccessMethodUtils.retainInputs(subTree.getDataSourceVariables(), subTree.getDataSourceRef().getValue(),
- afterSelectRefs),
- false, false, context, null);
+ ILogicalOperator primaryIndexUnnestOp =
+ createIndexSearchPlan(afterSelectRefs, selectRef, selectOp.getCondition(),
+ subTree.getAssignsAndUnnestsRefs(), subTree, null, chosenIndex, analysisCtx,
+ AccessMethodUtils.retainInputs(subTree.getDataSourceVariables(),
+ subTree.getDataSourceRef().getValue(), afterSelectRefs),
+ false, false, context, null, null);
if (primaryIndexUnnestOp == null) {
return false;
@@ -220,7 +222,8 @@
List<Mutable<ILogicalOperator>> assignBeforeTopRefs, OptimizableOperatorSubTree indexSubTree,
OptimizableOperatorSubTree probeSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
boolean retainInput, boolean retainNull, boolean requiresBroadcast, IOptimizationContext context,
- LogicalVariable newNullPlaceHolderForLOJ) throws AlgebricksException {
+ LogicalVariable newMissingNullPlaceHolderForLOJ, IAlgebricksConstantValue leftOuterMissingValue)
+ throws AlgebricksException {
// TODO: We can probably do something smarter here based on selectivity or MBR area.
IOptimizableFuncExpr optFuncExpr = AccessMethodUtils.chooseFirstOptFuncExpr(chosenIndex, analysisCtx);
@@ -302,7 +305,7 @@
ILogicalOperator secondaryIndexUnnestOp = AccessMethodUtils.createSecondaryIndexUnnestMap(dataset, recordType,
metaRecordType, chosenIndex, assignSearchKeys, jobGenParams, context, retainInput, retainNull,
- generateInstantTrylockResultFromIndexSearch);
+ generateInstantTrylockResultFromIndexSearch, leftOuterMissingValue);
// Generates the rest of the upstream plan which feeds the search results into the primary index.
return dataset.getDatasetType() == DatasetType.EXTERNAL
@@ -311,15 +314,15 @@
: AccessMethodUtils.createRestOfIndexSearchPlan(afterTopRefs, topRef, conditionRef, assignBeforeTopRefs,
dataSourceOp, dataset, recordType, metaRecordType, secondaryIndexUnnestOp, context, true,
retainInput, retainNull, false, chosenIndex, analysisCtx, indexSubTree, null,
- newNullPlaceHolderForLOJ);
+ newMissingNullPlaceHolderForLOJ, leftOuterMissingValue);
}
@Override
public boolean applyJoinPlanTransformation(List<Mutable<ILogicalOperator>> afterJoinRefs,
Mutable<ILogicalOperator> joinRef, OptimizableOperatorSubTree leftSubTree,
OptimizableOperatorSubTree rightSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
- IOptimizationContext context, boolean isLeftOuterJoin, boolean isLeftOuterJoinWithSpecialGroupBy)
- throws AlgebricksException {
+ IOptimizationContext context, boolean isLeftOuterJoin, boolean isLeftOuterJoinWithSpecialGroupBy,
+ IAlgebricksConstantValue leftOuterMissingValue) throws AlgebricksException {
AbstractBinaryJoinOperator joinOp = (AbstractBinaryJoinOperator) joinRef.getValue();
Mutable<ILogicalExpression> conditionRef = joinOp.getCondition();
@@ -344,12 +347,12 @@
return false;
}
- LogicalVariable newNullPlaceHolderVar = null;
+ LogicalVariable newMissingNullPlaceHolderVar = null;
if (isLeftOuterJoin) {
- // Gets a new null place holder variable that is the first field variable of the primary key
+ // Gets a new missing (or null) place holder variable that is the first field variable of the primary key
// from the indexSubTree's datasourceScanOp.
// We need this for all left outer joins, even those that do not have a special GroupBy
- newNullPlaceHolderVar = indexSubTree.getDataSourceVariables().get(0);
+ newMissingNullPlaceHolderVar = indexSubTree.getDataSourceVariables().get(0);
}
boolean canContinue = AccessMethodUtils.setIndexOnlyPlanInfo(afterJoinRefs, joinRef, probeSubTree, indexSubTree,
@@ -360,15 +363,15 @@
ILogicalOperator indexSearchOp = createIndexSearchPlan(afterJoinRefs, joinRef, conditionRef,
indexSubTree.getAssignsAndUnnestsRefs(), indexSubTree, probeSubTree, chosenIndex, analysisCtx, true,
- isLeftOuterJoin, true, context, newNullPlaceHolderVar);
+ isLeftOuterJoin, true, context, newMissingNullPlaceHolderVar, leftOuterMissingValue);
if (indexSearchOp == null) {
return false;
}
return AccessMethodUtils.finalizeJoinPlanTransformation(afterJoinRefs, joinRef, indexSubTree, probeSubTree,
- analysisCtx, context, isLeftOuterJoin, isLeftOuterJoinWithSpecialGroupBy, indexSearchOp,
- newNullPlaceHolderVar, conditionRef, dataset, chosenIndex);
+ analysisCtx, context, isLeftOuterJoin, isLeftOuterJoinWithSpecialGroupBy, leftOuterMissingValue,
+ indexSearchOp, newMissingNullPlaceHolderVar, conditionRef, dataset, chosenIndex);
}
@Override
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/AbstractOperatorFromSubplanRewrite.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/AbstractOperatorFromSubplanRewrite.java
index 0f7f354..ba70aff 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/AbstractOperatorFromSubplanRewrite.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/AbstractOperatorFromSubplanRewrite.java
@@ -136,7 +136,7 @@
// Create a copy of this SELECT, and set this to our rewrite root.
SelectOperator rewriteRootSelect = new SelectOperator(new MutableObject<>(normalizedSelectCondition),
- optimizableSelect.getRetainMissing(), optimizableSelect.getMissingPlaceholderVariable());
+ optimizableSelect.getRetainMissingAsValue(), optimizableSelect.getMissingPlaceholderVariable());
rewriteRootSelect.setSourceLocation(sourceLocation);
rewriteRootSelect.setExecutionMode(optimizableSelect.getExecutionMode());
@@ -178,7 +178,8 @@
// include this condition.
updatedSelectCond = coalesceConditions(rewriteRootSelect, workingOriginalOperator);
updatedSelectOperator = new SelectOperator(new MutableObject<>(updatedSelectCond),
- rewriteRootSelect.getRetainMissing(), rewriteRootSelect.getMissingPlaceholderVariable());
+ rewriteRootSelect.getRetainMissingAsValue(),
+ rewriteRootSelect.getMissingPlaceholderVariable());
updatedSelectOperator.setSourceLocation(sourceLocation);
updatedSelectOperator.getInputs().addAll(rewriteRootSelect.getInputs());
rewriteRootSelect = updatedSelectOperator;
@@ -194,7 +195,7 @@
if (traversalOutput != null) {
updatedSelectCond = coalesceConditions(rewriteRootSelect, traversalOutput.first);
updatedSelectOperator = new SelectOperator(new MutableObject<>(updatedSelectCond),
- rewriteRootSelect.getRetainMissing(),
+ rewriteRootSelect.getRetainMissingAsValue(),
rewriteRootSelect.getMissingPlaceholderVariable());
updatedSelectOperator.setSourceLocation(sourceLocation);
updatedSelectOperator.getInputs().addAll(rewriteRootSelect.getInputs());
@@ -317,7 +318,7 @@
// First, try to create a SELECT from the aggregate itself (i.e. handle the SOME AND EVERY case).
if (isNonEmptyStream && aggregateCondition != null) {
- SelectOperator selectFromAgg = new SelectOperator(new MutableObject<>(aggregateCondition), false, null);
+ SelectOperator selectFromAgg = new SelectOperator(new MutableObject<>(aggregateCondition));
selectFromAgg.getInputs().addAll(subplanRoot.getInputs());
selectFromAgg.setSourceLocation(sourceLocation);
return selectFromAgg;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/MergedSelectRewrite.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/MergedSelectRewrite.java
index 5aefdbf..75168f5 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/MergedSelectRewrite.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/MergedSelectRewrite.java
@@ -84,7 +84,7 @@
// Return a new plan that removes all SELECTs that were pushed up.
SelectOperator newSelectOperator = new SelectOperator(new MutableObject<>(andCond),
- originalOperator.getRetainMissing(), originalOperator.getMissingPlaceholderVariable());
+ originalOperator.getRetainMissingAsValue(), originalOperator.getMissingPlaceholderVariable());
newSelectOperator.setSourceLocation(originalOperator.getSourceLocation());
ILogicalPlan newSelectInputPlan = OperatorManipulationUtil
.deepCopy(new ALogicalPlanImpl(originalOperator.getInputs().get(0)), context);
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 5616f3f..a8dfe1e 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,6 +29,7 @@
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.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;
@@ -177,6 +178,6 @@
private static Set<FunctionIdentifier> createSupportedTypeCheckFunctions() {
return Set.of(BuiltinFunctions.IS_ARRAY, BuiltinFunctions.IS_OBJECT, BuiltinFunctions.IS_ATOMIC,
BuiltinFunctions.IS_NUMBER, BuiltinFunctions.IS_BOOLEAN, BuiltinFunctions.IS_STRING,
- BuiltinFunctions.IS_MISSING, BuiltinFunctions.IS_NULL, BuiltinFunctions.IS_UNKNOWN);
+ AlgebricksBuiltinFunctions.IS_MISSING, AlgebricksBuiltinFunctions.IS_NULL, BuiltinFunctions.IS_UNKNOWN);
}
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineLeftNtsInSubplanJoinFlatteningVisitor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineLeftNtsInSubplanJoinFlatteningVisitor.java
index 8326f85..d4b7853 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineLeftNtsInSubplanJoinFlatteningVisitor.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineLeftNtsInSubplanJoinFlatteningVisitor.java
@@ -21,6 +21,7 @@
import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
+import java.util.Objects;
import java.util.Set;
import org.apache.commons.lang3.mutable.Mutable;
@@ -33,6 +34,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.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
@@ -89,6 +91,7 @@
// The target Nts operator.
private final ILogicalOperator targetNts;
+ private final IAlgebricksConstantValue leftOuterMissingValue;
// The live variables in <code>subplanInputOperator</code> to enforce.
private final Set<LogicalVariable> liveVarsFromSubplanInput = new HashSet<>();
@@ -112,13 +115,16 @@
* the input operator to the target SubplanOperator
* @param nts
* the NestedTupleSourceOperator to be replaced by <code>subplanInputOperator</code>
+ * @param leftOuterMissingValue
* @throws AlgebricksException
*/
public InlineLeftNtsInSubplanJoinFlatteningVisitor(IOptimizationContext context,
- ILogicalOperator subplanInputOperator, ILogicalOperator nts) throws AlgebricksException {
+ ILogicalOperator subplanInputOperator, ILogicalOperator nts, IAlgebricksConstantValue leftOuterMissingValue)
+ throws AlgebricksException {
this.context = context;
this.subplanInputOperator = subplanInputOperator;
this.targetNts = nts;
+ this.leftOuterMissingValue = Objects.requireNonNull(leftOuterMissingValue);
VariableUtilities.getSubplanLocalLiveVariables(subplanInputOperator, liveVarsFromSubplanInput);
}
@@ -192,7 +198,7 @@
AbstractBinaryJoinOperator returnOp = op;
// After rewriting, the original inner join should become an left outer join.
if (rewritten) {
- returnOp = new LeftOuterJoinOperator(op.getCondition());
+ returnOp = new LeftOuterJoinOperator(op.getCondition(), leftOuterMissingValue);
returnOp.setSourceLocation(op.getSourceLocation());
returnOp.getInputs().addAll(op.getInputs());
injectNullCheckVars(returnOp);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineSubplanInputForNestedTupleSourceRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineSubplanInputForNestedTupleSourceRule.java
index ae9a886..9f932a8 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineSubplanInputForNestedTupleSourceRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineSubplanInputForNestedTupleSourceRule.java
@@ -43,6 +43,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.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
@@ -59,6 +60,7 @@
import org.apache.hyracks.algebricks.core.algebra.plan.ALogicalPlanImpl;
import org.apache.hyracks.algebricks.core.algebra.properties.FunctionalDependency;
import org.apache.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
+import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
import org.apache.hyracks.api.exceptions.SourceLocation;
@@ -300,16 +302,19 @@
return changedAndVarMap;
}
- /**
+ IAlgebricksConstantValue leftOuterMissingValue = ConstantExpression.MISSING.getValue();
+
+ /*
* Apply the special join-based rewriting.
*/
- Pair<Boolean, LinkedHashMap<LogicalVariable, LogicalVariable>> result = applySpecialFlattening(opRef, context);
+ Pair<Boolean, LinkedHashMap<LogicalVariable, LogicalVariable>> result =
+ applySpecialFlattening(opRef, context, leftOuterMissingValue);
if (!result.first) {
- /**
+ /*
* If the special join-based rewriting does not apply, apply the general
* rewriting which blindly inlines all NTSs.
*/
- result = applyGeneralFlattening(opRef, context);
+ result = applyGeneralFlattening(opRef, context, leftOuterMissingValue);
}
LinkedHashMap<LogicalVariable, LogicalVariable> returnedMap = new LinkedHashMap<>();
// Adds variable mappings from input operators.
@@ -359,7 +364,8 @@
}
private Pair<Boolean, LinkedHashMap<LogicalVariable, LogicalVariable>> applyGeneralFlattening(
- Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+ Mutable<ILogicalOperator> opRef, IOptimizationContext context,
+ IAlgebricksConstantValue leftOuterMissingValue) throws AlgebricksException {
SubplanOperator subplanOp = (SubplanOperator) opRef.getValue();
if (!SubplanFlatteningUtil.containsOperators(subplanOp, EnumSet.of(LogicalOperatorTag.DATASOURCESCAN,
LogicalOperatorTag.INNERJOIN, LogicalOperatorTag.LEFTOUTERJOIN))) {
@@ -437,8 +443,8 @@
} else {
joinExpr = joinPredicates.size() > 0 ? joinPredicates.get(0).getValue() : ConstantExpression.TRUE;
}
- LeftOuterJoinOperator leftOuterJoinOp =
- new LeftOuterJoinOperator(new MutableObject<>(joinExpr), inputOpRef, rightInputOpRef);
+ LeftOuterJoinOperator leftOuterJoinOp = new LeftOuterJoinOperator(new MutableObject<>(joinExpr), inputOpRef,
+ rightInputOpRef, leftOuterMissingValue);
leftOuterJoinOp.setSourceLocation(sourceLoc);
OperatorManipulationUtil.computeTypeEnvironmentBottomUp(rightInputOp, context);
context.computeAndSetTypeEnvironmentForOperator(leftOuterJoinOp);
@@ -481,7 +487,7 @@
lowestAggregateRefInSubplan.getValue().getInputs().add(currentOpRef);
}
- // Adds a select operator into the nested plan for group-by to remove tuples with NULL on {@code assignVar},
+ // Adds a select operator into the nested plan for group-by to remove tuples with MISSING on {@code assignVar},
// i.e., subplan input tuples that are filtered out within a subplan.
VariableReferenceExpression assignVarRef = new VariableReferenceExpression(assignVar);
assignVarRef.setSourceLocation(sourceLoc);
@@ -489,14 +495,15 @@
List<Mutable<ILogicalExpression>> args = new ArrayList<>();
args.add(filterVarExpr);
List<Mutable<ILogicalExpression>> argsForNotFunction = new ArrayList<>();
- ScalarFunctionCallExpression isMissingExpr =
- new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.IS_MISSING), args);
+ ScalarFunctionCallExpression isMissingExpr = new ScalarFunctionCallExpression(
+ FunctionUtil.getFunctionInfo(OperatorPropertiesUtil.getIsMissingNullFunction(leftOuterMissingValue)),
+ args);
isMissingExpr.setSourceLocation(sourceLoc);
argsForNotFunction.add(new MutableObject<>(isMissingExpr));
ScalarFunctionCallExpression notExpr = new ScalarFunctionCallExpression(
FunctionUtil.getFunctionInfo(BuiltinFunctions.NOT), argsForNotFunction);
notExpr.setSourceLocation(sourceLoc);
- SelectOperator selectOp = new SelectOperator(new MutableObject<>(notExpr), false, null);
+ SelectOperator selectOp = new SelectOperator(new MutableObject<>(notExpr));
selectOp.setSourceLocation(sourceLoc);
currentOpRef.getValue().getInputs().add(new MutableObject<>(selectOp));
@@ -524,7 +531,8 @@
}
private Pair<Boolean, LinkedHashMap<LogicalVariable, LogicalVariable>> applySpecialFlattening(
- Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+ Mutable<ILogicalOperator> opRef, IOptimizationContext context,
+ IAlgebricksConstantValue leftOuterMissingValue) throws AlgebricksException {
SubplanOperator subplanOp = (SubplanOperator) opRef.getValue();
SourceLocation sourceLoc = subplanOp.getSourceLocation();
Mutable<ILogicalOperator> inputOpRef = subplanOp.getInputs().get(0);
@@ -554,8 +562,8 @@
Set<LogicalVariable> liveVars = new HashSet<>();
VariableUtilities.getLiveVariables(inputOp, liveVars);
- Pair<Set<LogicalVariable>, Mutable<ILogicalOperator>> notNullVarsAndTopJoinRef =
- SubplanFlatteningUtil.inlineLeftNtsInSubplanJoin(subplanOp, context, newPrimaryKeyFd);
+ Pair<Set<LogicalVariable>, Mutable<ILogicalOperator>> notNullVarsAndTopJoinRef = SubplanFlatteningUtil
+ .inlineLeftNtsInSubplanJoin(subplanOp, context, newPrimaryKeyFd, leftOuterMissingValue);
if (notNullVarsAndTopJoinRef.first == null) {
inputOpRef.setValue(inputOpBackup);
return new Pair<>(false, replacedVarMap);
@@ -588,7 +596,7 @@
groupbyOp.getInputs().add(new MutableObject<>(topJoinRef.getValue()));
if (!notNullVars.isEmpty()) {
- // Adds a select operator into the nested plan for group-by to remove tuples with NULL on {@code assignVar},
+ // Adds a select operator into the nested plan for group-by to remove tuples with MISSING on {@code assignVar},
// i.e., subplan input tuples that are filtered out within a subplan.
List<Mutable<ILogicalExpression>> nullCheckExprRefs = new ArrayList<>();
for (LogicalVariable notNullVar : notNullVars) {
@@ -598,8 +606,8 @@
List<Mutable<ILogicalExpression>> args = new ArrayList<>();
args.add(filterVarExpr);
List<Mutable<ILogicalExpression>> argsForNotFunction = new ArrayList<>();
- ScalarFunctionCallExpression isMissingExpr = new ScalarFunctionCallExpression(
- FunctionUtil.getFunctionInfo(BuiltinFunctions.IS_MISSING), args);
+ ScalarFunctionCallExpression isMissingExpr = new ScalarFunctionCallExpression(FunctionUtil
+ .getFunctionInfo(OperatorPropertiesUtil.getIsMissingNullFunction(leftOuterMissingValue)), args);
isMissingExpr.setSourceLocation(sourceLoc);
argsForNotFunction.add(new MutableObject<>(isMissingExpr));
ScalarFunctionCallExpression notExpr = new ScalarFunctionCallExpression(
@@ -616,7 +624,7 @@
} else {
selectExprRef = nullCheckExprRefs.get(0);
}
- SelectOperator selectOp = new SelectOperator(selectExprRef, false, null);
+ SelectOperator selectOp = new SelectOperator(selectExprRef);
selectOp.setSourceLocation(sourceLoc);
topJoinRef.setValue(selectOp);
NestedTupleSourceOperator ntsOp = new NestedTupleSourceOperator(new MutableObject<>(groupbyOp));
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/SubplanFlatteningUtil.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/SubplanFlatteningUtil.java
index e3c7d69..916d75d 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/SubplanFlatteningUtil.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/SubplanFlatteningUtil.java
@@ -32,6 +32,7 @@
import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
@@ -101,13 +102,14 @@
* the optimization context
* @param extraPrimaryKeyFd
* extra primary key dependency that needs to be added to the context before performing the rewrite
+ * @param leftOuterMissingValue
* @return A set of variables used for further null-checks, i.e., variables indicating
* whether a tuple produced by a transformed left outer join is a non-match;
* a reference to the top join operator in the nested subplan.
*/
public static Pair<Set<LogicalVariable>, Mutable<ILogicalOperator>> inlineLeftNtsInSubplanJoin(
- SubplanOperator subplanOp, IOptimizationContext context, FunctionalDependency extraPrimaryKeyFd)
- throws AlgebricksException {
+ SubplanOperator subplanOp, IOptimizationContext context, FunctionalDependency extraPrimaryKeyFd,
+ IAlgebricksConstantValue leftOuterMissingValue) throws AlgebricksException {
Pair<Boolean, ILogicalOperator> applicableAndNtsToRewrite =
SubplanFlatteningUtil.isQualifiedForSpecialFlattening(subplanOp);
if (!applicableAndNtsToRewrite.first) {
@@ -120,8 +122,8 @@
ILogicalOperator qualifiedNts = applicableAndNtsToRewrite.second;
ILogicalOperator subplanInputOp = subplanOp.getInputs().get(0).getValue();
- InlineLeftNtsInSubplanJoinFlatteningVisitor specialVisitor =
- new InlineLeftNtsInSubplanJoinFlatteningVisitor(context, subplanInputOp, qualifiedNts);
+ InlineLeftNtsInSubplanJoinFlatteningVisitor specialVisitor = new InlineLeftNtsInSubplanJoinFlatteningVisitor(
+ context, subplanInputOp, qualifiedNts, leftOuterMissingValue);
// Rewrites the query plan.
Mutable<ILogicalOperator> topRef = subplanOp.getNestedPlans().get(0).getRoots().get(0);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
index d664da1..794a0b7 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
@@ -1372,7 +1372,7 @@
switch (qe.getQuantifier()) {
case SOME:
- SelectOperator s = new SelectOperator(new MutableObject<>(eo2.first), false, null);
+ SelectOperator s = new SelectOperator(new MutableObject<>(eo2.first));
s.getInputs().add(eo2.second);
s.setSourceLocation(sourceLoc);
AggregateFunctionCallExpression fAgg = BuiltinFunctions
@@ -1400,7 +1400,7 @@
ScalarFunctionCallExpression notExpr = new ScalarFunctionCallExpression(
BuiltinFunctions.getBuiltinFunctionInfo(AlgebricksBuiltinFunctions.NOT), notArgs);
notExpr.setSourceLocation(sourceLoc);
- s = new SelectOperator(new MutableObject<>(notExpr), false, null);
+ s = new SelectOperator(new MutableObject<>(notExpr));
s.getInputs().add(eo2.second);
s.setSourceLocation(sourceLoc);
fAgg = BuiltinFunctions.makeAggregateFunctionExpression(BuiltinFunctions.EMPTY_STREAM,
@@ -1570,7 +1570,7 @@
public Pair<ILogicalOperator, LogicalVariable> visit(WhereClause w, Mutable<ILogicalOperator> tupSource)
throws CompilationException {
Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = langExprToAlgExpression(w.getWhereExpr(), tupSource);
- SelectOperator s = new SelectOperator(new MutableObject<>(p.first), false, null);
+ SelectOperator s = new SelectOperator(new MutableObject<>(p.first));
s.getInputs().add(p.second);
s.setSourceLocation(w.getSourceLocation());
return new Pair<>(s, null);
@@ -2024,7 +2024,7 @@
subplanOp.setSourceLocation(sourceLoc);
NestedTupleSourceOperator ntsOp = new NestedTupleSourceOperator(new MutableObject<>(subplanOp));
ntsOp.setSourceLocation(sourceLoc);
- SelectOperator select = new SelectOperator(selectExpr, false, null);
+ SelectOperator select = new SelectOperator(selectExpr);
// The select operator cannot be moved up and down, otherwise it will cause
// typing issues (ASTERIXDB-1203).
OperatorPropertiesUtil.markMovable(select, false);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java
index 2dd4f96..65b0f03 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java
@@ -112,6 +112,7 @@
import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractLogicalExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.AggregateFunctionCallExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionAnnotation;
import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.UnnestingFunctionCallExpression;
@@ -347,7 +348,7 @@
if (joinClause.getJoinType() == JoinType.INNER) {
Mutable<ILogicalOperator> rightInputRef = uncorrelatedRightBranchStack.peek();
Pair<ILogicalOperator, LogicalVariable> rightBranch =
- generateUnnestForBinaryCorrelateRightBranch(joinClause, rightInputRef, true);
+ generateUnnestForBinaryCorrelateRightBranch(joinClause, rightInputRef, false, null);
// A join operator with condition TRUE.
AbstractBinaryJoinOperator joinOperator = new InnerJoinOperator(
new MutableObject<>(ConstantExpression.TRUE), leftInputRef, new MutableObject<>(rightBranch.first));
@@ -357,7 +358,7 @@
// Add an additional filter operator.
Pair<ILogicalExpression, Mutable<ILogicalOperator>> conditionExprOpPair =
langExprToAlgExpression(joinClause.getConditionExpression(), joinOpRef);
- SelectOperator filter = new SelectOperator(new MutableObject<>(conditionExprOpPair.first), false, null);
+ SelectOperator filter = new SelectOperator(new MutableObject<>(conditionExprOpPair.first));
filter.getInputs().add(conditionExprOpPair.second);
filter.setSourceLocation(conditionExprOpPair.first.getSourceLocation());
return new Pair<>(filter, rightBranch.second);
@@ -375,13 +376,13 @@
// Adds an unnest operator to unnest to right expression.
Pair<ILogicalOperator, LogicalVariable> rightBranch =
- generateUnnestForBinaryCorrelateRightBranch(joinClause, ntsRef, true);
+ generateUnnestForBinaryCorrelateRightBranch(joinClause, ntsRef, false, null);
AbstractUnnestNonMapOperator rightUnnestOp = (AbstractUnnestNonMapOperator) rightBranch.first;
// Adds an additional filter operator for the join condition.
Pair<ILogicalExpression, Mutable<ILogicalOperator>> conditionExprOpPair =
langExprToAlgExpression(joinClause.getConditionExpression(), new MutableObject<>(rightUnnestOp));
- SelectOperator filter = new SelectOperator(new MutableObject<>(conditionExprOpPair.first), false, null);
+ SelectOperator filter = new SelectOperator(new MutableObject<>(conditionExprOpPair.first));
filter.getInputs().add(conditionExprOpPair.second);
filter.setSourceLocation(conditionExprOpPair.first.getSourceLocation());
@@ -448,7 +449,8 @@
Pair<ILogicalExpression, Mutable<ILogicalOperator>> pUnnestExpr =
makeUnnestExpression(aggVarRefExpr, new MutableObject<>(subplanOp));
LeftOuterUnnestOperator outerUnnestOp =
- new LeftOuterUnnestOperator(outerUnnestVar, new MutableObject<>(pUnnestExpr.first));
+ new LeftOuterUnnestOperator(outerUnnestVar, new MutableObject<>(pUnnestExpr.first),
+ translateLeftOuterMissingValue(joinClause.getOuterJoinMissingValueType()));
outerUnnestOp.getInputs().add(pUnnestExpr.second);
outerUnnestOp.setSourceLocation(aggOp.getSourceLocation());
currentTopOp = outerUnnestOp;
@@ -506,6 +508,18 @@
}
}
+ private static IAlgebricksConstantValue translateLeftOuterMissingValue(Literal.Type type)
+ throws CompilationException {
+ switch (type) {
+ case MISSING:
+ return ConstantExpression.MISSING.getValue();
+ case NULL:
+ return ConstantExpression.NULL.getValue();
+ default:
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, String.valueOf(type));
+ }
+ }
+
@Override
public Pair<ILogicalOperator, LogicalVariable> visit(NestClause nestClause, Mutable<ILogicalOperator> arg)
throws CompilationException {
@@ -516,8 +530,9 @@
@Override
public Pair<ILogicalOperator, LogicalVariable> visit(UnnestClause unnestClause,
Mutable<ILogicalOperator> inputOpRef) throws CompilationException {
- return generateUnnestForBinaryCorrelateRightBranch(unnestClause, inputOpRef,
- unnestClause.getUnnestType() == UnnestType.INNER);
+ boolean outerUnnest = unnestClause.getUnnestType() == UnnestType.LEFTOUTER;
+ return generateUnnestForBinaryCorrelateRightBranch(unnestClause, inputOpRef, outerUnnest,
+ outerUnnest ? translateLeftOuterMissingValue(unnestClause.getOuterUnnestMissingValueType()) : null);
}
@Override
@@ -525,14 +540,14 @@
throws CompilationException {
Pair<ILogicalExpression, Mutable<ILogicalOperator>> p =
langExprToAlgExpression(havingClause.getFilterExpression(), tupSource);
- SelectOperator s = new SelectOperator(new MutableObject<>(p.first), false, null);
+ SelectOperator s = new SelectOperator(new MutableObject<>(p.first));
s.getInputs().add(p.second);
return new Pair<>(s, null);
}
private Pair<ILogicalOperator, LogicalVariable> generateUnnestForBinaryCorrelateRightBranch(
- AbstractBinaryCorrelateClause binaryCorrelate, Mutable<ILogicalOperator> inputOpRef, boolean innerUnnest)
- throws CompilationException {
+ AbstractBinaryCorrelateClause binaryCorrelate, Mutable<ILogicalOperator> inputOpRef, boolean outerUnnest,
+ IAlgebricksConstantValue outerUnnestMissingValue) throws CompilationException {
LogicalVariable rightVar = context.newVarFromExpression(binaryCorrelate.getRightVariable());
Expression rightExpr = binaryCorrelate.getRightExpression();
Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = langExprToAlgExpression(rightExpr, inputOpRef);
@@ -541,13 +556,15 @@
if (binaryCorrelate.hasPositionalVariable()) {
LogicalVariable pVar = context.newVarFromExpression(binaryCorrelate.getPositionalVariable());
// We set the positional variable type as BIGINT type.
- unnestOp = innerUnnest
- ? new UnnestOperator(rightVar, new MutableObject<>(pUnnestExpr.first), pVar, BuiltinType.AINT64)
- : new LeftOuterUnnestOperator(rightVar, new MutableObject<>(pUnnestExpr.first), pVar,
- BuiltinType.AINT64);
+ unnestOp = outerUnnest
+ ? new LeftOuterUnnestOperator(rightVar, new MutableObject<>(pUnnestExpr.first), pVar,
+ BuiltinType.AINT64, outerUnnestMissingValue)
+ : new UnnestOperator(rightVar, new MutableObject<>(pUnnestExpr.first), pVar, BuiltinType.AINT64);
} else {
- unnestOp = innerUnnest ? new UnnestOperator(rightVar, new MutableObject<>(pUnnestExpr.first))
- : new LeftOuterUnnestOperator(rightVar, new MutableObject<>(pUnnestExpr.first));
+ unnestOp = outerUnnest
+ ? new LeftOuterUnnestOperator(rightVar, new MutableObject<>(pUnnestExpr.first),
+ outerUnnestMissingValue)
+ : new UnnestOperator(rightVar, new MutableObject<>(pUnnestExpr.first));
}
unnestOp.getInputs().add(pUnnestExpr.second);
unnestOp.setSourceLocation(binaryCorrelate.getRightVariable().getSourceLocation());
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
index f2bb66d..3a9d54d 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
@@ -320,6 +320,7 @@
builder.setHashFunctionFactoryProvider(format.getBinaryHashFunctionFactoryProvider());
builder.setHashFunctionFamilyProvider(format.getBinaryHashFunctionFamilyProvider());
builder.setMissingWriterFactory(format.getMissingWriterFactory());
+ builder.setNullWriterFactory(format.getNullWriterFactory());
builder.setUnnestingPositionWriterFactory(format.getUnnestingPositionWriterFactory());
builder.setPredicateEvaluatorFactoryProvider(format.getPredicateEvaluatorFactoryProvider());
builder.setPrinterProvider(getPrinterFactoryProvider(format, conf.fmt()));
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
index d4e9cf8..f40628fb 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
@@ -439,7 +439,7 @@
storageComponentProvider.getStorageManager(), primaryIndexInfo.getFileSplitProvider());
BTreeSearchOperatorDescriptor searchOpDesc = new BTreeSearchOperatorDescriptor(spec, primaryIndexInfo.rDesc,
null, null, true, true, indexDataflowHelperFactory, false, false, null,
- NoOpOperationCallbackFactory.INSTANCE, filterFields, filterFields, false);
+ NoOpOperationCallbackFactory.INSTANCE, filterFields, filterFields, false, null);
BTreeSearchOperatorNodePushable searchOp =
searchOpDesc.createPushRuntime(ctx, primaryIndexInfo.getSearchRecordDescriptorProvider(),
ctx.getTaskAttemptId().getTaskId().getPartition(), 1);
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/resource/PlanStagesGeneratorTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/resource/PlanStagesGeneratorTest.java
index 8969f18..b0de85e 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/resource/PlanStagesGeneratorTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/resource/PlanStagesGeneratorTest.java
@@ -210,7 +210,8 @@
exchangeOperator2.setPhysicalOperator(new HashPartitionExchangePOperator(Collections.emptyList(), null));
exchangeOperator2.getInputs().add(new MutableObject<>(groupByOperator));
- LeftOuterJoinOperator secondJoin = new LeftOuterJoinOperator(new MutableObject<>(ConstantExpression.TRUE));
+ LeftOuterJoinOperator secondJoin = new LeftOuterJoinOperator(new MutableObject<>(ConstantExpression.TRUE),
+ ConstantExpression.MISSING.getValue());
secondJoin.setExecutionMode(PARTITIONED);
secondJoin.setPhysicalOperator(new NestedLoopJoinPOperator(secondJoin.getJoinKind(),
AbstractJoinPOperator.JoinPartitioningType.BROADCAST));
@@ -270,7 +271,8 @@
order2.setPhysicalOperator(new StableSortPOperator());
order2.getInputs().add(new MutableObject<>(replicateOperator));
- LeftOuterJoinOperator secondJoin = new LeftOuterJoinOperator(new MutableObject<>(ConstantExpression.TRUE));
+ LeftOuterJoinOperator secondJoin = new LeftOuterJoinOperator(new MutableObject<>(ConstantExpression.TRUE),
+ ConstantExpression.MISSING.getValue());
secondJoin.setExecutionMode(PARTITIONED);
secondJoin.setPhysicalOperator(new NestedLoopJoinPOperator(secondJoin.getJoinKind(),
AbstractJoinPOperator.JoinPartitioningType.BROADCAST));
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppRQGGroupingSetsIT.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppRQGGroupingSetsIT.java
index 2ccab70..e9db5fa 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppRQGGroupingSetsIT.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppRQGGroupingSetsIT.java
@@ -102,7 +102,7 @@
@Test
public void test() throws Exception {
- runTestCase(testcaseId, groupByClause, sqlQuery, sqlppQuery);
+ runTestCase(testcaseId, groupByClause, sqlQuery, sqlppQuery, null);
}
private static TestQuery generateQuery(int testcaseId, Random random) {
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppRQGJoinsIT.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppRQGJoinsIT.java
index 96e188e..7dfd0f0 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppRQGJoinsIT.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppRQGJoinsIT.java
@@ -21,9 +21,13 @@
import java.util.ArrayList;
import java.util.Collection;
+import java.util.Collections;
import java.util.List;
import java.util.function.IntUnaryOperator;
+import org.apache.asterix.api.http.server.QueryServiceRequestParameters;
+import org.apache.asterix.testframework.xml.ParameterTypeEnum;
+import org.apache.asterix.testframework.xml.TestCase;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
@@ -39,7 +43,9 @@
private final String sqlppQuery;
- private final String desc;
+ private final boolean sqlCompatMode;
+
+ private final String description;
static final String PROJECT_FIELD = "unique1";
@@ -49,31 +55,34 @@
static final char[] SHAPES = new char[] { 'c', 's', 'q' };
- @Parameterized.Parameters(name = "SqlppRQGJoinsIT {index}: {3}")
+ @Parameterized.Parameters(name = "SqlppRQGJoinsIT {index}: {1}")
public static Collection<Object[]> tests() {
List<Object[]> testCases = new ArrayList<>();
IntUnaryOperator filterComputer = i -> 2 * (i + 1);
+ boolean[] allSqlCompatModes = new boolean[] { false, true };
String[] allJoinKinds = new String[] { "INNER", "LEFT", "RIGHT" };
String[] queryJoinKinds = new String[3];
int id = 0;
- for (String jk0 : allJoinKinds) {
- queryJoinKinds[0] = jk0;
- TestQuery q1 = generateQuery(queryJoinKinds, 1, filterComputer, SHAPES[0]);
- testCases.add(new Object[] { id++, q1.sqlQuery, q1.sqlppQuery, q1.summary });
+ for (boolean sqlCompatMode : allSqlCompatModes) {
+ for (String jk0 : allJoinKinds) {
+ queryJoinKinds[0] = jk0;
+ TestQuery q1 = generateQuery(queryJoinKinds, 1, filterComputer, SHAPES[0], sqlCompatMode);
+ addTestCase(testCases, id++, q1);
- for (char s : SHAPES) {
- for (String jk1 : allJoinKinds) {
- queryJoinKinds[1] = jk1;
- TestQuery q2 = generateQuery(queryJoinKinds, 2, filterComputer, s);
- testCases.add(new Object[] { id++, q2.sqlQuery, q2.sqlppQuery, q2.summary });
+ for (char s : SHAPES) {
+ for (String jk1 : allJoinKinds) {
+ queryJoinKinds[1] = jk1;
+ TestQuery q2 = generateQuery(queryJoinKinds, 2, filterComputer, s, sqlCompatMode);
+ addTestCase(testCases, id++, q2);
- for (String jk2 : allJoinKinds) {
- queryJoinKinds[2] = jk2;
- TestQuery q3 = generateQuery(queryJoinKinds, 3, filterComputer, s);
- testCases.add(new Object[] { id++, q3.sqlQuery, q3.sqlppQuery, q3.summary });
+ for (String jk2 : allJoinKinds) {
+ queryJoinKinds[2] = jk2;
+ TestQuery q3 = generateQuery(queryJoinKinds, 3, filterComputer, s, sqlCompatMode);
+ addTestCase(testCases, id++, q3);
+ }
}
}
}
@@ -83,7 +92,7 @@
}
private static TestQuery generateQuery(String[] joinKinds, int joinKindsSize, IntUnaryOperator filterComputer,
- char shape) {
+ char shape, boolean sqlCompatMode) {
int tCount = joinKindsSize + 1;
List<String> tDefs = new ArrayList<>(tCount);
for (int i = 0; i < tCount; i++) {
@@ -175,7 +184,7 @@
summary.append(joinKinds[i - 1]);
}
String projectFieldExprSql = String.format(fieldExprFormat, tThis, PROJECT_FIELD);
- String projectFieldExprSqlpp = missing2Null(projectFieldExprSql);
+ String projectFieldExprSqlpp = sqlCompatMode ? projectFieldExprSql : missing2Null(projectFieldExprSql);
String projectFieldAlias = String.format("%s_%s", tThis, PROJECT_FIELD);
String projectFormat = "%s AS %s";
selectClauseSql.append(String.format(projectFormat, projectFieldExprSql, projectFieldAlias));
@@ -188,22 +197,32 @@
summary.append(';').append(shape);
}
+ if (sqlCompatMode) {
+ summary.append(";sql-compat");
+ }
+
String queryFormat = "SELECT %s %sORDER BY %s";
String sqlQuery = String.format(queryFormat, selectClauseSql, fromClauseSql, orderbyClauseSql);
String sqlppQuery = String.format(queryFormat, selectClauseSqlpp, fromClauseSqlpp, orderbyClauseSqlpp);
- return new TestQuery(sqlQuery, sqlppQuery, summary.toString());
+ return new TestQuery(sqlQuery, sqlppQuery, sqlCompatMode, summary.toString());
}
private static String missing2Null(String expr) {
return String.format("if_missing(%s, null)", expr);
}
- public SqlppRQGJoinsIT(int testcaseId, String sqlQuery, String sqlppQuery, String desc) {
+ private static void addTestCase(List<Object[]> testCases, int id, TestQuery q) {
+ testCases.add(new Object[] { id, q.summary, q.sqlQuery, q.sqlppQuery, q.sqlCompatMode });
+ }
+
+ public SqlppRQGJoinsIT(int testcaseId, String description, String sqlQuery, String sqlppQuery,
+ boolean sqlCompatMode) {
this.testcaseId = testcaseId;
+ this.description = description;
this.sqlQuery = sqlQuery;
this.sqlppQuery = sqlppQuery;
- this.desc = desc;
+ this.sqlCompatMode = sqlCompatMode;
}
@BeforeClass
@@ -218,17 +237,27 @@
@Test
public void test() throws Exception {
- runTestCase(testcaseId, desc, sqlQuery, sqlppQuery);
+ List<TestCase.CompilationUnit.Parameter> params = null;
+ if (sqlCompatMode) {
+ TestCase.CompilationUnit.Parameter sqlCompatModeParam = new TestCase.CompilationUnit.Parameter();
+ sqlCompatModeParam.setName(QueryServiceRequestParameters.Parameter.SQL_COMPAT.str());
+ sqlCompatModeParam.setType(ParameterTypeEnum.JSON);
+ sqlCompatModeParam.setValue(String.valueOf(sqlCompatMode));
+ params = Collections.singletonList(sqlCompatModeParam);
+ }
+ runTestCase(testcaseId, description, sqlQuery, sqlppQuery, params);
}
private static class TestQuery {
final String sqlQuery;
final String sqlppQuery;
final String summary;
+ boolean sqlCompatMode;
- TestQuery(String sqlQuery, String sqlppQuery, String summary) {
+ TestQuery(String sqlQuery, String sqlppQuery, boolean sqlCompatMode, String summary) {
this.sqlQuery = sqlQuery;
this.sqlppQuery = sqlppQuery;
+ this.sqlCompatMode = sqlCompatMode;
this.summary = summary;
}
}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppRQGTestBase.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppRQGTestBase.java
index 50f2b93..aa8783a 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppRQGTestBase.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppRQGTestBase.java
@@ -25,6 +25,7 @@
import java.io.InputStream;
import java.io.InputStreamReader;
import java.io.PrintWriter;
+import java.net.URI;
import java.nio.charset.StandardCharsets;
import java.nio.file.Files;
import java.nio.file.Path;
@@ -54,6 +55,7 @@
import org.apache.asterix.test.common.TestExecutor;
import org.apache.asterix.test.common.TestHelper;
import org.apache.asterix.testframework.context.TestCaseContext;
+import org.apache.asterix.testframework.xml.TestCase;
import org.apache.commons.io.FileUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.logging.log4j.LogManager;
@@ -141,8 +143,8 @@
return result;
}
- protected void runTestCase(int testcaseId, String testcaseDescription, String sqlQuery, String sqlppQuery)
- throws Exception {
+ protected void runTestCase(int testcaseId, String testcaseDescription, String sqlQuery, String sqlppQuery,
+ List<TestCase.CompilationUnit.Parameter> params) throws Exception {
LOGGER.info(String.format("Starting testcase #%d: %s", testcaseId, testcaseDescription));
LOGGER.info("Running SQL");
@@ -156,8 +158,9 @@
LOGGER.info("Running SQL++");
LOGGER.info(sqlppQuery);
ArrayNode sqlppResult;
- try (InputStream resultStream = testExecutor.executeQueryService(sqlppQuery,
- testExecutor.getEndpoint(Servlets.QUERY_SERVICE), TestCaseContext.OutputFormat.ADM)) {
+ URI endpoint = testExecutor.getEndpoint(Servlets.QUERY_SERVICE);
+ try (InputStream resultStream = testExecutor.executeQueryService(sqlppQuery, TestCaseContext.OutputFormat.ADM,
+ endpoint, params == null ? Collections.emptyList() : params, false, StandardCharsets.UTF_8)) {
sqlppResult = asJson(
ResultExtractor.extract(resultStream, StandardCharsets.UTF_8, TestCaseContext.OutputFormat.ADM));
}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.1.ddl.sqlpp
new file mode 100644
index 0000000..909b74c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.1.ddl.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/* Test that outer join emits NULLs for non-matching values */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create dataset ds1(id int) open type primary key id;
+
+create dataset ds2(id int) open type primary key id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.2.update.sqlpp
new file mode 100644
index 0000000..65e9d54
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.2.update.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into ds1 ({"id": 1, "a": 1, "b": 1 });
+insert into ds1 ({"id": 2, "a": 1, "b": 2 });
+insert into ds1 ({"id": 3, "a": 2, "b": 3 });
+insert into ds1 ({"id": 4, "a": 2, "b": 4 });
+insert into ds1 ({"id": 5, "a": 3, "b": 5 });
+insert into ds1 ({"id": 6, "a": 3, "b": 6 });
+insert into ds1 ({"id": 7, "a": 4, "b": 7 });
+
+insert into ds2 ({"id": 100, "x": 2, "y": 100 });
+insert into ds2 ({"id": 101, "x": 2, "y": 101 });
+insert into ds2 ({"id": 102, "x": 3, "y": 102 });
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.3.query.sqlpp
new file mode 100644
index 0000000..c2edfe6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.3.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+/* Left outer join / Hash */
+
+use test;
+
+select ds1.b, ds2.y
+from ds1 left outer join ds2 on ds1.a = ds2.x
+order by ds1.b, ds2.y;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.4.query.sqlpp
new file mode 100644
index 0000000..3f85f0c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.4.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+/* Right outer join / Hash */
+
+use test;
+
+select ds1.b, ds2.y
+from ds2 right outer join ds1 on ds1.a = ds2.x
+order by ds1.b, ds2.y;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.5.query.sqlpp
new file mode 100644
index 0000000..6ae5a89
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.5.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+/* Left outer join / NL */
+
+use test;
+
+select ds1.b, ds2.y
+from ds1 left outer join ds2 on (ds1.a - ds2.x >= 0 and ds2.x - ds1.a >= 0)
+order by ds1.b, ds2.y;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.6.query.sqlpp
new file mode 100644
index 0000000..e4a5ef5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.6.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+/* Right outer join / NL */
+
+use test;
+
+select ds1.b, ds2.y
+from ds2 right outer join ds1 on (ds1.a - ds2.x >= 0 and ds2.x - ds1.a >= 0)
+order by ds1.b, ds2.y;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.7.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.7.ddl.sqlpp
new file mode 100644
index 0000000..a58785a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.7.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+create index idx_x on ds2(x:int);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.8.query.sqlpp
new file mode 100644
index 0000000..279ed0f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.8.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+/* Left outer join / Index-NL */
+
+use test;
+
+select ds1.b, ds2.y
+from ds1 left outer join ds2 on bigint(ds1.a) /* +indexnl */ = ds2.x
+order by ds1.b, ds2.y;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.9.query.sqlpp
new file mode 100644
index 0000000..e6b0c54
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.9.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+/* Right outer join / Index-NL */
+
+use test;
+
+select ds1.b, ds2.y
+from ds2 right outer join ds1 on bigint(ds1.a) /* +indexnl */ = ds2.x
+order by ds1.b, ds2.y;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_unnest_01/outer_unnest_01.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_unnest_01/outer_unnest_01.1.ddl.sqlpp
new file mode 100644
index 0000000..5f5c3fe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_unnest_01/outer_unnest_01.1.ddl.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/* Test that outer unnest emits NULLs if there are no items in the input list */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create dataset ds1(id int) open type primary key id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_unnest_01/outer_unnest_01.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_unnest_01/outer_unnest_01.2.update.sqlpp
new file mode 100644
index 0000000..1fb895a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_unnest_01/outer_unnest_01.2.update.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into ds1 ({"id": 1, "a": [1] });
+insert into ds1 ({"id": 2, "a": [1, 2] });
+insert into ds1 ({"id": 3, "a": [] });
+insert into ds1 ({"id": 4, "a": null });
+insert into ds1 ({"id": 5 /*a:missing*/ });
+insert into ds1 ({"id": 6, "a": [1,2,3] });
+insert into ds1 ({"id": 7, "a": [1, missing, null, 4] });
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_unnest_01/outer_unnest_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_unnest_01/outer_unnest_01.3.query.sqlpp
new file mode 100644
index 0000000..ec5c8af
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_unnest_01/outer_unnest_01.3.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+/* Test that outer unnest emits NULLs if there are no items in the input list */
+
+use test;
+
+select ds1.id, ai
+from ds1 left outer unnest ds1.a as ai
+order by ds1.id, ai;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_unnest_01/outer_unnest_01.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_unnest_01/outer_unnest_01.4.query.sqlpp
new file mode 100644
index 0000000..02b34a0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_unnest_01/outer_unnest_01.4.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+/* Test that outer unnest emits NULLs for positional variable as well */
+
+use test;
+
+select ds1.id, ai, pi
+from ds1 left outer unnest ds1.a as ai at pi
+order by ds1.id, ai;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.3.adm
new file mode 100644
index 0000000..e4704d3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.3.adm
@@ -0,0 +1,9 @@
+{ "b": 1, "y": null }
+{ "b": 2, "y": null }
+{ "b": 3, "y": 100 }
+{ "b": 3, "y": 101 }
+{ "b": 4, "y": 100 }
+{ "b": 4, "y": 101 }
+{ "b": 5, "y": 102 }
+{ "b": 6, "y": 102 }
+{ "b": 7, "y": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.4.adm
new file mode 100644
index 0000000..e4704d3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.4.adm
@@ -0,0 +1,9 @@
+{ "b": 1, "y": null }
+{ "b": 2, "y": null }
+{ "b": 3, "y": 100 }
+{ "b": 3, "y": 101 }
+{ "b": 4, "y": 100 }
+{ "b": 4, "y": 101 }
+{ "b": 5, "y": 102 }
+{ "b": 6, "y": 102 }
+{ "b": 7, "y": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.5.adm
new file mode 100644
index 0000000..e4704d3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.5.adm
@@ -0,0 +1,9 @@
+{ "b": 1, "y": null }
+{ "b": 2, "y": null }
+{ "b": 3, "y": 100 }
+{ "b": 3, "y": 101 }
+{ "b": 4, "y": 100 }
+{ "b": 4, "y": 101 }
+{ "b": 5, "y": 102 }
+{ "b": 6, "y": 102 }
+{ "b": 7, "y": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.6.adm
new file mode 100644
index 0000000..e4704d3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.6.adm
@@ -0,0 +1,9 @@
+{ "b": 1, "y": null }
+{ "b": 2, "y": null }
+{ "b": 3, "y": 100 }
+{ "b": 3, "y": 101 }
+{ "b": 4, "y": 100 }
+{ "b": 4, "y": 101 }
+{ "b": 5, "y": 102 }
+{ "b": 6, "y": 102 }
+{ "b": 7, "y": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.8.adm
new file mode 100644
index 0000000..e4704d3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.8.adm
@@ -0,0 +1,9 @@
+{ "b": 1, "y": null }
+{ "b": 2, "y": null }
+{ "b": 3, "y": 100 }
+{ "b": 3, "y": 101 }
+{ "b": 4, "y": 100 }
+{ "b": 4, "y": 101 }
+{ "b": 5, "y": 102 }
+{ "b": 6, "y": 102 }
+{ "b": 7, "y": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.9.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.9.adm
new file mode 100644
index 0000000..e4704d3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.9.adm
@@ -0,0 +1,9 @@
+{ "b": 1, "y": null }
+{ "b": 2, "y": null }
+{ "b": 3, "y": 100 }
+{ "b": 3, "y": 101 }
+{ "b": 4, "y": 100 }
+{ "b": 4, "y": 101 }
+{ "b": 5, "y": 102 }
+{ "b": 6, "y": 102 }
+{ "b": 7, "y": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_unnest_01/outer_unnest_01.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_unnest_01/outer_unnest_01.3.adm
new file mode 100644
index 0000000..5e02a06
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_unnest_01/outer_unnest_01.3.adm
@@ -0,0 +1,13 @@
+{ "id": 1, "ai": 1 }
+{ "id": 2, "ai": 1 }
+{ "id": 2, "ai": 2 }
+{ "id": 3, "ai": null }
+{ "id": 4, "ai": null }
+{ "id": 5, "ai": null }
+{ "id": 6, "ai": 1 }
+{ "id": 6, "ai": 2 }
+{ "id": 6, "ai": 3 }
+{ "id": 7 }
+{ "id": 7, "ai": null }
+{ "id": 7, "ai": 1 }
+{ "id": 7, "ai": 4 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_unnest_01/outer_unnest_01.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_unnest_01/outer_unnest_01.4.adm
new file mode 100644
index 0000000..d0a8bd8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_unnest_01/outer_unnest_01.4.adm
@@ -0,0 +1,13 @@
+{ "id": 1, "pi": 1, "ai": 1 }
+{ "id": 2, "pi": 1, "ai": 1 }
+{ "id": 2, "pi": 2, "ai": 2 }
+{ "id": 3, "pi": null, "ai": null }
+{ "id": 4, "pi": null, "ai": null }
+{ "id": 5, "pi": null, "ai": null }
+{ "id": 6, "pi": 1, "ai": 1 }
+{ "id": 6, "pi": 2, "ai": 2 }
+{ "id": 6, "pi": 3, "ai": 3 }
+{ "id": 7, "pi": 2 }
+{ "id": 7, "pi": 3, "ai": null }
+{ "id": 7, "pi": 1, "ai": 1 }
+{ "id": 7, "pi": 4, "ai": 4 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
index cc0f92a..4cafa0e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -10164,6 +10164,16 @@
</compilation-unit>
</test-case>
<test-case FilePath="sql-compat">
+ <compilation-unit name="outer_join_01">
+ <output-dir compare="Text">outer_join_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="sql-compat">
+ <compilation-unit name="outer_unnest_01">
+ <output-dir compare="Text">outer_unnest_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="sql-compat">
<compilation-unit name="select_star_01">
<output-dir compare="Text">select_star_01</output-dir>
</compilation-unit>
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalBTreeSearchOperatorDescriptor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalBTreeSearchOperatorDescriptor.java
index 5ecd283..56bedea 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalBTreeSearchOperatorDescriptor.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalBTreeSearchOperatorDescriptor.java
@@ -40,7 +40,7 @@
int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, int version) {
super(spec, outRecDesc, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory,
retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, minFilterFieldIndexes,
- maxFilterFieldIndexes, false);
+ maxFilterFieldIndexes, false, null);
this.version = version;
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalBTreeSearchOperatorNodePushable.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalBTreeSearchOperatorNodePushable.java
index 80ebb4c..4e669c3 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalBTreeSearchOperatorNodePushable.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalBTreeSearchOperatorNodePushable.java
@@ -46,7 +46,8 @@
boolean retainMissing, IMissingWriterFactory missingWriterFactory,
ISearchOperationCallbackFactory searchCallbackFactory, int version) throws HyracksDataException {
super(ctx, partition, intputRecDesc, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive, null, null,
- indexHelperFactory, retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, false);
+ indexHelperFactory, retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, false,
+ null);
this.version = version;
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalRTreeSearchOperatorDescriptor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalRTreeSearchOperatorDescriptor.java
index 493d3ba..bd6c922 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalRTreeSearchOperatorDescriptor.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalRTreeSearchOperatorDescriptor.java
@@ -41,7 +41,7 @@
int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, int version) {
super(spec, outRecDesc, keyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory, retainInput,
retainMissing, missingWriterFactory, searchCallbackFactory, minFilterFieldIndexes,
- maxFilterFieldIndexes, false);
+ maxFilterFieldIndexes, false, null);
this.version = version;
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalRTreeSearchOperatorNodePushable.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalRTreeSearchOperatorNodePushable.java
index 0b294de..508cff7 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalRTreeSearchOperatorNodePushable.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalRTreeSearchOperatorNodePushable.java
@@ -46,7 +46,7 @@
IMissingWriterFactory missingWriterFactory, ISearchOperationCallbackFactory searchCallbackFactory,
int version) throws HyracksDataException {
super(ctx, partition, inputRecDesc, keyFields, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
- retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, false);
+ retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, false, null);
this.version = version;
}
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/JoinClause.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/JoinClause.java
index 18b7a24..be3ebda 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/JoinClause.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/JoinClause.java
@@ -19,8 +19,11 @@
package org.apache.asterix.lang.sqlpp.clause;
+import java.util.Objects;
+
import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.lang.common.base.Expression;
+import org.apache.asterix.lang.common.base.Literal;
import org.apache.asterix.lang.common.expression.VariableExpr;
import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
import org.apache.asterix.lang.sqlpp.optype.JoinType;
@@ -30,10 +33,21 @@
private final JoinType joinType;
+ private Literal.Type outerJoinMissingValueType;
+
public JoinClause(JoinType joinType, Expression rightExpr, VariableExpr rightVar, VariableExpr rightPosVar,
- Expression conditionExpr) {
+ Expression conditionExpr, Literal.Type outerJoinMissingValueType) {
super(rightExpr, rightVar, rightPosVar, conditionExpr);
this.joinType = joinType;
+ setOuterJoinMissingValueType(outerJoinMissingValueType);
+ }
+
+ public Literal.Type getOuterJoinMissingValueType() {
+ return outerJoinMissingValueType;
+ }
+
+ public void setOuterJoinMissingValueType(Literal.Type outerJoinMissingValueType) {
+ this.outerJoinMissingValueType = validateMissingValueType(joinType, outerJoinMissingValueType);
}
@Override
@@ -52,7 +66,8 @@
@Override
public int hashCode() {
- return 31 * super.hashCode() + joinType.hashCode();
+ return 31 * super.hashCode() + 31 * joinType.hashCode()
+ + (outerJoinMissingValueType != null ? outerJoinMissingValueType.hashCode() : 0);
}
@Override
@@ -64,6 +79,28 @@
return false;
}
JoinClause target = (JoinClause) object;
- return super.equals(target) && joinType.equals(target.getJoinType());
+ return super.equals(target) && joinType.equals(target.getJoinType())
+ && Objects.equals(outerJoinMissingValueType, target.outerJoinMissingValueType);
+ }
+
+ private static Literal.Type validateMissingValueType(JoinType joinType, Literal.Type missingValueType) {
+ switch (joinType) {
+ case INNER:
+ if (missingValueType != null) {
+ throw new IllegalArgumentException(String.valueOf(missingValueType));
+ }
+ return null;
+ case LEFTOUTER:
+ case RIGHTOUTER:
+ switch (Objects.requireNonNull(missingValueType)) {
+ case MISSING:
+ case NULL:
+ return missingValueType;
+ default:
+ throw new IllegalArgumentException(String.valueOf(missingValueType));
+ }
+ default:
+ throw new IllegalStateException(String.valueOf(joinType));
+ }
}
}
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/UnnestClause.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/UnnestClause.java
index f035be4..054ff26 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/UnnestClause.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/UnnestClause.java
@@ -19,8 +19,11 @@
package org.apache.asterix.lang.sqlpp.clause;
+import java.util.Objects;
+
import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.lang.common.base.Expression;
+import org.apache.asterix.lang.common.base.Literal;
import org.apache.asterix.lang.common.expression.VariableExpr;
import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
import org.apache.asterix.lang.sqlpp.optype.UnnestType;
@@ -30,9 +33,21 @@
private final UnnestType unnestType;
- public UnnestClause(UnnestType unnestType, Expression rightExpr, VariableExpr rightVar, VariableExpr rightPosVar) {
+ private Literal.Type outerUnnestMissingValueType;
+
+ public UnnestClause(UnnestType unnestType, Expression rightExpr, VariableExpr rightVar, VariableExpr rightPosVar,
+ Literal.Type outerUnnestMissingValueType) {
super(rightExpr, rightVar, rightPosVar);
this.unnestType = unnestType;
+ setOuterUnnestMissingValueType(outerUnnestMissingValueType);
+ }
+
+ public Literal.Type getOuterUnnestMissingValueType() {
+ return outerUnnestMissingValueType;
+ }
+
+ public void setOuterUnnestMissingValueType(Literal.Type outerUnnestMissingValueType) {
+ this.outerUnnestMissingValueType = validateMissingValueType(unnestType, outerUnnestMissingValueType);
}
@Override
@@ -51,7 +66,8 @@
@Override
public int hashCode() {
- return 31 * super.hashCode() + unnestType.hashCode();
+ return 31 * super.hashCode() + 31 * unnestType.hashCode()
+ + +(outerUnnestMissingValueType != null ? outerUnnestMissingValueType.hashCode() : 0);
}
@Override
@@ -63,6 +79,27 @@
return false;
}
UnnestClause target = (UnnestClause) object;
- return super.equals(target) && unnestType.equals(target.getUnnestType());
+ return super.equals(target) && unnestType.equals(target.getUnnestType())
+ && Objects.equals(outerUnnestMissingValueType, target.outerUnnestMissingValueType);
+ }
+
+ private static Literal.Type validateMissingValueType(UnnestType unnestType, Literal.Type missingValueType) {
+ switch (unnestType) {
+ case INNER:
+ if (missingValueType != null) {
+ throw new IllegalArgumentException(String.valueOf(missingValueType));
+ }
+ return null;
+ case LEFTOUTER:
+ switch (Objects.requireNonNull(missingValueType)) {
+ case MISSING:
+ case NULL:
+ return missingValueType;
+ default:
+ throw new IllegalArgumentException(String.valueOf(missingValueType));
+ }
+ default:
+ throw new IllegalStateException(String.valueOf(unnestType));
+ }
}
}
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlCompatRewriteVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlCompatRewriteVisitor.java
index 1b44377..a0939c3 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlCompatRewriteVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlCompatRewriteVisitor.java
@@ -28,6 +28,7 @@
import org.apache.asterix.lang.common.base.AbstractClause;
import org.apache.asterix.lang.common.base.Expression;
import org.apache.asterix.lang.common.base.ILangExpression;
+import org.apache.asterix.lang.common.base.Literal;
import org.apache.asterix.lang.common.clause.LetClause;
import org.apache.asterix.lang.common.clause.LimitClause;
import org.apache.asterix.lang.common.expression.CallExpr;
@@ -114,6 +115,17 @@
if (expr.getKind() == Expression.Kind.SELECT_EXPRESSION) {
annotateSubqueryNoCoercion((SelectExpression) expr);
}
+ switch (joinClause.getJoinType()) {
+ case LEFTOUTER:
+ case RIGHTOUTER:
+ joinClause.setOuterJoinMissingValueType(Literal.Type.NULL);
+ break;
+ case INNER:
+ break;
+ default:
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, joinClause.getSourceLocation(),
+ String.valueOf(joinClause.getJoinType()));
+ }
return super.visit(joinClause, arg);
}
@@ -123,6 +135,17 @@
if (expr.getKind() == Expression.Kind.SELECT_EXPRESSION) {
annotateSubqueryNoCoercion((SelectExpression) expr);
}
+ // keep UNNEST clause aligned with JOIN clause when it comes to producing NULL values
+ switch (unnestClause.getUnnestType()) {
+ case LEFTOUTER:
+ unnestClause.setOuterUnnestMissingValueType(Literal.Type.NULL);
+ break;
+ case INNER:
+ break;
+ default:
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, unnestClause.getSourceLocation(),
+ String.valueOf(unnestClause.getUnnestType()));
+ }
return super.visit(unnestClause, arg);
}
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppRightJoinRewriteVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppRightJoinRewriteVisitor.java
index 8a3b913..26c6915 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppRightJoinRewriteVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppRightJoinRewriteVisitor.java
@@ -32,6 +32,7 @@
import org.apache.asterix.lang.common.base.Clause;
import org.apache.asterix.lang.common.base.Expression;
import org.apache.asterix.lang.common.base.ILangExpression;
+import org.apache.asterix.lang.common.base.Literal;
import org.apache.asterix.lang.common.clause.LetClause;
import org.apache.asterix.lang.common.expression.VariableExpr;
import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
@@ -161,9 +162,10 @@
VariableExpr rightVar = joinClause.getRightVariable();
VariableExpr rightPosVar = joinClause.getPositionalVariable();
Expression condExpr = joinClause.getConditionExpression();
+ Literal.Type outerMissingValueType = joinClause.getOuterJoinMissingValueType();
if (i == 0) {
- JoinClause newJoinClause =
- new JoinClause(JoinType.LEFTOUTER, fromExpr, fromVar, fromPosVar, condExpr);
+ JoinClause newJoinClause = new JoinClause(JoinType.LEFTOUTER, fromExpr, fromVar, fromPosVar,
+ condExpr, outerMissingValueType);
newJoinClause.setSourceLocation(joinClauseSourceLoc);
fromExpr = rightExpr;
@@ -249,7 +251,7 @@
? newVariableExpr(newPrecedingClauseRightPosVar,
precedingClauseRightPosVarSourceLoc)
: null,
- newCondExpr);
+ newCondExpr, joinPrecedingClause.getOuterJoinMissingValueType());
newPrecedingClause.setSourceLocation(precedingClauseSourceLoc);
break;
case UNNEST_CLAUSE:
@@ -260,10 +262,10 @@
newPrecedingClause = new UnnestClause(unnestPrecedingClause.getUnnestType(),
newRightExpr,
newVariableExpr(newPrecedingClauseRightVar, precedingClauseRightVarSourceLoc),
- newPrecedingClauseRightPosVar != null
- ? newVariableExpr(newPrecedingClauseRightPosVar,
- precedingClauseRightPosVarSourceLoc)
- : null);
+ newPrecedingClauseRightPosVar != null ? newVariableExpr(
+ newPrecedingClauseRightPosVar, precedingClauseRightPosVarSourceLoc)
+ : null,
+ unnestPrecedingClause.getOuterUnnestMissingValueType());
newPrecedingClause.setSourceLocation(precedingClauseSourceLoc);
break;
default:
@@ -291,8 +293,8 @@
Expression newCondExpr = SqlppRewriteUtil.substituteExpression(
(Expression) SqlppRewriteUtil.deepCopy(condExpr), substMapOuterFinal, context);
- JoinClause newJoinClause =
- new JoinClause(JoinType.LEFTOUTER, newRightExpr, newRightVarExpr, null, newCondExpr);
+ JoinClause newJoinClause = new JoinClause(JoinType.LEFTOUTER, newRightExpr, newRightVarExpr, null,
+ newCondExpr, outerMissingValueType);
newJoinClause.setSourceLocation(joinClauseSourceLoc);
fromExpr = rightExpr;
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/DeepCopyVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/DeepCopyVisitor.java
index 283b416..d74d047 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/DeepCopyVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/DeepCopyVisitor.java
@@ -112,7 +112,7 @@
: (VariableExpr) joinClause.getPositionalVariable().accept(this, arg);
Expression conditionExpresion = (Expression) joinClause.getConditionExpression().accept(this, arg);
JoinClause copy = new JoinClause(joinClause.getJoinType(), rightExpression, rightVar, rightPositionVar,
- conditionExpresion);
+ conditionExpresion, joinClause.getOuterJoinMissingValueType());
copy.setSourceLocation(joinClause.getSourceLocation());
return copy;
}
@@ -136,7 +136,8 @@
VariableExpr rightVar = (VariableExpr) unnestClause.getRightVariable().accept(this, arg);
VariableExpr rightPositionVar = unnestClause.getPositionalVariable() == null ? null
: (VariableExpr) unnestClause.getPositionalVariable().accept(this, arg);
- UnnestClause copy = new UnnestClause(unnestClause.getUnnestType(), rightExpression, rightVar, rightPositionVar);
+ UnnestClause copy = new UnnestClause(unnestClause.getUnnestType(), rightExpression, rightVar, rightPositionVar,
+ unnestClause.getOuterUnnestMissingValueType());
copy.setSourceLocation(unnestClause.getSourceLocation());
return copy;
}
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppAstPrintVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppAstPrintVisitor.java
index db663cd..c803d83 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppAstPrintVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppAstPrintVisitor.java
@@ -28,6 +28,7 @@
import org.apache.asterix.common.metadata.DatasetFullyQualifiedName;
import org.apache.asterix.lang.common.base.AbstractClause;
import org.apache.asterix.lang.common.base.Expression;
+import org.apache.asterix.lang.common.base.Literal;
import org.apache.asterix.lang.common.clause.GroupbyClause;
import org.apache.asterix.lang.common.clause.LetClause;
import org.apache.asterix.lang.common.clause.OrderbyClause;
@@ -109,6 +110,7 @@
out.print(" AT ");
joinClause.getPositionalVariable().accept(this, 0);
}
+ out.print(Literal.Type.NULL.equals(joinClause.getOuterJoinMissingValueType()) ? "(OR NULL) " : "");
out.println(skip(step + 1) + "ON");
joinClause.getConditionExpression().accept(this, step + 1);
return null;
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppCloneAndSubstituteVariablesVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppCloneAndSubstituteVariablesVisitor.java
index e664d49..ce0d0a1 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppCloneAndSubstituteVariablesVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppCloneAndSubstituteVariablesVisitor.java
@@ -143,8 +143,8 @@
// The condition can refer to the newRightVar and newRightPosVar.
Expression conditionExpr = (Expression) joinClause.getConditionExpression().accept(this, currentEnv).first;
- JoinClause newJoinClause =
- new JoinClause(joinClause.getJoinType(), newRightExpr, newRightVar, newRightPosVar, conditionExpr);
+ JoinClause newJoinClause = new JoinClause(joinClause.getJoinType(), newRightExpr, newRightVar, newRightPosVar,
+ conditionExpr, joinClause.getOuterJoinMissingValueType());
newJoinClause.setSourceLocation(joinClause.getSourceLocation());
return new Pair<>(newJoinClause, currentEnv);
}
@@ -193,8 +193,8 @@
currentEnv.removeSubstitution(newRightPosVar);
}
// The condition can refer to the newRightVar and newRightPosVar.
- UnnestClause newUnnestClause =
- new UnnestClause(unnestClause.getUnnestType(), rightExpr, newRightVar, newRightPosVar);
+ UnnestClause newUnnestClause = new UnnestClause(unnestClause.getUnnestType(), rightExpr, newRightVar,
+ newRightPosVar, unnestClause.getOuterUnnestMissingValueType());
newUnnestClause.setSourceLocation(unnestClause.getSourceLocation());
return new Pair<>(newUnnestClause, currentEnv);
}
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
index 395f63d..4e64944 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
+++ b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
@@ -4841,7 +4841,7 @@
<JOIN> { startToken = token; } rightInput = JoinClauseRightInput() <ON> conditionExpr = Expression()
{
JoinClause joinClause = new JoinClause(joinType, rightInput.first, rightInput.second, rightInput.third,
- conditionExpr);
+ conditionExpr, joinType == JoinType.INNER ? null : Literal.Type.MISSING);
return addSourceLocation(joinClause, startToken);
}
}
@@ -4856,7 +4856,7 @@
<JOIN> { startToken = token; } rightInput = JoinClauseRightInput()
{
JoinClause joinClause = new JoinClause(JoinType.INNER, rightInput.first, rightInput.second, rightInput.third,
- new LiteralExpr(TrueLiteral.INSTANCE));
+ new LiteralExpr(TrueLiteral.INSTANCE), null);
return addSourceLocation(joinClause, startToken);
}
}
@@ -4890,7 +4890,8 @@
if (rightVar == null) {
rightVar = ExpressionToVariableUtil.getGeneratedVariable(rightExpr, true);
}
- UnnestClause unnestClause = new UnnestClause(unnestType, rightExpr, rightVar, posVar);
+ UnnestClause unnestClause = new UnnestClause(unnestType, rightExpr, rightVar, posVar,
+ unnestType == UnnestType.INNER ? null : Literal.Type.MISSING);
return addSourceLocation(unnestClause, startToken);
}
}
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 06f4f73..1d82b19 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
@@ -132,10 +132,10 @@
int[] minFilterFieldIndexes = createFilterIndexes(minFilterVars, opSchema);
int[] maxFilterFieldIndexes = createFilterIndexes(maxFilterVars, opSchema);
- return metadataProvider.buildBtreeRuntime(jobSpec, opSchema, typeEnv, context, true, false,
+ return metadataProvider.buildBtreeRuntime(jobSpec, opSchema, typeEnv, context, true, false, null,
((DatasetDataSource) dataSource).getDataset(), primaryIndex.getIndexName(), null, null, true,
- true, false, minFilterFieldIndexes, maxFilterFieldIndexes, tupleFilterFactory, outputLimit,
- false, false);
+ true, false, null, minFilterFieldIndexes, maxFilterFieldIndexes, tupleFilterFactory,
+ outputLimit, false, false);
default:
throw new AlgebricksException("Unknown datasource type");
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
index 011d862..5ce97eb 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
@@ -540,8 +540,9 @@
public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildBtreeRuntime(JobSpecification jobSpec,
IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv, JobGenContext context, boolean retainInput,
- boolean retainMissing, Dataset dataset, String indexName, int[] lowKeyFields, int[] highKeyFields,
- boolean lowKeyInclusive, boolean highKeyInclusive, boolean propagateFilter, int[] minFilterFieldIndexes,
+ boolean retainMissing, IMissingWriterFactory nonMatchWriterFactory, Dataset dataset, String indexName,
+ int[] lowKeyFields, int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
+ boolean propagateFilter, IMissingWriterFactory nonFilterWriterFactory, int[] minFilterFieldIndexes,
int[] maxFilterFieldIndexes, ITupleFilterFactory tupleFilterFactory, long outputLimit,
boolean isIndexOnlyPlan, boolean isPrimaryIndexPointSearch) throws AlgebricksException {
boolean isSecondary = true;
@@ -601,27 +602,28 @@
btreeSearchOp = !isSecondary && isPrimaryIndexPointSearch
? new LSMBTreeBatchPointSearchOperatorDescriptor(jobSpec, outputRecDesc, lowKeyFields,
highKeyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory, retainInput,
- retainMissing, context.getMissingWriterFactory(), searchCallbackFactory,
- minFilterFieldIndexes, maxFilterFieldIndexes, tupleFilterFactory, outputLimit)
+ retainMissing, nonMatchWriterFactory, searchCallbackFactory, minFilterFieldIndexes,
+ maxFilterFieldIndexes, tupleFilterFactory, outputLimit)
: new BTreeSearchOperatorDescriptor(jobSpec, outputRecDesc, lowKeyFields, highKeyFields,
lowKeyInclusive, highKeyInclusive, indexHelperFactory, retainInput, retainMissing,
- context.getMissingWriterFactory(), searchCallbackFactory, minFilterFieldIndexes,
- maxFilterFieldIndexes, propagateFilter, tupleFilterFactory, outputLimit,
+ nonMatchWriterFactory, searchCallbackFactory, minFilterFieldIndexes, maxFilterFieldIndexes,
+ propagateFilter, nonFilterWriterFactory, tupleFilterFactory, outputLimit,
proceedIndexOnlyPlan, failValueForIndexOnlyPlan, successValueForIndexOnlyPlan);
} else {
btreeSearchOp = new ExternalBTreeSearchOperatorDescriptor(jobSpec, outputRecDesc, lowKeyFields,
highKeyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory, retainInput, retainMissing,
- context.getMissingWriterFactory(), searchCallbackFactory, minFilterFieldIndexes,
- maxFilterFieldIndexes, ExternalDatasetsRegistry.INSTANCE.getAndLockDatasetVersion(dataset, this));
+ nonMatchWriterFactory, searchCallbackFactory, minFilterFieldIndexes, maxFilterFieldIndexes,
+ ExternalDatasetsRegistry.INSTANCE.getAndLockDatasetVersion(dataset, this));
}
return new Pair<>(btreeSearchOp, spPc.second);
}
public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildRtreeRuntime(JobSpecification jobSpec,
List<LogicalVariable> outputVars, IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv,
- JobGenContext context, boolean retainInput, boolean retainMissing, Dataset dataset, String indexName,
- int[] keyFields, boolean propagateFilter, int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes,
- boolean isIndexOnlyPlan) throws AlgebricksException {
+ JobGenContext context, boolean retainInput, boolean retainMissing,
+ IMissingWriterFactory nonMatchWriterFactory, Dataset dataset, String indexName, int[] keyFields,
+ boolean propagateFilter, IMissingWriterFactory nonFilterWriterFactory, int[] minFilterFieldIndexes,
+ int[] maxFilterFieldIndexes, boolean isIndexOnlyPlan) throws AlgebricksException {
int numPrimaryKeys = dataset.getPrimaryKeys().size();
Index secondaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
dataset.getDatasetName(), indexName);
@@ -667,13 +669,13 @@
new IndexDataflowHelperFactory(storageComponentProvider.getStorageManager(), spPc.first);
if (dataset.getDatasetType() == DatasetType.INTERNAL) {
rtreeSearchOp = new RTreeSearchOperatorDescriptor(jobSpec, outputRecDesc, keyFields, true, true,
- indexDataflowHelperFactory, retainInput, retainMissing, context.getMissingWriterFactory(),
+ indexDataflowHelperFactory, retainInput, retainMissing, nonMatchWriterFactory,
searchCallbackFactory, minFilterFieldIndexes, maxFilterFieldIndexes, propagateFilter,
- isIndexOnlyPlan, failValueForIndexOnlyPlan, successValueForIndexOnlyPlan);
+ nonFilterWriterFactory, isIndexOnlyPlan, failValueForIndexOnlyPlan, successValueForIndexOnlyPlan);
} else {
// Create the operator
rtreeSearchOp = new ExternalRTreeSearchOperatorDescriptor(jobSpec, outputRecDesc, keyFields, true, true,
- indexDataflowHelperFactory, retainInput, retainMissing, context.getMissingWriterFactory(),
+ indexDataflowHelperFactory, retainInput, retainMissing, nonMatchWriterFactory,
searchCallbackFactory, minFilterFieldIndexes, maxFilterFieldIndexes,
ExternalDatasetsRegistry.INSTANCE.getAndLockDatasetVersion(dataset, this));
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java
index 143dc4c..6529794 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java
@@ -19,7 +19,6 @@
package org.apache.asterix.metadata.utils;
import static org.apache.asterix.common.utils.IdentifierUtil.dataset;
-import static org.apache.asterix.common.utils.IdentifierUtil.dataverse;
import java.io.DataOutput;
import java.util.ArrayList;
@@ -417,7 +416,7 @@
metadataProvider.getStorageComponentProvider().getStorageManager(), primaryFileSplitProvider);
BTreeSearchOperatorDescriptor primarySearchOp = new BTreeSearchOperatorDescriptor(spec,
dataset.getPrimaryRecordDescriptor(metadataProvider), lowKeyFields, highKeyFields, true, true,
- indexHelperFactory, false, false, null, searchCallbackFactory, null, null, false);
+ indexHelperFactory, false, false, null, searchCallbackFactory, null, null, false, null);
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, primarySearchOp,
primaryPartitionConstraint);
return primarySearchOp;
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/MissableTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/MissableTypeComputer.java
index 91b5d19..b51ebb4 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/MissableTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/MissableTypeComputer.java
@@ -19,10 +19,11 @@
package org.apache.asterix.dataflow.data.common;
import org.apache.asterix.om.typecomputer.impl.TypeComputeUtils;
+import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.IAType;
import org.apache.asterix.om.types.TypeHelper;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.expressions.IMissableTypeComputer;
public class MissableTypeComputer implements IMissableTypeComputer {
@@ -33,9 +34,10 @@
}
@Override
- public IAType makeMissableType(Object type) throws AlgebricksException {
+ public IAType makeMissableType(Object type) {
IAType t = (IAType) type;
- return AUnionType.createMissableType(t);
+ return TypeHelper.canBeMissing(t) ? t
+ : t.getTypeTag() == ATypeTag.NULL ? BuiltinType.ANY : AUnionType.createMissableType(t);
}
@Override
@@ -45,8 +47,45 @@
}
@Override
+ public Object makeNullableType(Object type) {
+ IAType t = (IAType) type;
+ return TypeHelper.canBeNull(t) ? t
+ : t.getTypeTag() == ATypeTag.MISSING ? BuiltinType.ANY : AUnionType.createNullableType(t);
+ }
+
+ @Override
+ public boolean canBeNull(Object type) {
+ IAType t = (IAType) type;
+ return TypeHelper.canBeNull(t);
+ }
+
+ @Override
public Object getNonOptionalType(Object type) {
IAType t = (IAType) type;
return TypeComputeUtils.getActualType(t);
}
+
+ @Override
+ public Object getNonMissableType(Object type) {
+ IAType t = (IAType) type;
+ if (t.getTypeTag() == ATypeTag.UNION) {
+ AUnionType ut = ((AUnionType) t);
+ IAType primeType = ut.getActualType();
+ return ut.isNullableType() ? AUnionType.createNullableType(primeType) : primeType;
+ } else {
+ return t;
+ }
+ }
+
+ @Override
+ public Object getNonNullableType(Object type) {
+ IAType t = (IAType) type;
+ if (t.getTypeTag() == ATypeTag.UNION) {
+ AUnionType ut = ((AUnionType) t);
+ IAType primeType = ut.getActualType();
+ return ut.isMissableType() ? AUnionType.createMissableType(primeType) : primeType;
+ } else {
+ return t;
+ }
+ }
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/NullWriterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/NullWriterFactory.java
new file mode 100644
index 0000000..0a5ef58
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/NullWriterFactory.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.dataflow.data.nontagged;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.api.dataflow.value.IMissingWriter;
+import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public final class NullWriterFactory implements IMissingWriterFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ public static final NullWriterFactory INSTANCE = new NullWriterFactory();
+
+ private NullWriterFactory() {
+ }
+
+ @Override
+ public IMissingWriter createMissingWriter() {
+ return NullWriterFactory::writeNull;
+ }
+
+ private static void writeNull(DataOutput out) throws HyracksDataException {
+ try {
+ out.writeByte(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/base/IDataFormat.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/base/IDataFormat.java
index 27b7802..82af52e 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/base/IDataFormat.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/base/IDataFormat.java
@@ -69,6 +69,8 @@
public IMissingWriterFactory getMissingWriterFactory();
+ public IMissingWriterFactory getNullWriterFactory();
+
public IUnnestingPositionWriterFactory getUnnestingPositionWriterFactory();
public Triple<IScalarEvaluatorFactory, ScalarFunctionCallExpression, IAType> partitioningEvaluatorFactory(
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/constants/AsterixConstantValue.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/constants/AsterixConstantValue.java
index 337e966..8b5da27 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/constants/AsterixConstantValue.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/constants/AsterixConstantValue.java
@@ -22,6 +22,7 @@
import org.apache.asterix.om.base.AMissing;
import org.apache.asterix.om.base.ANull;
import org.apache.asterix.om.base.IAObject;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
public class AsterixConstantValue implements IAlgebricksConstantValue {
@@ -63,15 +64,27 @@
@Override
public boolean equals(Object o) {
- if (!(o instanceof AsterixConstantValue)) {
+ if (o instanceof AsterixConstantValue) {
+ AsterixConstantValue v2 = (AsterixConstantValue) o;
+ return object.deepEqual(v2.getObject());
+ } else if (o instanceof IAlgebricksConstantValue) {
+ return o.equals(this);
+ } else {
return false;
}
- AsterixConstantValue v2 = (AsterixConstantValue) o;
- return object.deepEqual(v2.getObject());
}
@Override
public int hashCode() {
- return object.hash();
+ switch (object.getType().getTypeTag()) {
+ case MISSING:
+ return ConstantExpression.MISSING.hashCode();
+ case NULL:
+ return ConstantExpression.NULL.hashCode();
+ case BOOLEAN:
+ return (isTrue() ? ConstantExpression.TRUE : ConstantExpression.FALSE).hashCode();
+ default:
+ return object.hash();
+ }
}
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
index 0a1a8e9..0c9f0dd 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
@@ -1047,6 +1047,8 @@
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "percent-rank-impl", FunctionIdentifier.VARARGS);
public static final FunctionIdentifier WIN_MARK_FIRST_MISSING_IMPL = new FunctionIdentifier(
FunctionConstants.ASTERIX_NS, "win-mark-first-missing-impl", FunctionIdentifier.VARARGS);
+ public static final FunctionIdentifier WIN_MARK_FIRST_NULL_IMPL = new FunctionIdentifier(
+ FunctionConstants.ASTERIX_NS, "win-mark-first-null-impl", FunctionIdentifier.VARARGS);
public static final FunctionIdentifier WIN_PARTITION_LENGTH_IMPL =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "win-partition-length-impl", 0);
@@ -2258,6 +2260,7 @@
addFunction(PERCENT_RANK, ADoubleTypeComputer.INSTANCE, false);
addFunction(PERCENT_RANK_IMPL, ADoubleTypeComputer.INSTANCE, false);
addPrivateFunction(WIN_MARK_FIRST_MISSING_IMPL, ABooleanTypeComputer.INSTANCE, false);
+ addPrivateFunction(WIN_MARK_FIRST_NULL_IMPL, ABooleanTypeComputer.INSTANCE, false);
addPrivateFunction(WIN_PARTITION_LENGTH_IMPL, AInt64TypeComputer.INSTANCE, false);
// Similarity functions
@@ -3230,6 +3233,7 @@
addWindowFunction(RATIO_TO_REPORT, RATIO_TO_REPORT_IMPL, HAS_LIST_ARG);
addWindowFunction(ROW_NUMBER, ROW_NUMBER_IMPL, NO_FRAME_CLAUSE);
addWindowFunction(null, WIN_MARK_FIRST_MISSING_IMPL, NO_FRAME_CLAUSE, INJECT_ORDER_ARGS);
+ addWindowFunction(null, WIN_MARK_FIRST_NULL_IMPL, NO_FRAME_CLAUSE, INJECT_ORDER_ARGS);
addWindowFunction(null, WIN_PARTITION_LENGTH_IMPL, NO_FRAME_CLAUSE, MATERIALIZE_PARTITION);
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/TypeHelper.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/TypeHelper.java
index b444053..645c12a 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/TypeHelper.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/TypeHelper.java
@@ -29,6 +29,7 @@
public static boolean canBeMissing(IAType t) {
switch (t.getTypeTag()) {
case MISSING:
+ case ANY:
return true;
case UNION:
return ((AUnionType) t).isMissableType();
@@ -37,6 +38,18 @@
}
}
+ public static boolean canBeNull(IAType t) {
+ switch (t.getTypeTag()) {
+ case NULL:
+ case ANY:
+ return true;
+ case UNION:
+ return ((AUnionType) t).isNullableType();
+ default:
+ return false;
+ }
+ }
+
public static boolean canBeUnknown(IAType t) {
switch (t.getTypeTag()) {
case MISSING:
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/formats/NonTaggedDataFormat.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/formats/NonTaggedDataFormat.java
index f2685f4..8c8b7e6 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/formats/NonTaggedDataFormat.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/formats/NonTaggedDataFormat.java
@@ -24,6 +24,7 @@
import org.apache.asterix.common.config.GlobalConfig;
import org.apache.asterix.dataflow.data.nontagged.MissingWriterFactory;
+import org.apache.asterix.dataflow.data.nontagged.NullWriterFactory;
import org.apache.asterix.formats.base.IDataFormat;
import org.apache.asterix.formats.nontagged.ADMPrinterFactoryProvider;
import org.apache.asterix.formats.nontagged.BinaryBooleanInspector;
@@ -352,6 +353,11 @@
}
@Override
+ public IMissingWriterFactory getNullWriterFactory() {
+ return NullWriterFactory.INSTANCE;
+ }
+
+ @Override
public IUnnestingPositionWriterFactory getUnnestingPositionWriterFactory() {
return UnnestingPositionWriterFactory.INSTANCE;
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
index f2d1f7a..35993cf 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
@@ -619,6 +619,7 @@
import org.apache.asterix.runtime.runningaggregates.std.RowNumberRunningAggregateDescriptor;
import org.apache.asterix.runtime.runningaggregates.std.TidRunningAggregateDescriptor;
import org.apache.asterix.runtime.runningaggregates.std.WinMarkFirstMissingRunningAggregateDescriptor;
+import org.apache.asterix.runtime.runningaggregates.std.WinMarkFirstNullRunningAggregateDescriptor;
import org.apache.asterix.runtime.runningaggregates.std.WinPartitionLenRunningAggregateDescriptor;
import org.apache.asterix.runtime.unnestingfunctions.std.RangeDescriptor;
import org.apache.asterix.runtime.unnestingfunctions.std.ScanCollectionDescriptor;
@@ -909,6 +910,7 @@
fc.add(RowNumberRunningAggregateDescriptor.FACTORY);
fc.add(PercentRankRunningAggregateDescriptor.FACTORY);
fc.add(WinMarkFirstMissingRunningAggregateDescriptor.FACTORY);
+ fc.add(WinMarkFirstNullRunningAggregateDescriptor.FACTORY);
fc.add(WinPartitionLenRunningAggregateDescriptor.FACTORY);
// boolean functions
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinMarkFirstMissingRunningAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinMarkFirstMissingRunningAggregateDescriptor.java
index 3f5f9f6..8a872c5 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinMarkFirstMissingRunningAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinMarkFirstMissingRunningAggregateDescriptor.java
@@ -21,6 +21,7 @@
import org.apache.asterix.om.functions.BuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.runtime.runningaggregates.base.AbstractRunningAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
@@ -60,7 +61,7 @@
for (int i = 0; i < args.length; i++) {
evals[i] = args[i].createScalarEvaluator(ctx);
}
- return new WinMarkFirstMissingRunningAggregateEvaluator(evals);
+ return new WinMarkFirstUnknownRunningAggregateEvaluator(ATypeTag.MISSING, evals);
}
};
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinMarkFirstNullRunningAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinMarkFirstNullRunningAggregateDescriptor.java
new file mode 100644
index 0000000..f8c2f3f
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinMarkFirstNullRunningAggregateDescriptor.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.runningaggregates.std;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.runningaggregates.base.AbstractRunningAggregateFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IRunningAggregateEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IRunningAggregateEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * This internal window function returns {@code TRUE} in the following two cases:
+ * <ol>
+ * <li>the argument is not NULL</li>
+ * <li>the argument is NULL and it comes from the first tuple in the current window partition</li>
+ * </ol>
+ * In all other cases the function returns {@code FALSE}.
+ * <p>
+ * The underlying assumption is that tuples in each window partition are sorted on the function's argument in the
+ * descending order.
+ */
+public class WinMarkFirstNullRunningAggregateDescriptor extends AbstractRunningAggregateFunctionDynamicDescriptor {
+
+ public static final IFunctionDescriptorFactory FACTORY = WinMarkFirstNullRunningAggregateDescriptor::new;
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IRunningAggregateEvaluatorFactory createRunningAggregateEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+ return new IRunningAggregateEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IRunningAggregateEvaluator createRunningAggregateEvaluator(IEvaluatorContext ctx)
+ throws HyracksDataException {
+ IScalarEvaluator[] evals = new IScalarEvaluator[args.length];
+ for (int i = 0; i < args.length; i++) {
+ evals[i] = args[i].createScalarEvaluator(ctx);
+ }
+ return new WinMarkFirstUnknownRunningAggregateEvaluator(ATypeTag.NULL, evals);
+ }
+ };
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.WIN_MARK_FIRST_NULL_IMPL;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinMarkFirstMissingRunningAggregateEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinMarkFirstUnknownRunningAggregateEvaluator.java
similarity index 84%
rename from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinMarkFirstMissingRunningAggregateEvaluator.java
rename to asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinMarkFirstUnknownRunningAggregateEvaluator.java
index c872921..93ba4ef 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinMarkFirstMissingRunningAggregateEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinMarkFirstUnknownRunningAggregateEvaluator.java
@@ -34,7 +34,7 @@
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
-public final class WinMarkFirstMissingRunningAggregateEvaluator implements IWindowAggregateEvaluator {
+public final class WinMarkFirstUnknownRunningAggregateEvaluator implements IWindowAggregateEvaluator {
@SuppressWarnings({ "rawtypes" })
private final ISerializerDeserializer boolSerde =
@@ -42,6 +42,8 @@
private final IScalarEvaluator[] argEvals;
+ private final byte unknownTypeTag;
+
private final TaggedValuePointable argValue;
private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -50,10 +52,11 @@
private boolean first;
- private boolean firstAllMissing;
+ private boolean firstAllUnknown;
- WinMarkFirstMissingRunningAggregateEvaluator(IScalarEvaluator[] argEvals) {
+ WinMarkFirstUnknownRunningAggregateEvaluator(ATypeTag unknownTypeTag, IScalarEvaluator[] argEvals) {
this.argEvals = argEvals;
+ this.unknownTypeTag = unknownTypeTag.serialize();
argValue = TaggedValuePointable.FACTORY.createPointable();
}
@@ -76,19 +79,19 @@
private boolean compute(IFrameTupleReference tuple) throws HyracksDataException {
if (first) {
- firstAllMissing = everyArgIsMissing(tuple);
+ firstAllUnknown = everyArgIsUnknown(tuple);
first = false;
return true;
} else {
- boolean thisAllMissing = firstAllMissing || everyArgIsMissing(tuple);
+ boolean thisAllMissing = firstAllUnknown || everyArgIsUnknown(tuple);
return !thisAllMissing;
}
}
- private boolean everyArgIsMissing(IFrameTupleReference tuple) throws HyracksDataException {
+ private boolean everyArgIsUnknown(IFrameTupleReference tuple) throws HyracksDataException {
for (IScalarEvaluator argEval : argEvals) {
argEval.evaluate(tuple, argValue);
- if (argValue.getTag() != ATypeTag.SERIALIZED_MISSING_TYPE_TAG) {
+ if (argValue.getTag() != unknownTypeTag) {
return false;
}
}
diff --git a/asterixdb/asterix-runtime/src/test/java/org/apache/asterix/runtime/functions/ScalarAggregateTypeComputerTest.java b/asterixdb/asterix-runtime/src/test/java/org/apache/asterix/runtime/functions/ScalarAggregateTypeComputerTest.java
index cbde36c..a18b04f 100644
--- a/asterixdb/asterix-runtime/src/test/java/org/apache/asterix/runtime/functions/ScalarAggregateTypeComputerTest.java
+++ b/asterixdb/asterix-runtime/src/test/java/org/apache/asterix/runtime/functions/ScalarAggregateTypeComputerTest.java
@@ -221,7 +221,8 @@
}
@Override
- public Object getVarType(LogicalVariable var, List<LogicalVariable> nonNullVariables,
+ public Object getVarType(LogicalVariable var, List<LogicalVariable> nonMissableVariables,
+ List<List<LogicalVariable>> correlatedMissableVariableLists, List<LogicalVariable> nonNullableVariables,
List<List<LogicalVariable>> correlatedNullableVariableLists) {
throw new IllegalStateException();
}
diff --git a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/AbstractCompilerFactoryBuilder.java b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/AbstractCompilerFactoryBuilder.java
index 3f03cc8..c22d54d 100644
--- a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/AbstractCompilerFactoryBuilder.java
+++ b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/AbstractCompilerFactoryBuilder.java
@@ -66,6 +66,7 @@
protected IConflictingTypeResolver conflictingTypeResolver;
protected IExpressionEvalSizeComputer expressionEvalSizeComputer;
protected IMissingWriterFactory missingWriterFactory;
+ protected IMissingWriterFactory nullWriterFactory;
protected IUnnestingPositionWriterFactory unnestingPositionWriterFactory;
protected INormalizedKeyComputerFactoryProvider normalizedKeyComputerFactoryProvider;
protected IPartialAggregationTypeComputer partialAggregationTypeComputer;
@@ -189,6 +190,14 @@
return missingWriterFactory;
}
+ public void setNullWriterFactory(IMissingWriterFactory nullWriterFactory) {
+ this.nullWriterFactory = nullWriterFactory;
+ }
+
+ public IMissingWriterFactory getNullWriterFactory() {
+ return nullWriterFactory;
+ }
+
public void setUnnestingPositionWriterFactory(IUnnestingPositionWriterFactory unnestingPositionWriterFactory) {
this.unnestingPositionWriterFactory = unnestingPositionWriterFactory;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java
index c6a79a5..891980f 100644
--- a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java
+++ b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java
@@ -101,7 +101,7 @@
JobGenContext context = new JobGenContext(null, metadata, appContext,
serializerDeserializerProvider, hashFunctionFactoryProvider, hashFunctionFamilyProvider,
comparatorFactoryProvider, typeTraitProvider, binaryBooleanInspectorFactory,
- binaryIntegerInspectorFactory, printerProvider, missingWriterFactory,
+ binaryIntegerInspectorFactory, printerProvider, missingWriterFactory, nullWriterFactory,
unnestingPositionWriterFactory, normalizedKeyComputerFactoryProvider,
expressionRuntimeProvider, expressionTypeComputer, oc, expressionEvalSizeComputer,
partialAggregationTypeComputer, predEvaluatorFactoryProvider,
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/ConstantExpression.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/ConstantExpression.java
index c49f62f..e2a68da 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/ConstantExpression.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/ConstantExpression.java
@@ -57,6 +57,16 @@
public String toString() {
return "TRUE";
}
+
+ @Override
+ public boolean equals(Object obj) {
+ return (obj instanceof IAlgebricksConstantValue) && ((IAlgebricksConstantValue) obj).isTrue();
+ }
+
+ @Override
+ public int hashCode() {
+ return Boolean.TRUE.hashCode();
+ }
});
public static final ConstantExpression FALSE = new ConstantExpression(new IAlgebricksConstantValue() {
@@ -84,6 +94,16 @@
public String toString() {
return "FALSE";
}
+
+ @Override
+ public boolean equals(Object obj) {
+ return (obj instanceof IAlgebricksConstantValue) && ((IAlgebricksConstantValue) obj).isFalse();
+ }
+
+ @Override
+ public int hashCode() {
+ return Boolean.FALSE.hashCode();
+ }
});
public static final ConstantExpression NULL = new ConstantExpression(new IAlgebricksConstantValue() {
@@ -111,6 +131,16 @@
public String toString() {
return "NULL";
}
+
+ @Override
+ public boolean equals(Object obj) {
+ return (obj instanceof IAlgebricksConstantValue) && ((IAlgebricksConstantValue) obj).isNull();
+ }
+
+ @Override
+ public int hashCode() {
+ return 0;
+ }
});
public static final ConstantExpression MISSING = new ConstantExpression(new IAlgebricksConstantValue() {
@@ -138,6 +168,16 @@
public String toString() {
return "MISSING";
}
+
+ @Override
+ public boolean equals(Object obj) {
+ return (obj instanceof IAlgebricksConstantValue) && ((IAlgebricksConstantValue) obj).isMissing();
+ }
+
+ @Override
+ public int hashCode() {
+ return 0;
+ }
});
public ConstantExpression(IAlgebricksConstantValue value) {
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/IMissableTypeComputer.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/IMissableTypeComputer.java
index 029f41a..c18e072 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/IMissableTypeComputer.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/IMissableTypeComputer.java
@@ -18,12 +18,18 @@
*/
package org.apache.hyracks.algebricks.core.algebra.expressions;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-
public interface IMissableTypeComputer {
- public Object makeMissableType(Object type) throws AlgebricksException;
+ Object makeMissableType(Object type);
- public boolean canBeMissing(Object type);
+ boolean canBeMissing(Object type);
- public Object getNonOptionalType(Object type);
+ Object makeNullableType(Object type);
+
+ boolean canBeNull(Object type);
+
+ Object getNonOptionalType(Object type);
+
+ Object getNonMissableType(Object type);
+
+ Object getNonNullableType(Object type);
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/IVariableTypeEnvironment.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/IVariableTypeEnvironment.java
index e024147..d160e54 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/IVariableTypeEnvironment.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/IVariableTypeEnvironment.java
@@ -27,7 +27,8 @@
public interface IVariableTypeEnvironment {
public Object getVarType(LogicalVariable var) throws AlgebricksException;
- public Object getVarType(LogicalVariable var, List<LogicalVariable> nonNullVariables,
+ public Object getVarType(LogicalVariable var, List<LogicalVariable> nonMissableVariables,
+ List<List<LogicalVariable>> correlatedMissableVariableLists, List<LogicalVariable> nonNullableVariables,
List<List<LogicalVariable>> correlatedNullableVariableLists) throws AlgebricksException;
public void setVarType(LogicalVariable var, Object type);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/EmptyTupleSourceOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/EmptyTupleSourceOperator.java
index 1af0ce4..1179dc4 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/EmptyTupleSourceOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/EmptyTupleSourceOperator.java
@@ -84,7 +84,9 @@
}
@Override
- public Object getVarType(LogicalVariable var, List<LogicalVariable> nonNullVariables,
+ public Object getVarType(LogicalVariable var, List<LogicalVariable> nonMissableVariables,
+ List<List<LogicalVariable>> correlatedMissableVariableLists,
+ List<LogicalVariable> nonNullableVariables,
List<List<LogicalVariable>> correlatedNullableVariableLists) throws AlgebricksException {
return null;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/LeftOuterJoinOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/LeftOuterJoinOperator.java
index 4e382d2..d709184 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/LeftOuterJoinOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/LeftOuterJoinOperator.java
@@ -27,24 +27,40 @@
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import org.apache.hyracks.algebricks.core.algebra.properties.LeftOuterTypePropagationPolicy;
import org.apache.hyracks.algebricks.core.algebra.properties.TypePropagationPolicy;
import org.apache.hyracks.algebricks.core.algebra.typing.ITypeEnvPointer;
import org.apache.hyracks.algebricks.core.algebra.typing.ITypingContext;
import org.apache.hyracks.algebricks.core.algebra.typing.OpRefTypeEnvPointer;
import org.apache.hyracks.algebricks.core.algebra.typing.PropagatingTypeEnvironment;
import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+import org.apache.hyracks.api.exceptions.ErrorCode;
public class LeftOuterJoinOperator extends AbstractBinaryJoinOperator {
- public LeftOuterJoinOperator(Mutable<ILogicalExpression> condition) {
+ private IAlgebricksConstantValue missingValue;
+
+ public LeftOuterJoinOperator(Mutable<ILogicalExpression> condition, IAlgebricksConstantValue missingValue) {
super(JoinKind.LEFT_OUTER, condition);
+ setMissingValue(missingValue);
}
public LeftOuterJoinOperator(Mutable<ILogicalExpression> condition, Mutable<ILogicalOperator> input1,
- Mutable<ILogicalOperator> input2) {
+ Mutable<ILogicalOperator> input2, IAlgebricksConstantValue missingValue) {
super(JoinKind.LEFT_OUTER, condition, input1, input2);
+ setMissingValue(missingValue);
+ }
+
+ public IAlgebricksConstantValue getMissingValue() {
+ return missingValue;
+ }
+
+ public void setMissingValue(IAlgebricksConstantValue value) {
+ this.missingValue = validateMissingValue(value);
}
@Override
@@ -64,13 +80,40 @@
for (int i = 0; i < n; i++) {
envPointers[i] = new OpRefTypeEnvPointer(inputs.get(i), ctx);
}
- PropagatingTypeEnvironment env =
- new PropagatingTypeEnvironment(ctx.getExpressionTypeComputer(), ctx.getMissableTypeComputer(),
- ctx.getMetadataProvider(), TypePropagationPolicy.LEFT_OUTER, envPointers);
+
+ TypePropagationPolicy typePropagationPolicy;
+ if (missingValue.isMissing()) {
+ typePropagationPolicy = LeftOuterTypePropagationPolicy.MISSABLE;
+ } else if (missingValue.isNull()) {
+ typePropagationPolicy = LeftOuterTypePropagationPolicy.NULLABLE;
+ } else {
+ throw new AlgebricksException(ErrorCode.ILLEGAL_STATE, String.valueOf(missingValue));
+ }
+ PropagatingTypeEnvironment env = new PropagatingTypeEnvironment(ctx.getExpressionTypeComputer(),
+ ctx.getMissableTypeComputer(), ctx.getMetadataProvider(), typePropagationPolicy, envPointers);
+
+ // live variables from right branch can be MISSING (or NULL) together
List<LogicalVariable> liveVars = new ArrayList<>();
- VariableUtilities.getLiveVariables(inputs.get(1).getValue(), liveVars); // live variables from right branch can be MISSING together
- env.getCorrelatedMissableVariableLists().add(liveVars);
+ VariableUtilities.getLiveVariables(inputs.get(1).getValue(), liveVars);
+ if (missingValue.isMissing()) {
+ env.getCorrelatedMissableVariableLists().add(liveVars);
+ } else if (missingValue.isNull()) {
+ env.getCorrelatedNullableVariableLists().add(liveVars);
+ } else {
+ throw new AlgebricksException(ErrorCode.ILLEGAL_STATE, String.valueOf(missingValue));
+ }
return env;
}
+ private static IAlgebricksConstantValue validateMissingValue(IAlgebricksConstantValue value) {
+ if (value == null) {
+ throw new NullPointerException();
+ } else if (value.isMissing()) {
+ return ConstantExpression.MISSING.getValue();
+ } else if (value.isNull()) {
+ return ConstantExpression.NULL.getValue();
+ } else {
+ throw new IllegalArgumentException(String.valueOf(value));
+ }
+ }
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/LeftOuterUnnestMapOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/LeftOuterUnnestMapOperator.java
index cd009c0..d91cf72 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/LeftOuterUnnestMapOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/LeftOuterUnnestMapOperator.java
@@ -25,10 +25,13 @@
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
import org.apache.hyracks.algebricks.core.algebra.typing.ITypingContext;
import org.apache.hyracks.algebricks.core.algebra.typing.PropagatingTypeEnvironment;
import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+import org.apache.hyracks.api.exceptions.ErrorCode;
/**
* Left-outer-unnest-map is similar to the unnest-map operator. The only
@@ -38,11 +41,21 @@
*/
public class LeftOuterUnnestMapOperator extends AbstractUnnestMapOperator {
+ private IAlgebricksConstantValue missingValue;
+
public LeftOuterUnnestMapOperator(List<LogicalVariable> variables, Mutable<ILogicalExpression> expression,
- List<Object> variableTypes, boolean propagateInput) {
- super(variables, expression, variableTypes, propagateInput);
+ List<Object> variableTypes, IAlgebricksConstantValue missingValue) {
// propagateInput is always set to true for this operator.
- this.propagateInput = true;
+ super(variables, expression, variableTypes, true);
+ setMissingValue(missingValue);
+ }
+
+ public IAlgebricksConstantValue getMissingValue() {
+ return missingValue;
+ }
+
+ public void setMissingValue(IAlgebricksConstantValue missingValue) {
+ this.missingValue = validateMissingValue(missingValue);
}
@Override
@@ -60,12 +73,32 @@
// Propagates all input variables that come from the outer branch.
PropagatingTypeEnvironment env = createPropagatingAllInputsTypeEnvironment(ctx);
- // The produced variables of the this operator are missable because of the left outer semantics.
+ // The produced variables of the this operator are missable (or nullable) because of the left outer semantics.
for (int i = 0; i < variables.size(); i++) {
- env.setVarType(variables.get(i), ctx.getMissableTypeComputer().makeMissableType(variableTypes.get(i)));
+ Object varType = variableTypes.get(i);
+ Object outVarType;
+ if (missingValue.isMissing()) {
+ outVarType = ctx.getMissableTypeComputer().makeMissableType(varType);
+ } else if (missingValue.isNull()) {
+ outVarType = ctx.getMissableTypeComputer().makeNullableType(varType);
+ } else {
+ throw new AlgebricksException(ErrorCode.ILLEGAL_STATE, getSourceLocation(), "");
+ }
+ env.setVarType(variables.get(i), outVarType);
}
return env;
}
+ private static IAlgebricksConstantValue validateMissingValue(IAlgebricksConstantValue value) {
+ if (value == null) {
+ throw new NullPointerException();
+ } else if (value.isMissing()) {
+ return ConstantExpression.MISSING.getValue();
+ } else if (value.isNull()) {
+ return ConstantExpression.NULL.getValue();
+ } else {
+ throw new IllegalArgumentException(String.valueOf(value));
+ }
+ }
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/LeftOuterUnnestOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/LeftOuterUnnestOperator.java
index 14996dd..1e493ad 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/LeftOuterUnnestOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/LeftOuterUnnestOperator.java
@@ -19,20 +19,37 @@
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IMissableTypeComputer;
import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
import org.apache.hyracks.algebricks.core.algebra.typing.ITypingContext;
import org.apache.hyracks.algebricks.core.algebra.typing.PropagatingTypeEnvironment;
import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+import org.apache.hyracks.api.exceptions.ErrorCode;
public class LeftOuterUnnestOperator extends AbstractUnnestNonMapOperator {
- public LeftOuterUnnestOperator(LogicalVariable variable, Mutable<ILogicalExpression> expression) {
+ private IAlgebricksConstantValue missingValue;
+
+ public LeftOuterUnnestOperator(LogicalVariable variable, Mutable<ILogicalExpression> expression,
+ IAlgebricksConstantValue missingValue) {
super(variable, expression);
+ setMissingValue(missingValue);
}
public LeftOuterUnnestOperator(LogicalVariable variable, Mutable<ILogicalExpression> expression,
- LogicalVariable positionalVariable, Object positionalVariableType) {
+ LogicalVariable positionalVariable, Object positionalVariableType, IAlgebricksConstantValue missingValue) {
super(variable, expression, positionalVariable, positionalVariableType);
+ setMissingValue(missingValue);
+ }
+
+ public IAlgebricksConstantValue getMissingValue() {
+ return missingValue;
+ }
+
+ public void setMissingValue(IAlgebricksConstantValue missingValue) {
+ this.missingValue = validateMissingValue(missingValue);
}
@Override
@@ -45,10 +62,26 @@
PropagatingTypeEnvironment env = createPropagatingAllInputsTypeEnvironment(ctx);
// The produced variables of the this operator are missable because of the left outer semantics.
+ Object outVarType, outPositionalVarType = null;
Object t = env.getType(expression.getValue());
- env.setVarType(variables.get(0), ctx.getMissableTypeComputer().makeMissableType(t));
+ IMissableTypeComputer tc = ctx.getMissableTypeComputer();
+ if (missingValue.isMissing()) {
+ outVarType = tc.makeMissableType(t);
+ if (positionalVariable != null) {
+ outPositionalVarType = tc.makeMissableType(positionalVariableType);
+ }
+ } else if (missingValue.isNull()) {
+ outVarType = tc.makeNullableType(t);
+ if (positionalVariable != null) {
+ outPositionalVarType = tc.makeNullableType(positionalVariableType);
+ }
+ } else {
+ throw new AlgebricksException(ErrorCode.ILLEGAL_STATE, getSourceLocation(), "");
+ }
+
+ env.setVarType(variables.get(0), outVarType);
if (positionalVariable != null) {
- env.setVarType(positionalVariable, ctx.getMissableTypeComputer().makeMissableType(positionalVariableType));
+ env.setVarType(positionalVariable, outPositionalVarType);
}
return env;
@@ -58,4 +91,16 @@
public LogicalOperatorTag getOperatorTag() {
return LogicalOperatorTag.LEFT_OUTER_UNNEST;
}
+
+ private static IAlgebricksConstantValue validateMissingValue(IAlgebricksConstantValue value) {
+ if (value == null) {
+ throw new NullPointerException();
+ } else if (value.isMissing()) {
+ return ConstantExpression.MISSING.getValue();
+ } else if (value.isNull()) {
+ return ConstantExpression.NULL.getValue();
+ } else {
+ throw new IllegalArgumentException(String.valueOf(value));
+ }
+ }
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/SelectOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/SelectOperator.java
index b2e2dfd..0b2c32d 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/SelectOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/SelectOperator.java
@@ -19,6 +19,7 @@
package org.apache.hyracks.algebricks.core.algebra.operators.logical;
import java.util.ArrayList;
+import java.util.List;
import org.apache.commons.lang3.mutable.Mutable;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -27,9 +28,12 @@
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.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.core.algebra.properties.TypePropagationPolicy;
import org.apache.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
import org.apache.hyracks.algebricks.core.algebra.typing.ITypeEnvPointer;
@@ -41,13 +45,28 @@
public class SelectOperator extends AbstractLogicalOperator {
private final Mutable<ILogicalExpression> condition;
- private final boolean retainMissing;
+ private final IAlgebricksConstantValue retainMissingAsValue;
private final LogicalVariable nullPlaceholderVar;
- public SelectOperator(Mutable<ILogicalExpression> condition, boolean retainMissing,
+ public SelectOperator(Mutable<ILogicalExpression> condition) {
+ this(condition, null, null);
+ }
+
+ public SelectOperator(Mutable<ILogicalExpression> condition, IAlgebricksConstantValue retainMissingAsValue,
LogicalVariable nullPlaceholderVar) {
this.condition = condition;
- this.retainMissing = retainMissing;
+ if (retainMissingAsValue == null) {
+ this.retainMissingAsValue = null;
+ if (nullPlaceholderVar != null) {
+ throw new IllegalArgumentException(nullPlaceholderVar.toString());
+ }
+ } else if (retainMissingAsValue.isMissing()) {
+ this.retainMissingAsValue = ConstantExpression.MISSING.getValue();
+ } else if (retainMissingAsValue.isNull()) {
+ this.retainMissingAsValue = ConstantExpression.NULL.getValue();
+ } else {
+ throw new IllegalArgumentException(retainMissingAsValue.toString());
+ }
this.nullPlaceholderVar = nullPlaceholderVar;
}
@@ -60,11 +79,11 @@
return condition;
}
- public boolean getRetainMissing() {
- return retainMissing;
+ public IAlgebricksConstantValue getRetainMissingAsValue() {
+ return retainMissingAsValue;
}
- public LogicalVariable getMissingPlaceholderVariable() throws AlgebricksException {
+ public LogicalVariable getMissingPlaceholderVariable() {
return nullPlaceholderVar;
}
@@ -109,12 +128,11 @@
ILogicalExpression a1 = f1.getArguments().get(0).getValue();
if (a1.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
AbstractFunctionCallExpression f2 = (AbstractFunctionCallExpression) a1;
- if (f2.getFunctionIdentifier().equals(AlgebricksBuiltinFunctions.IS_MISSING)) {
- ILogicalExpression a2 = f2.getArguments().get(0).getValue();
- if (a2.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
- LogicalVariable var = ((VariableReferenceExpression) a2).getVariableReference();
- env.getNonMissableVariables().add(var);
- }
+ FunctionIdentifier f2id = f2.getFunctionIdentifier();
+ if (f2id.equals(AlgebricksBuiltinFunctions.IS_MISSING)) {
+ extractFunctionArgVarInto(f2, env.getNonMissableVariables());
+ } else if (f2id.equals(AlgebricksBuiltinFunctions.IS_NULL)) {
+ extractFunctionArgVarInto(f2, env.getNonNullableVariables());
}
}
return env;
@@ -124,4 +142,13 @@
public boolean requiresVariableReferenceExpressions() {
return false;
}
+
+ private static void extractFunctionArgVarInto(AbstractFunctionCallExpression callExpr,
+ List<? super LogicalVariable> outList) {
+ ILogicalExpression arg = callExpr.getArguments().get(0).getValue();
+ if (arg.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+ LogicalVariable var = ((VariableReferenceExpression) arg).getVariableReference();
+ outList.add(var);
+ }
+ }
}
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 aa5b5b4..e5d0241 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
@@ -225,6 +225,9 @@
return Boolean.FALSE;
}
LeftOuterJoinOperator joinOpArg = (LeftOuterJoinOperator) copyAndSubstituteVar(op, arg);
+ if (!op.getMissingValue().equals(joinOpArg.getMissingValue())) {
+ return Boolean.FALSE;
+ }
boolean isomorphic = op.getCondition().getValue().equals(joinOpArg.getCondition().getValue());
return isomorphic;
}
@@ -451,6 +454,10 @@
if (!isomorphic) {
return Boolean.FALSE;
}
+ isomorphic = op.getMissingValue().equals(loUnnestOpArg.getMissingValue());
+ if (!isomorphic) {
+ return Boolean.FALSE;
+ }
isomorphic = op.getExpressionRef().getValue().equals(loUnnestOpArg.getExpressionRef().getValue());
return isomorphic;
}
@@ -818,6 +825,10 @@
if (!isomorphic) {
return Boolean.FALSE;
}
+ isomorphic = op.getMissingValue().equals(unnestOpArg.getMissingValue());
+ if (!isomorphic) {
+ return Boolean.FALSE;
+ }
isomorphic = op.getExpressionRef().getValue().equals(unnestOpArg.getExpressionRef().getValue());
return isomorphic;
}
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 e05d12a..199c2e1 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
@@ -389,7 +389,7 @@
LeftOuterJoinOperator opCopy =
new LeftOuterJoinOperator(exprDeepCopyVisitor.deepCopyExpressionReference(op.getCondition()),
deepCopyOperatorReference(op.getInputs().get(0), arg),
- deepCopyOperatorReference(op.getInputs().get(1), arg));
+ deepCopyOperatorReference(op.getInputs().get(1), arg), op.getMissingValue());
copyAnnotations(op, opCopy);
copySourceLocation(op, opCopy);
opCopy.setExecutionMode(op.getExecutionMode());
@@ -474,7 +474,7 @@
@Override
public ILogicalOperator visitSelectOperator(SelectOperator op, ILogicalOperator arg) throws AlgebricksException {
SelectOperator opCopy = new SelectOperator(exprDeepCopyVisitor.deepCopyExpressionReference(op.getCondition()),
- op.getRetainMissing(), deepCopyVariable(op.getMissingPlaceholderVariable()));
+ op.getRetainMissingAsValue(), deepCopyVariable(op.getMissingPlaceholderVariable()));
deepCopyInputsAnnotationsAndExecutionMode(op, arg, opCopy);
return opCopy;
}
@@ -544,7 +544,7 @@
throws AlgebricksException {
LeftOuterUnnestMapOperator opCopy = new LeftOuterUnnestMapOperator(deepCopyVariableList(op.getVariables()),
exprDeepCopyVisitor.deepCopyExpressionReference(op.getExpressionRef()), op.getVariableTypes(),
- op.propagatesInput());
+ op.getMissingValue());
deepCopyInputsAnnotationsAndExecutionMode(op, arg, opCopy);
return opCopy;
}
@@ -590,7 +590,7 @@
throws AlgebricksException {
LeftOuterUnnestOperator opCopy = new LeftOuterUnnestOperator(deepCopyVariable(op.getVariable()),
exprDeepCopyVisitor.deepCopyExpressionReference(op.getExpressionRef()),
- deepCopyVariable(op.getPositionalVariable()), op.getPositionalVariableType());
+ deepCopyVariable(op.getPositionalVariable()), op.getPositionalVariableType(), op.getMissingValue());
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 1c91d7b..7b67af1 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
@@ -133,7 +133,7 @@
@Override
public ILogicalOperator visitLeftOuterJoinOperator(LeftOuterJoinOperator op, Void arg) throws AlgebricksException {
return new LeftOuterJoinOperator(deepCopyExpressionRef(op.getCondition()), op.getInputs().get(0),
- op.getInputs().get(1));
+ op.getInputs().get(1), op.getMissingValue());
}
@Override
@@ -158,7 +158,7 @@
@Override
public ILogicalOperator visitSelectOperator(SelectOperator op, Void arg) throws AlgebricksException {
- return new SelectOperator(deepCopyExpressionRef(op.getCondition()), op.getRetainMissing(),
+ return new SelectOperator(deepCopyExpressionRef(op.getCondition()), op.getRetainMissingAsValue(),
op.getMissingPlaceholderVariable());
}
@@ -247,7 +247,7 @@
ArrayList<LogicalVariable> newInputList = new ArrayList<>();
newInputList.addAll(op.getVariables());
return new LeftOuterUnnestMapOperator(newInputList, deepCopyExpressionRef(op.getExpressionRef()),
- new ArrayList<>(op.getVariableTypes()), op.propagatesInput());
+ new ArrayList<>(op.getVariableTypes()), op.getMissingValue());
}
@Override
@@ -420,7 +420,7 @@
public ILogicalOperator visitLeftOuterUnnestOperator(LeftOuterUnnestOperator op, Void arg)
throws AlgebricksException {
return new LeftOuterUnnestOperator(op.getVariable(), deepCopyExpressionRef(op.getExpressionRef()),
- op.getPositionalVariable(), op.getPositionalVariableType());
+ op.getPositionalVariable(), op.getPositionalVariableType(), op.getMissingValue());
}
@Override
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractUnnestPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractUnnestPOperator.java
index 90e88dc..45360d4 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractUnnestPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractUnnestPOperator.java
@@ -31,11 +31,13 @@
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractUnnestNonMapOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterUnnestOperator;
import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenHelper;
import org.apache.hyracks.algebricks.data.IUnnestingPositionWriterFactory;
import org.apache.hyracks.algebricks.runtime.base.IUnnestingEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.operators.std.UnnestRuntimeFactory;
+import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
import org.apache.hyracks.util.LogRedactionUtil;
@@ -84,8 +86,11 @@
int[] projectionList = JobGenHelper.projectAllVariables(opSchema);
IUnnestingPositionWriterFactory positionWriterFactory =
unnest.hasPositionalVariable() ? context.getUnnestingPositionWriterFactory() : null;
+ IMissingWriterFactory missingWriterFactory = leftOuter
+ ? JobGenHelper.getMissingWriterFactory(context, ((LeftOuterUnnestOperator) op).getMissingValue())
+ : null;
UnnestRuntimeFactory unnestRuntime = new UnnestRuntimeFactory(outCol, unnestingFactory, projectionList,
- positionWriterFactory, leftOuter, context.getMissingWriterFactory());
+ positionWriterFactory, leftOuter, missingWriterFactory);
unnestRuntime.setSourceLocation(unnest.getSourceLocation());
RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema, context);
builder.contributeMicroOperator(unnest, unnestRuntime, recDesc);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/HybridHashJoinPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/HybridHashJoinPOperator.java
index c7c7588..7e7d012 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/HybridHashJoinPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/HybridHashJoinPOperator.java
@@ -36,6 +36,7 @@
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator.JoinKind;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
import org.apache.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
import org.apache.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
import org.apache.hyracks.algebricks.core.algebra.properties.LocalGroupingProperty;
@@ -128,9 +129,9 @@
IOperatorDescriptorRegistry spec = builder.getJobSpec();
IOperatorDescriptor opDesc;
- opDesc = generateOptimizedHashJoinRuntime(context, inputSchemas, keysLeft, keysRight, leftHashFunFamilies,
- rightHashFunFamilies, comparatorFactory, reverseComparatorFactory, predEvaluatorFactory, recDescriptor,
- spec);
+ opDesc = generateOptimizedHashJoinRuntime(context, joinOp, inputSchemas, keysLeft, keysRight,
+ leftHashFunFamilies, rightHashFunFamilies, comparatorFactory, reverseComparatorFactory,
+ predEvaluatorFactory, recDescriptor, spec);
opDesc.setSourceLocation(op.getSourceLocation());
contributeOpDesc(builder, (AbstractLogicalOperator) op, opDesc);
@@ -140,11 +141,12 @@
builder.contributeGraphEdge(src2, 0, op, 1);
}
- private IOperatorDescriptor generateOptimizedHashJoinRuntime(JobGenContext context, IOperatorSchema[] inputSchemas,
- int[] keysLeft, int[] keysRight, IBinaryHashFunctionFamily[] leftHashFunFamilies,
- IBinaryHashFunctionFamily[] rightHashFunFamilies, ITuplePairComparatorFactory comparatorFactory,
- ITuplePairComparatorFactory reverseComparatorFactory, IPredicateEvaluatorFactory predEvaluatorFactory,
- RecordDescriptor recDescriptor, IOperatorDescriptorRegistry spec) {
+ private IOperatorDescriptor generateOptimizedHashJoinRuntime(JobGenContext context,
+ AbstractBinaryJoinOperator joinOp, IOperatorSchema[] inputSchemas, int[] keysLeft, int[] keysRight,
+ IBinaryHashFunctionFamily[] leftHashFunFamilies, IBinaryHashFunctionFamily[] rightHashFunFamilies,
+ ITuplePairComparatorFactory comparatorFactory, ITuplePairComparatorFactory reverseComparatorFactory,
+ IPredicateEvaluatorFactory predEvaluatorFactory, RecordDescriptor recDescriptor,
+ IOperatorDescriptorRegistry spec) throws AlgebricksException {
int memSizeInFrames = localMemoryRequirements.getMemoryBudgetInFrames();
switch (kind) {
case INNER:
@@ -152,10 +154,8 @@
getFudgeFactor(), keysLeft, keysRight, leftHashFunFamilies, rightHashFunFamilies, recDescriptor,
comparatorFactory, reverseComparatorFactory, predEvaluatorFactory);
case LEFT_OUTER:
- IMissingWriterFactory[] nonMatchWriterFactories = new IMissingWriterFactory[inputSchemas[1].getSize()];
- for (int j = 0; j < nonMatchWriterFactories.length; j++) {
- nonMatchWriterFactories[j] = context.getMissingWriterFactory();
- }
+ IMissingWriterFactory[] nonMatchWriterFactories = JobGenHelper.createMissingWriterFactories(context,
+ ((LeftOuterJoinOperator) joinOp).getMissingValue(), inputSchemas[1].getSize());
return new OptimizedHybridHashJoinOperatorDescriptor(spec, memSizeInFrames, maxInputBuildSizeInFrames,
getFudgeFactor(), keysLeft, keysRight, leftHashFunFamilies, rightHashFunFamilies, recDescriptor,
comparatorFactory, reverseComparatorFactory, predEvaluatorFactory, true,
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InMemoryHashJoinPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InMemoryHashJoinPOperator.java
index 8f911b4..84dda11 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InMemoryHashJoinPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InMemoryHashJoinPOperator.java
@@ -34,6 +34,7 @@
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator.JoinKind;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
import org.apache.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
import org.apache.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
@@ -119,10 +120,8 @@
memSizeInFrames);
break;
case LEFT_OUTER:
- IMissingWriterFactory[] nonMatchWriterFactories = new IMissingWriterFactory[inputSchemas[1].getSize()];
- for (int j = 0; j < nonMatchWriterFactories.length; j++) {
- nonMatchWriterFactories[j] = context.getMissingWriterFactory();
- }
+ IMissingWriterFactory[] nonMatchWriterFactories = JobGenHelper.createMissingWriterFactories(context,
+ ((LeftOuterJoinOperator) joinOp).getMissingValue(), inputSchemas[1].getSize());
opDesc = new InMemoryHashJoinOperatorDescriptor(spec, keysLeft, keysRight, leftHashFunFactories,
rightHashFunFactories, comparatorFactory, predEvaluatorFactory, recDescriptor, true,
nonMatchWriterFactories, tableSize, memSizeInFrames);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/NestedLoopJoinPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/NestedLoopJoinPOperator.java
index 0991db8..6c5c8d0 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/NestedLoopJoinPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/NestedLoopJoinPOperator.java
@@ -29,6 +29,7 @@
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator.JoinKind;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
import org.apache.hyracks.algebricks.core.algebra.properties.BroadcastPartitioningProperty;
import org.apache.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
import org.apache.hyracks.algebricks.core.algebra.properties.IPartitioningRequirementsCoordinator;
@@ -135,10 +136,8 @@
null);
break;
case LEFT_OUTER:
- IMissingWriterFactory[] nonMatchWriterFactories = new IMissingWriterFactory[inputSchemas[1].getSize()];
- for (int j = 0; j < nonMatchWriterFactories.length; j++) {
- nonMatchWriterFactories[j] = context.getMissingWriterFactory();
- }
+ IMissingWriterFactory[] nonMatchWriterFactories = JobGenHelper.createMissingWriterFactories(context,
+ ((LeftOuterJoinOperator) join).getMissingValue(), inputSchemas[1].getSize());
opDesc = new NestedLoopJoinOperatorDescriptor(spec, comparatorFactory, recDescriptor, memSize, true,
nonMatchWriterFactories);
break;
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StreamSelectPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StreamSelectPOperator.java
index a519275..5af7f98 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StreamSelectPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StreamSelectPOperator.java
@@ -23,6 +23,7 @@
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
import org.apache.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionRuntimeProvider;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
@@ -32,6 +33,7 @@
import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenHelper;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.operators.std.StreamSelectRuntimeFactory;
+import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
public class StreamSelectPOperator extends AbstractPhysicalOperator {
@@ -66,10 +68,13 @@
IExpressionRuntimeProvider expressionRuntimeProvider = context.getExpressionRuntimeProvider();
IScalarEvaluatorFactory cond = expressionRuntimeProvider.createEvaluatorFactory(
select.getCondition().getValue(), context.getTypeEnvironment(op), inputSchemas, context);
+ IAlgebricksConstantValue retainMissingAsValue = select.getRetainMissingAsValue();
+ boolean retainMissing = retainMissingAsValue != null;
+ IMissingWriterFactory missingWriterFactory =
+ retainMissing ? JobGenHelper.getMissingWriterFactory(context, retainMissingAsValue) : null;
StreamSelectRuntimeFactory runtime =
- new StreamSelectRuntimeFactory(cond, null, context.getBinaryBooleanInspectorFactory(),
- select.getRetainMissing(), inputSchemas[0].findVariable(select.getMissingPlaceholderVariable()),
- context.getMissingWriterFactory());
+ new StreamSelectRuntimeFactory(cond, null, context.getBinaryBooleanInspectorFactory(), retainMissing,
+ inputSchemas[0].findVariable(select.getMissingPlaceholderVariable()), missingWriterFactory);
runtime.setSourceLocation(select.getSourceLocation());
// contribute one Asterix framewriter
RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema, context);
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 625418a..64abaee 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
@@ -30,6 +30,7 @@
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalPlan;
import org.apache.hyracks.algebricks.core.algebra.base.IPhysicalOperator;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
import org.apache.hyracks.algebricks.core.algebra.metadata.IProjectionInfo;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
@@ -180,8 +181,9 @@
@Override
public Void visitLeftOuterJoinOperator(LeftOuterJoinOperator op, Integer indent) throws AlgebricksException {
- addIndent(indent).append("left outer join (").append(op.getCondition().getValue().accept(exprVisitor, indent))
- .append(")");
+ addIndent(indent).append("left outer join ").append("(")
+ .append(op.getCondition().getValue().accept(exprVisitor, indent)).append(")")
+ .append(printLeftOuterMissingValue(op.getMissingValue()));
return null;
}
@@ -244,8 +246,11 @@
@Override
public Void visitSelectOperator(SelectOperator op, Integer indent) throws AlgebricksException {
+ String retainMissing = op.getMissingPlaceholderVariable() != null
+ ? " retain-untrue (" + op.getMissingPlaceholderVariable() + " <- " + op.getRetainMissingAsValue() + ")"
+ : "";
addIndent(indent).append("select (").append(op.getCondition().getValue().accept(exprVisitor, indent))
- .append(")");
+ .append(")").append(retainMissing);
return null;
}
@@ -306,17 +311,18 @@
@Override
public Void visitLeftOuterUnnestOperator(LeftOuterUnnestOperator op, Integer indent) throws AlgebricksException {
- addIndent(indent).append("outer-unnest " + op.getVariable());
+ addIndent(indent).append("outer-unnest ").append(String.valueOf(op.getVariable()));
if (op.getPositionalVariable() != null) {
buffer.append(" at " + op.getPositionalVariable());
}
buffer.append(" <- " + op.getExpressionRef().getValue().accept(exprVisitor, indent));
+ buffer.append(printLeftOuterMissingValue(op.getMissingValue()));
return null;
}
@Override
public Void visitUnnestMapOperator(UnnestMapOperator op, Integer indent) throws AlgebricksException {
- AlgebricksStringBuilderWriter plan = printAbstractUnnestMapOperator(op, indent, "unnest-map");
+ AlgebricksStringBuilderWriter plan = printAbstractUnnestMapOperator(op, indent, "unnest-map", null);
appendSelectConditionInformation(plan, op.getSelectCondition(), indent);
appendLimitInformation(plan, op.getOutputLimit());
return null;
@@ -325,14 +331,16 @@
@Override
public Void visitLeftOuterUnnestMapOperator(LeftOuterUnnestMapOperator op, Integer indent)
throws AlgebricksException {
- printAbstractUnnestMapOperator(op, indent, "left-outer-unnest-map");
+ printAbstractUnnestMapOperator(op, indent, "left-outer-unnest-map", op.getMissingValue());
return null;
}
private AlgebricksStringBuilderWriter printAbstractUnnestMapOperator(AbstractUnnestMapOperator op, Integer indent,
- String opSignature) throws AlgebricksException {
- AlgebricksStringBuilderWriter plan = addIndent(indent).append(opSignature + " " + op.getVariables() + " <- "
- + op.getExpressionRef().getValue().accept(exprVisitor, indent));
+ String opSignature, IAlgebricksConstantValue leftOuterMissingValue) throws AlgebricksException {
+ AlgebricksStringBuilderWriter plan =
+ addIndent(indent).append(opSignature).append(' ').append(String.valueOf(op.getVariables()))
+ .append(" <- ").append(op.getExpressionRef().getValue().accept(exprVisitor, indent)).append(
+ leftOuterMissingValue != null ? printLeftOuterMissingValue(leftOuterMissingValue) : "");
appendFilterInformation(plan, op.getMinFilterVars(), op.getMaxFilterVars());
return plan;
}
@@ -665,4 +673,8 @@
}
}
}
+
+ private static String printLeftOuterMissingValue(IAlgebricksConstantValue leftOuterMissingValue) {
+ return leftOuterMissingValue.isNull() ? " (or <- " + leftOuterMissingValue + ")" : "";
+ }
}
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 ed5299c..2728e46 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java
@@ -34,6 +34,7 @@
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalPlan;
import org.apache.hyracks.algebricks.core.algebra.base.IPhysicalOperator;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
import org.apache.hyracks.algebricks.core.algebra.metadata.IProjectionInfo;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
@@ -94,6 +95,7 @@
private static final String EXPRESSIONS_FIELD = "expressions";
private static final String EXPRESSION_FIELD = "expression";
private static final String CONDITION_FIELD = "condition";
+ private static final String MISSING_VALUE_FIELD = "missing-value";
private final Map<AbstractLogicalOperator, String> operatorIdentity = new HashMap<>();
private final IdCounter idCounter = new IdCounter();
@@ -301,6 +303,9 @@
try {
jsonGenerator.writeStringField(OPERATOR_FIELD, "left-outer-join");
writeStringFieldExpression(CONDITION_FIELD, op.getCondition(), indent);
+ if (op.getMissingValue().isNull()) {
+ writeNullField(MISSING_VALUE_FIELD);
+ }
return null;
} catch (IOException e) {
throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
@@ -467,14 +472,21 @@
@Override
public Void visitLeftOuterUnnestOperator(LeftOuterUnnestOperator op, Void indent) throws AlgebricksException {
- writeUnnestNonMapOperator(op, "outer-unnest", indent);
- return null;
+ try {
+ writeUnnestNonMapOperator(op, "outer-unnest", indent);
+ if (op.getMissingValue().isNull()) {
+ writeNullField(MISSING_VALUE_FIELD);
+ }
+ return null;
+ } catch (IOException e) {
+ throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
+ }
}
@Override
public Void visitUnnestMapOperator(UnnestMapOperator op, Void indent) throws AlgebricksException {
try {
- writeUnnestMapOperator(op, indent, "unnest-map");
+ writeUnnestMapOperator(op, indent, "unnest-map", null);
writeSelectLimitInformation(op.getSelectCondition(), op.getOutputLimit(), indent);
return null;
} catch (IOException e) {
@@ -484,7 +496,7 @@
@Override
public Void visitLeftOuterUnnestMapOperator(LeftOuterUnnestMapOperator op, Void indent) throws AlgebricksException {
- writeUnnestMapOperator(op, indent, "left-outer-unnest-map");
+ writeUnnestMapOperator(op, indent, "left-outer-unnest-map", op.getMissingValue());
return null;
}
@@ -782,8 +794,8 @@
}
}
- private void writeUnnestMapOperator(AbstractUnnestMapOperator op, Void indent, String opName)
- throws AlgebricksException {
+ private void writeUnnestMapOperator(AbstractUnnestMapOperator op, Void indent, String opName,
+ IAlgebricksConstantValue leftOuterMissingValue) throws AlgebricksException {
try {
jsonGenerator.writeStringField(OPERATOR_FIELD, opName);
List<LogicalVariable> variables = op.getVariables();
@@ -792,6 +804,9 @@
}
writeArrayFieldOfExpression(EXPRESSIONS_FIELD, op.getExpressionRef(), indent);
writeFilterInformation(op.getMinFilterVars(), op.getMaxFilterVars());
+ if (leftOuterMissingValue != null && leftOuterMissingValue.isNull()) {
+ writeNullField(MISSING_VALUE_FIELD);
+ }
} catch (IOException e) {
throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
}
@@ -963,6 +978,13 @@
jsonGenerator.writeEndObject();
}
+ /////////////// other fields ///////////////
+
+ /** Writes "fieldName": null */
+ private void writeNullField(String fieldName) throws IOException {
+ jsonGenerator.writeNullField(fieldName);
+ }
+
private void flushContentToWriter() throws AlgebricksException {
try {
jsonGenerator.flush();
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/LeftOuterTypePropagationPolicy.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/LeftOuterTypePropagationPolicy.java
new file mode 100644
index 0000000..07f1f7f
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/LeftOuterTypePropagationPolicy.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.hyracks.algebricks.core.algebra.properties;
+
+import java.util.List;
+
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IMissableTypeComputer;
+import org.apache.hyracks.algebricks.core.algebra.typing.ITypeEnvPointer;
+import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
+
+public abstract class LeftOuterTypePropagationPolicy extends TypePropagationPolicy {
+
+ public static final TypePropagationPolicy MISSABLE = new LeftOuterTypePropagationPolicy() {
+
+ protected Object computeInnerBranchVarType(LogicalVariable var, Object varType, IMissableTypeComputer ntc,
+ List<LogicalVariable> nonMissableVariableList,
+ List<List<LogicalVariable>> correlatedMissableVariableLists,
+ List<LogicalVariable> nonNullableVariableList,
+ List<List<LogicalVariable>> correlatedNullableVariableLists) {
+ boolean makeMissable =
+ !inCorrelatedVariableList(var, correlatedMissableVariableLists, nonMissableVariableList);
+ return makeMissable ? ntc.makeMissableType(varType) : varType;
+ }
+ };
+
+ public static final TypePropagationPolicy NULLABLE = new LeftOuterTypePropagationPolicy() {
+
+ protected Object computeInnerBranchVarType(LogicalVariable var, Object varType, IMissableTypeComputer ntc,
+ List<LogicalVariable> nonMissableVariableList,
+ List<List<LogicalVariable>> correlatedMissableVariableLists,
+ List<LogicalVariable> nonNullableVariableList,
+ List<List<LogicalVariable>> correlatedNullableVariableLists) {
+ boolean makeNullable =
+ !inCorrelatedVariableList(var, correlatedNullableVariableLists, nonNullableVariableList);
+ return makeNullable ? ntc.makeNullableType(varType) : varType;
+ }
+ };
+
+ @Override
+ public Object getVarType(LogicalVariable var, IMissableTypeComputer ntc,
+ List<LogicalVariable> nonMissableVariableList, List<List<LogicalVariable>> correlatedMissableVariableLists,
+ List<LogicalVariable> nonNullableVariableList, List<List<LogicalVariable>> correlatedNullableVariableLists,
+ ITypeEnvPointer... typeEnvs) throws AlgebricksException {
+ int n = typeEnvs.length;
+ // Searches from the inner branch to the outer branch.
+ // TODO(buyingyi): A split operator could lead to the case that the type for a variable could be
+ // found in both inner and outer branches. Fix computeOutputTypeEnvironment() in ProjectOperator
+ // and investigate why many test queries fail if only live variables' types are propagated.
+ for (int i = n - 1; i >= 0; i--) {
+ Object varType = typeEnvs[i].getTypeEnv().getVarType(var, nonMissableVariableList,
+ correlatedMissableVariableLists, nonNullableVariableList, correlatedNullableVariableLists);
+ if (varType == null) {
+ continue;
+ }
+ if (i == 0) { // outer branch
+ return varType;
+ }
+ // inner branch
+ return computeInnerBranchVarType(var, varType, ntc, nonMissableVariableList,
+ correlatedMissableVariableLists, nonNullableVariableList, correlatedNullableVariableLists);
+ }
+ return null;
+ }
+
+ protected abstract Object computeInnerBranchVarType(LogicalVariable var, Object varType, IMissableTypeComputer ntc,
+ List<LogicalVariable> nonMissableVariableList, List<List<LogicalVariable>> correlatedMissableVariableLists,
+ List<LogicalVariable> nonNullableVariableList, List<List<LogicalVariable>> correlatedNullableVariableLists);
+
+ protected static boolean inCorrelatedVariableList(LogicalVariable var,
+ List<List<LogicalVariable>> correlatedOptionalVariableLists,
+ List<LogicalVariable> nonOptionalVariableList) {
+ if (!nonOptionalVariableList.isEmpty()) {
+ for (List<LogicalVariable> correlatedVariables : correlatedOptionalVariableLists) {
+ if (correlatedVariables.contains(var)
+ && !OperatorPropertiesUtil.disjoint(correlatedVariables, nonOptionalVariableList)) {
+ return true;
+ }
+ }
+ }
+ return false;
+ }
+};
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/TypePropagationPolicy.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/TypePropagationPolicy.java
index c37c674..9cd5fe1 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/TypePropagationPolicy.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/TypePropagationPolicy.java
@@ -32,7 +32,9 @@
@Override
public Object getVarType(LogicalVariable var, IMissableTypeComputer ntc,
List<LogicalVariable> nonMissableVariableList,
- List<List<LogicalVariable>> correlatedMissableVariableLists, ITypeEnvPointer... typeEnvs)
+ List<List<LogicalVariable>> correlatedMissableVariableLists,
+ List<LogicalVariable> nonNullableVariableList,
+ List<List<LogicalVariable>> correlatedNullableVariableLists, ITypeEnvPointer... typeEnvs)
throws AlgebricksException {
for (ITypeEnvPointer p : typeEnvs) {
IVariableTypeEnvironment env = p.getTypeEnv();
@@ -40,67 +42,22 @@
throw new AlgebricksException(
"Null environment for pointer " + p + " in getVarType for var=" + var);
}
- Object t = env.getVarType(var, nonMissableVariableList, correlatedMissableVariableLists);
+ Object t = env.getVarType(var, nonMissableVariableList, correlatedMissableVariableLists,
+ nonNullableVariableList, correlatedNullableVariableLists);
if (t != null) {
- if (ntc != null && ntc.canBeMissing(t)) {
- for (List<LogicalVariable> list : correlatedMissableVariableLists) {
- if (list.contains(var)) {
- for (LogicalVariable v : list) {
- if (nonMissableVariableList.contains(v)) {
- return ntc.getNonOptionalType(t);
- }
- }
- }
- }
+ boolean makeNonMissable = !nonMissableVariableList.isEmpty() && ntc.canBeMissing(t)
+ && nonMissableVariableList.contains(var);
+ boolean makeNonNullable = !nonNullableVariableList.isEmpty() && ntc.canBeNull(t)
+ && nonNullableVariableList.contains(var);
+ if (makeNonMissable && makeNonNullable) {
+ return ntc.getNonOptionalType(t);
+ } else if (makeNonMissable) {
+ return ntc.getNonMissableType(t);
+ } else if (makeNonNullable) {
+ return ntc.getNonNullableType(t);
+ } else {
+ return t;
}
- return t;
- }
- }
- return null;
- }
- };
-
- public static final TypePropagationPolicy LEFT_OUTER = new TypePropagationPolicy() {
-
- @Override
- public Object getVarType(LogicalVariable var, IMissableTypeComputer ntc,
- List<LogicalVariable> nonMissableVariableList,
- List<List<LogicalVariable>> correlatedMissableVariableLists, ITypeEnvPointer... typeEnvs)
- throws AlgebricksException {
- int n = typeEnvs.length;
- // Searches from the inner branch to the outer branch.
- // TODO(buyingyi): A split operator could lead to the case that the type for a variable could be
- // found in both inner and outer branches. Fix computeOutputTypeEnvironment() in ProjectOperator
- // and investigate why many test queries fail if only live variables' types are propagated.
- for (int i = n - 1; i >= 0; i--) {
- Object t = typeEnvs[i].getTypeEnv().getVarType(var, nonMissableVariableList,
- correlatedMissableVariableLists);
- if (t == null) {
- continue;
- }
- if (i == 0) { // outer branch
- return t;
- }
-
- // inner branch
- boolean nonMissingVarIsProduced = false;
- for (LogicalVariable v : nonMissableVariableList) {
- boolean toBreak = false;
- if (v == var) {
- nonMissingVarIsProduced = true;
- toBreak = true;
- } else if (typeEnvs[i].getTypeEnv().getVarType(v) != null) {
- nonMissingVarIsProduced = true;
- toBreak = true;
- }
- if (toBreak) {
- break;
- }
- }
- if (nonMissingVarIsProduced) {
- return t;
- } else {
- return ntc.makeMissableType(t);
}
}
return null;
@@ -109,5 +66,6 @@
public abstract Object getVarType(LogicalVariable var, IMissableTypeComputer ntc,
List<LogicalVariable> nonMissableVariableList, List<List<LogicalVariable>> correlatedMissableVariableLists,
+ List<LogicalVariable> nonNullableVariableList, List<List<LogicalVariable>> correlatedNullableVariableLists,
ITypeEnvPointer... typeEnvs) throws AlgebricksException;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/typing/NonPropagatingTypeEnvironment.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/typing/NonPropagatingTypeEnvironment.java
index 547737c..3bd4bc3 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/typing/NonPropagatingTypeEnvironment.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/typing/NonPropagatingTypeEnvironment.java
@@ -38,7 +38,8 @@
}
@Override
- public Object getVarType(LogicalVariable var, List<LogicalVariable> nonNullVariables,
+ public Object getVarType(LogicalVariable var, List<LogicalVariable> nonMissableVariables,
+ List<List<LogicalVariable>> correlatedMissableVariableLists, List<LogicalVariable> nonNullableVariables,
List<List<LogicalVariable>> correlatedNullableVariableLists) throws AlgebricksException {
return getVarType(var);
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/typing/PropagateOperatorInputsTypeEnvironment.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/typing/PropagateOperatorInputsTypeEnvironment.java
deleted file mode 100644
index 1a85768..0000000
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/typing/PropagateOperatorInputsTypeEnvironment.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.hyracks.algebricks.core.algebra.typing;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.commons.lang3.mutable.Mutable;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
-import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
-import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionTypeComputer;
-import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
-import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
-
-public class PropagateOperatorInputsTypeEnvironment extends AbstractTypeEnvironment {
-
- private final List<LogicalVariable> nonNullVariables = new ArrayList<LogicalVariable>();
- private final List<List<LogicalVariable>> correlatedNullableVariableLists = new ArrayList<List<LogicalVariable>>();
- private final ILogicalOperator op;
- private final ITypingContext ctx;
-
- public PropagateOperatorInputsTypeEnvironment(ILogicalOperator op, ITypingContext ctx,
- IExpressionTypeComputer expressionTypeComputer, IMetadataProvider<?, ?> metadataProvider) {
- super(expressionTypeComputer, metadataProvider);
- this.op = op;
- this.ctx = ctx;
- }
-
- public List<LogicalVariable> getNonNullVariables() {
- return nonNullVariables;
- }
-
- @Override
- public Object getVarType(LogicalVariable var, List<LogicalVariable> nonNullVariableList,
- List<List<LogicalVariable>> correlatedNullableVariableLists) throws AlgebricksException {
- nonNullVariableList.addAll(nonNullVariables);
- return getVarTypeFullList(var, nonNullVariableList, correlatedNullableVariableLists);
- }
-
- private Object getVarTypeFullList(LogicalVariable var, List<LogicalVariable> nonNullVariableList,
- List<List<LogicalVariable>> correlatedNullableVariableLists) throws AlgebricksException {
- Object t = varTypeMap.get(var);
- if (t != null) {
- return t;
- }
- for (Mutable<ILogicalOperator> r : op.getInputs()) {
- ILogicalOperator c = r.getValue();
- IVariableTypeEnvironment env = ctx.getOutputTypeEnvironment(c);
- Object t2 = env.getVarType(var, nonNullVariableList, correlatedNullableVariableLists);
- if (t2 != null) {
- return t2;
- }
- }
- return null;
- }
-
- @Override
- public Object getVarType(LogicalVariable var) throws AlgebricksException {
- return getVarTypeFullList(var, nonNullVariables, correlatedNullableVariableLists);
- }
-
-}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/typing/PropagatingTypeEnvironment.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/typing/PropagatingTypeEnvironment.java
index 27aa902..96b55d5 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/typing/PropagatingTypeEnvironment.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/typing/PropagatingTypeEnvironment.java
@@ -27,6 +27,7 @@
import org.apache.hyracks.algebricks.core.algebra.expressions.IMissableTypeComputer;
import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
import org.apache.hyracks.algebricks.core.algebra.properties.TypePropagationPolicy;
+import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
public class PropagatingTypeEnvironment extends AbstractTypeEnvironment {
@@ -40,6 +41,10 @@
private final List<List<LogicalVariable>> correlatedMissableVariableLists = new ArrayList<>();
+ private final List<LogicalVariable> nonNullableVariables = new ArrayList<>();
+
+ private final List<List<LogicalVariable>> correlatedNullableVariableLists = new ArrayList<>();
+
public PropagatingTypeEnvironment(IExpressionTypeComputer expressionTypeComputer,
IMissableTypeComputer missableTypeComputer, IMetadataProvider<?, ?> metadataProvider,
TypePropagationPolicy policy, ITypeEnvPointer[] envPointers) {
@@ -51,7 +56,7 @@
@Override
public Object getVarType(LogicalVariable var) throws AlgebricksException {
- return getVarTypeFullList(var, nonMissableVariables, correlatedMissableVariableLists);
+ return getVarTypeFullList(var, null, null, null, null);
}
public List<LogicalVariable> getNonMissableVariables() {
@@ -62,31 +67,42 @@
return correlatedMissableVariableLists;
}
- @Override
- public Object getVarType(LogicalVariable var, List<LogicalVariable> nonMissableVariableList,
- List<List<LogicalVariable>> correlatedMissableVariableLists) throws AlgebricksException {
- for (LogicalVariable v : nonMissableVariables) {
- if (!nonMissableVariableList.contains(v)) {
- nonMissableVariableList.add(v);
- }
- }
- Object t = getVarTypeFullList(var, nonMissableVariableList, correlatedMissableVariableLists);
- for (List<LogicalVariable> list : correlatedMissableVariableLists) {
- if (!correlatedMissableVariableLists.contains(list)) {
- correlatedMissableVariableLists.add(list);
- }
- }
- return t;
+ public List<LogicalVariable> getNonNullableVariables() {
+ return nonNullableVariables;
}
- private Object getVarTypeFullList(LogicalVariable var, List<LogicalVariable> nonMissableVariableList,
- List<List<LogicalVariable>> correlatedMissableVariableLists) throws AlgebricksException {
+ public List<List<LogicalVariable>> getCorrelatedNullableVariableLists() {
+ return correlatedNullableVariableLists;
+ }
+
+ @Override
+ public Object getVarType(LogicalVariable var, List<LogicalVariable> nonMissableVariables,
+ List<List<LogicalVariable>> correlatedMissableVariableLists, List<LogicalVariable> nonNullableVariables,
+ List<List<LogicalVariable>> correlatedNullableVariableLists) throws AlgebricksException {
+ return getVarTypeFullList(var, nonMissableVariables, correlatedMissableVariableLists, nonNullableVariables,
+ correlatedNullableVariableLists);
+ }
+
+ private Object getVarTypeFullList(LogicalVariable var, List<LogicalVariable> nonMissableVariableListExtra,
+ List<List<LogicalVariable>> correlatedMissableVariableListsExtra,
+ List<LogicalVariable> nonNullableVariableListExtra,
+ List<List<LogicalVariable>> correlatedNullableVariableListsExtra) throws AlgebricksException {
Object t = varTypeMap.get(var);
if (t != null) {
return t;
}
- return policy.getVarType(var, missableTypeComputer, nonMissableVariableList, correlatedMissableVariableLists,
- envPointers);
+
+ List<LogicalVariable> nonMissable =
+ OperatorPropertiesUtil.unionAll(nonMissableVariables, nonMissableVariableListExtra);
+ List<LogicalVariable> nonNullable =
+ OperatorPropertiesUtil.unionAll(nonNullableVariables, nonNullableVariableListExtra);
+ List<List<LogicalVariable>> correlatedMissable =
+ OperatorPropertiesUtil.unionAll(correlatedMissableVariableLists, correlatedMissableVariableListsExtra);
+ List<List<LogicalVariable>> correlatedNullable =
+ OperatorPropertiesUtil.unionAll(correlatedNullableVariableLists, correlatedNullableVariableListsExtra);
+
+ return policy.getVarType(var, missableTypeComputer, nonMissable, correlatedMissable, nonNullable,
+ correlatedNullable, envPointers);
}
@Override
@@ -100,6 +116,14 @@
missableVarList.add(v2);
}
}
+ if (nonNullableVariables.remove(v1)) {
+ nonNullableVariables.add(v2);
+ }
+ for (List<LogicalVariable> nullableVarList : correlatedNullableVariableLists) {
+ if (nullableVarList.remove(v1)) {
+ nullableVarList.add(v2);
+ }
+ }
return result;
}
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorPropertiesUtil.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorPropertiesUtil.java
index c0d72bb..c66ee40 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorPropertiesUtil.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorPropertiesUtil.java
@@ -18,8 +18,11 @@
*/
package org.apache.hyracks.algebricks.core.algebra.util;
+import java.util.ArrayList;
import java.util.Collection;
+import java.util.Collections;
import java.util.HashSet;
+import java.util.List;
import java.util.Set;
import org.apache.commons.lang3.mutable.Mutable;
@@ -32,13 +35,13 @@
import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
-import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
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.AbstractLogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.CardinalityInferenceVisitor;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
import org.apache.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
@@ -221,33 +224,6 @@
}
}
- public static boolean isMissingTest(AbstractLogicalOperator op) {
- if (op.getOperatorTag() != LogicalOperatorTag.SELECT) {
- return false;
- }
- AbstractLogicalOperator doubleUnder = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
- if (doubleUnder.getOperatorTag() != LogicalOperatorTag.NESTEDTUPLESOURCE) {
- return false;
- }
- ILogicalExpression eu = ((SelectOperator) op).getCondition().getValue();
- if (eu.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
- return false;
- }
- AbstractFunctionCallExpression f1 = (AbstractFunctionCallExpression) eu;
- if (!f1.getFunctionIdentifier().equals(AlgebricksBuiltinFunctions.NOT)) {
- return false;
- }
- ILogicalExpression a1 = f1.getArguments().get(0).getValue();
- if (!a1.getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL)) {
- return false;
- }
- AbstractFunctionCallExpression f2 = (AbstractFunctionCallExpression) a1;
- if (!f2.getFunctionIdentifier().equals(AlgebricksBuiltinFunctions.IS_MISSING)) {
- return false;
- }
- return true;
- }
-
public static void typePlan(ILogicalPlan p, IOptimizationContext context) throws AlgebricksException {
for (Mutable<ILogicalOperator> r : p.getRoots()) {
typeOpRec(r, context);
@@ -355,4 +331,22 @@
LogicalOperatorTag opTag = op.getOperatorTag();
return opTag == LogicalOperatorTag.REPLICATE || opTag == LogicalOperatorTag.SPLIT;
}
+
+ public static <T> List<T> unionAll(List<T> list1, List<T> list2) {
+ if (list2 == null || list2.isEmpty()) {
+ return list1 == null ? Collections.emptyList() : list1;
+ } else if (list1 == null || list1.isEmpty()) {
+ return list2;
+ } else {
+ List<T> result = new ArrayList<>(list1.size() + list2.size());
+ result.addAll(list1);
+ result.addAll(list2);
+ return result;
+ }
+ }
+
+ public static FunctionIdentifier getIsMissingNullFunction(IAlgebricksConstantValue value) {
+ return value.isMissing() ? AlgebricksBuiltinFunctions.IS_MISSING
+ : value.isNull() ? AlgebricksBuiltinFunctions.IS_NULL : null;
+ }
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobGenContext.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobGenContext.java
index 2ca4a36..7c7d5a8 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobGenContext.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobGenContext.java
@@ -60,6 +60,7 @@
private final ITypeTraitProvider typeTraitProvider;
private final IMetadataProvider<?, ?> metadataProvider;
private final IMissingWriterFactory missingWriterFactory;
+ private final IMissingWriterFactory nullWriterFactory;
private final IUnnestingPositionWriterFactory unnestingPositionWriterFactory;
private final INormalizedKeyComputerFactoryProvider normalizedKeyComputerFactoryProvider;
private final Object appContext;
@@ -85,7 +86,8 @@
IBinaryComparatorFactoryProvider comparatorFactoryProvider, ITypeTraitProvider typeTraitProvider,
IBinaryBooleanInspectorFactory booleanInspectorFactory,
IBinaryIntegerInspectorFactory integerInspectorFactory, IPrinterFactoryProvider printerFactoryProvider,
- IMissingWriterFactory missingWriterFactory, IUnnestingPositionWriterFactory unnestingPositionWriterFactory,
+ IMissingWriterFactory missingWriterFactory, IMissingWriterFactory nullWriterFactory,
+ IUnnestingPositionWriterFactory unnestingPositionWriterFactory,
INormalizedKeyComputerFactoryProvider normalizedKeyComputerFactoryProvider,
IExpressionRuntimeProvider expressionRuntimeProvider, IExpressionTypeComputer expressionTypeComputer,
ITypingContext typingContext, IExpressionEvalSizeComputer expressionEvalSizeComputer,
@@ -107,6 +109,7 @@
this.clusterLocations = clusterLocations;
this.normalizedKeyComputerFactoryProvider = normalizedKeyComputerFactoryProvider;
this.missingWriterFactory = missingWriterFactory;
+ this.nullWriterFactory = nullWriterFactory;
this.unnestingPositionWriterFactory = unnestingPositionWriterFactory;
this.expressionRuntimeProvider = expressionRuntimeProvider;
this.expressionTypeComputer = expressionTypeComputer;
@@ -199,6 +202,10 @@
return missingWriterFactory;
}
+ public IMissingWriterFactory getNullWriterFactory() {
+ return nullWriterFactory;
+ }
+
public IUnnestingPositionWriterFactory getUnnestingPositionWriterFactory() {
return unnestingPositionWriterFactory;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobGenHelper.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobGenHelper.java
index 5142ce7..42364c8 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobGenHelper.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobGenHelper.java
@@ -18,11 +18,13 @@
*/
package org.apache.hyracks.algebricks.core.jobgen.impl;
+import java.util.Arrays;
import java.util.Collection;
import java.util.List;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
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.expressions.IVariableTypeEnvironment;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
@@ -38,10 +40,12 @@
import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import org.apache.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
import org.apache.hyracks.api.dataflow.value.IBinaryHashFunctionFamily;
+import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
import org.apache.hyracks.api.dataflow.value.ITypeTraits;
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.ErrorCode;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
@@ -213,4 +217,25 @@
}
return projectionList;
}
+
+ public static IMissingWriterFactory[] createMissingWriterFactories(JobGenContext context,
+ IAlgebricksConstantValue missingOrNullValue, int size) throws AlgebricksException {
+ IMissingWriterFactory missingWriterFactory = getMissingWriterFactory(context, missingOrNullValue);
+ IMissingWriterFactory[] nonMatchWriterFactories = new IMissingWriterFactory[size];
+ Arrays.fill(nonMatchWriterFactories, missingWriterFactory);
+ return nonMatchWriterFactories;
+ }
+
+ public static IMissingWriterFactory getMissingWriterFactory(JobGenContext context,
+ IAlgebricksConstantValue missingOrNullValue) throws AlgebricksException {
+ IMissingWriterFactory missingWriterFactory;
+ if (missingOrNullValue.isMissing()) {
+ missingWriterFactory = context.getMissingWriterFactory();
+ } else if (missingOrNullValue.isNull()) {
+ missingWriterFactory = context.getNullWriterFactory();
+ } else {
+ throw new AlgebricksException(ErrorCode.ILLEGAL_STATE);
+ }
+ return missingWriterFactory;
+ }
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/LogicalOperatorDotVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/LogicalOperatorDotVisitor.java
index 01f7403..ee7f7aa 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/LogicalOperatorDotVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/LogicalOperatorDotVisitor.java
@@ -166,7 +166,9 @@
@Override
public String visitLeftOuterJoinOperator(LeftOuterJoinOperator op, Boolean showDetails) throws AlgebricksException {
stringBuilder.setLength(0);
- stringBuilder.append("left outer join (").append(op.getCondition().getValue().toString()).append(")");
+ stringBuilder.append("left outer join ").append("(").append(op.getCondition().getValue().toString())
+ .append(")");
+ stringBuilder.append(op.getMissingValue().isNull() ? " (or null) " : "");
appendSchema(op, showDetails);
appendAnnotations(op, showDetails);
appendPhysicalOperatorInfo(op, showDetails);
@@ -351,6 +353,7 @@
if (op.getPositionalVariable() != null) {
stringBuilder.append(" at ").append(op.getPositionalVariable());
}
+ stringBuilder.append(op.getMissingValue().isNull() ? " (or null) " : "");
stringBuilder.append(" <- ").append(op.getExpressionRef().getValue().toString());
appendSchema(op, showDetails);
appendAnnotations(op, showDetails);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/test/java/org/apache/hyracks/algebricks/core/algebra/plan/PlanStructureVerifierTest.java b/hyracks-fullstack/algebricks/algebricks-core/src/test/java/org/apache/hyracks/algebricks/core/algebra/plan/PlanStructureVerifierTest.java
index f047cec..46e41f1 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/test/java/org/apache/hyracks/algebricks/core/algebra/plan/PlanStructureVerifierTest.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/test/java/org/apache/hyracks/algebricks/core/algebra/plan/PlanStructureVerifierTest.java
@@ -283,14 +283,14 @@
computeAndSetTypeEnvironmentForOperator(ets);
ets.recomputeSchema();
- SelectOperator op1 = new SelectOperator(newMutable(ConstantExpression.TRUE), false, null);
+ SelectOperator op1 = new SelectOperator(newMutable(ConstantExpression.TRUE));
op1.getInputs().add(newMutable(ets));
computeAndSetTypeEnvironmentForOperator(op1);
op1.recomputeSchema();
op1.getInputs().clear();
- SelectOperator op2 = new SelectOperator(newMutable(ConstantExpression.FALSE), false, null);
+ SelectOperator op2 = new SelectOperator(newMutable(ConstantExpression.FALSE));
op2.getInputs().add(newMutable(ets));
op2.recomputeSchema();
// no type env
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/BreakSelectIntoConjunctsRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/BreakSelectIntoConjunctsRule.java
index 50c7703..c0773b2 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/BreakSelectIntoConjunctsRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/BreakSelectIntoConjunctsRule.java
@@ -105,9 +105,9 @@
firstExpression = expression;
} else {
// New select operator
- SelectOperator newSelectOperator =
- new SelectOperator(new MutableObject<>(expression), originalSelectOperator.getRetainMissing(),
- originalSelectOperator.getMissingPlaceholderVariable());
+ SelectOperator newSelectOperator = new SelectOperator(new MutableObject<>(expression),
+ originalSelectOperator.getRetainMissingAsValue(),
+ originalSelectOperator.getMissingPlaceholderVariable());
newSelectOperator.setSourceLocation(sourceLoc);
// Put the new operator at the bottom (child of current operator)
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PullSelectOutOfEqJoin.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PullSelectOutOfEqJoin.java
index afbbc4f..410b2ec 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PullSelectOutOfEqJoin.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PullSelectOutOfEqJoin.java
@@ -80,7 +80,7 @@
}
// pull up
ILogicalExpression pulledCond = makeCondition(otherPredicates, context);
- SelectOperator select = new SelectOperator(new MutableObject<ILogicalExpression>(pulledCond), false, null);
+ SelectOperator select = new SelectOperator(new MutableObject<ILogicalExpression>(pulledCond));
ILogicalExpression newJoinCond = makeCondition(eqVarVarComps, context);
join.getCondition().setValue(newJoinCond);
select.getInputs().add(new MutableObject<ILogicalOperator>(join));
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PushSelectIntoJoinRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PushSelectIntoJoinRule.java
index 5f66c2f..5477662 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PushSelectIntoJoinRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PushSelectIntoJoinRule.java
@@ -38,9 +38,11 @@
import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
@@ -98,10 +100,10 @@
ILogicalOperator joinBranchRight = joinBranchRightRef.getValue();
VariableUtilities.getLiveVariables(joinBranchLeft, joinLiveVarsLeft);
VariableUtilities.getLiveVariables(joinBranchRight, joinLiveVarsRight);
- Mutable<ILogicalOperator> opIterRef = opRef2;
+ Mutable<ILogicalOperator> opIterRef;
ILogicalOperator opIter = op2;
while (opIter != join) {
- LogicalOperatorTag tag = ((AbstractLogicalOperator) opIter).getOperatorTag();
+ LogicalOperatorTag tag = opIter.getOperatorTag();
if (tag == LogicalOperatorTag.PROJECT) {
notPushedStack.addFirst(opIter);
} else {
@@ -129,11 +131,11 @@
boolean intersectsAllBranches = true;
boolean[] intersectsBranch = new boolean[join.getInputs().size()];
- LinkedList<LogicalVariable> selectVars = new LinkedList<LogicalVariable>();
+ LinkedList<LogicalVariable> selectVars = new LinkedList<>();
select.getCondition().getValue().getUsedVariables(selectVars);
int i = 0;
for (Mutable<ILogicalOperator> branch : join.getInputs()) {
- LinkedList<LogicalVariable> branchVars = new LinkedList<LogicalVariable>();
+ LinkedList<LogicalVariable> branchVars = new LinkedList<>();
VariableUtilities.getLiveVariables(branch.getValue(), branchVars);
if (i == 0) {
branchVars.addAll(liveInOpsToPushLeft);
@@ -181,7 +183,9 @@
if (j > 0 && isLoj) {
// if a LOJ and the select condition is not-missing filtering,
// we rewrite LOJ to IJ for this case.
- if (containsNotMissingFiltering(selectCondition)) {
+ FunctionIdentifier isMissingNullFunction = OperatorPropertiesUtil
+ .getIsMissingNullFunction(((LeftOuterJoinOperator) join).getMissingValue());
+ if (containsNotMissingFiltering(selectCondition, isMissingNullFunction)) {
lojToInner = true;
}
// Do not push conditions into the right branch of a LOJ;
@@ -247,7 +251,7 @@
ILogicalOperator op = iter.previous();
List<Mutable<ILogicalOperator>> opInpList = op.getInputs();
opInpList.clear();
- opInpList.add(new MutableObject<ILogicalOperator>(topOp));
+ opInpList.add(new MutableObject<>(topOp));
topOp = op;
context.computeAndSetTypeEnvironmentForOperator(op);
}
@@ -276,7 +280,7 @@
if (!bAddedToConj) {
AbstractFunctionCallExpression newCond = new ScalarFunctionCallExpression(
context.getMetadataProvider().lookupFunction(AlgebricksBuiltinFunctions.AND),
- select.getCondition(), new MutableObject<ILogicalExpression>(join.getCondition().getValue()));
+ select.getCondition(), new MutableObject<>(join.getCondition().getValue()));
join.getCondition().setValue(newCond);
}
}
@@ -284,9 +288,9 @@
private static void copySelectToBranch(SelectOperator select, Mutable<ILogicalOperator> branch,
IOptimizationContext context) throws AlgebricksException {
- ILogicalOperator newSelect = new SelectOperator(select.getCondition(), select.getRetainMissing(),
+ ILogicalOperator newSelect = new SelectOperator(select.getCondition(), select.getRetainMissingAsValue(),
select.getMissingPlaceholderVariable());
- Mutable<ILogicalOperator> newRef = new MutableObject<ILogicalOperator>(branch.getValue());
+ Mutable<ILogicalOperator> newRef = new MutableObject<>(branch.getValue());
newSelect.getInputs().add(newRef);
branch.setValue(newSelect);
context.computeAndSetTypeEnvironmentForOperator(newSelect);
@@ -296,16 +300,17 @@
* Whether the expression contains a not-missing filtering
*
* @param expr
+ * @param isMissingNullFunId
* @return true if the expression contains a not-missing filtering function call; false otherwise.
*/
- private boolean containsNotMissingFiltering(ILogicalExpression expr) {
+ private boolean containsNotMissingFiltering(ILogicalExpression expr, FunctionIdentifier isMissingNullFunId) {
if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
return false;
}
ScalarFunctionCallExpression func = (ScalarFunctionCallExpression) expr;
if (func.getFunctionIdentifier() == AlgebricksBuiltinFunctions.AND) {
for (Mutable<ILogicalExpression> argumentRef : func.getArguments()) {
- if (containsNotMissingFiltering(argumentRef.getValue())) {
+ if (containsNotMissingFiltering(argumentRef.getValue(), isMissingNullFunId)) {
return true;
}
}
@@ -319,34 +324,6 @@
return false;
}
ScalarFunctionCallExpression func2 = (ScalarFunctionCallExpression) arg;
- if (func2.getFunctionIdentifier() != AlgebricksBuiltinFunctions.IS_MISSING) {
- return false;
- }
- return true;
- }
-
- /**
- * Whether the expression contains a missing filtering
- *
- * @param expr
- * @return true if the expression contains a missing filtering function call; false otherwise.
- */
- private boolean containsMissingFiltering(ILogicalExpression expr) {
- if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
- return false;
- }
- ScalarFunctionCallExpression func = (ScalarFunctionCallExpression) expr;
- if (func.getFunctionIdentifier() == AlgebricksBuiltinFunctions.AND) {
- for (Mutable<ILogicalExpression> argumentRef : func.getArguments()) {
- if (containsMissingFiltering(argumentRef.getValue())) {
- return true;
- }
- }
- return false;
- }
- if (func.getFunctionIdentifier() != AlgebricksBuiltinFunctions.IS_MISSING) {
- return false;
- }
- return true;
+ return func2.getFunctionIdentifier().equals(isMissingNullFunId);
}
}
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/IntroduceGroupByForSubplanRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/IntroduceGroupByForSubplanRule.java
index c1e613b..391d03a 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/IntroduceGroupByForSubplanRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/IntroduceGroupByForSubplanRule.java
@@ -65,6 +65,8 @@
import org.apache.hyracks.api.exceptions.SourceLocation;
/**
+ * NOTE: This rule is currently not used
+ *
* The rule searches for SUBPLAN operator with a optional PROJECT operator and
* an AGGREGATE followed by a join operator.
*
@@ -258,8 +260,7 @@
ScalarFunctionCallExpression nonNullTest =
new ScalarFunctionCallExpression(finfoNot, new MutableObject<ILogicalExpression>(isNullTest));
nonNullTest.setSourceLocation(sourceLoc);
- SelectOperator selectNonNull =
- new SelectOperator(new MutableObject<ILogicalExpression>(nonNullTest), false, null);
+ SelectOperator selectNonNull = new SelectOperator(new MutableObject<ILogicalExpression>(nonNullTest));
selectNonNull.setSourceLocation(sourceLoc);
GroupByOperator g = new GroupByOperator();
g.setSourceLocation(sourceLoc);
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/IntroduceLeftOuterJoinForSubplanRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/IntroduceLeftOuterJoinForSubplanRule.java
index e37be1b..d7f8d38 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/IntroduceLeftOuterJoinForSubplanRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/IntroduceLeftOuterJoinForSubplanRule.java
@@ -22,17 +22,26 @@
import org.apache.commons.lang3.mutable.Mutable;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalPlan;
import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.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.operators.logical.AbstractLogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
-import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+/**
+ * NOTE: This rule is currently not used
+ *
+ */
public class IntroduceLeftOuterJoinForSubplanRule implements IAlgebraicRewriteRule {
@Override
@@ -65,7 +74,7 @@
AbstractLogicalOperator op1 = (AbstractLogicalOperator) subplanRoot.getValue();
Mutable<ILogicalOperator> opUnder = subplan.getInputs().get(0);
- if (OperatorPropertiesUtil.isMissingTest((AbstractLogicalOperator) opUnder.getValue())) {
+ if (isMissingTest((AbstractLogicalOperator) opUnder.getValue())) {
return false;
}
@@ -87,7 +96,8 @@
}
}
ntsRef.setValue(opUnder.getValue());
- LeftOuterJoinOperator loj = new LeftOuterJoinOperator(join.getCondition());
+ LeftOuterJoinOperator loj =
+ new LeftOuterJoinOperator(join.getCondition(), ConstantExpression.MISSING.getValue());
loj.setSourceLocation(join.getSourceLocation());
loj.getInputs().add(leftRef);
loj.getInputs().add(rightRef);
@@ -124,4 +134,30 @@
return getNtsAtEndOfPipeline(op.getInputs().get(0));
}
+ public static boolean isMissingTest(AbstractLogicalOperator op) {
+ if (op.getOperatorTag() != LogicalOperatorTag.SELECT) {
+ return false;
+ }
+ AbstractLogicalOperator doubleUnder = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
+ if (doubleUnder.getOperatorTag() != LogicalOperatorTag.NESTEDTUPLESOURCE) {
+ return false;
+ }
+ ILogicalExpression eu = ((SelectOperator) op).getCondition().getValue();
+ if (eu.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+ return false;
+ }
+ AbstractFunctionCallExpression f1 = (AbstractFunctionCallExpression) eu;
+ if (!f1.getFunctionIdentifier().equals(AlgebricksBuiltinFunctions.NOT)) {
+ return false;
+ }
+ ILogicalExpression a1 = f1.getArguments().get(0).getValue();
+ if (!a1.getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL)) {
+ return false;
+ }
+ AbstractFunctionCallExpression f2 = (AbstractFunctionCallExpression) a1;
+ if (!f2.getFunctionIdentifier().equals(AlgebricksBuiltinFunctions.IS_MISSING)) {
+ return false;
+ }
+ return true;
+ }
}
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/SubplanOutOfGroupRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/SubplanOutOfGroupRule.java
index d3ecf69..428b710 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/SubplanOutOfGroupRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/SubplanOutOfGroupRule.java
@@ -25,19 +25,25 @@
import org.apache.commons.lang3.mutable.Mutable;
import org.apache.commons.lang3.mutable.MutableObject;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalPlan;
import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
-import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
/**
+ * NOTE: This rule is currently not used
+ *
* Looks for a nested group-by plan ending in
* subplan {
* ...
@@ -82,7 +88,7 @@
if (op1.getOperatorTag() == LogicalOperatorTag.SUBPLAN) {
SubplanOperator subplan = (SubplanOperator) op1;
AbstractLogicalOperator op2 = (AbstractLogicalOperator) subplan.getInputs().get(0).getValue();
- if (OperatorPropertiesUtil.isMissingTest(op2)) {
+ if (isMissingTest(op2)) {
if (subplan.getNestedPlans().size() == 1) {
ILogicalPlan p1 = subplan.getNestedPlans().get(0);
if (p1.getRoots().size() == 1) {
@@ -122,4 +128,31 @@
return true;
}
+
+ private static boolean isMissingTest(AbstractLogicalOperator op) {
+ if (op.getOperatorTag() != LogicalOperatorTag.SELECT) {
+ return false;
+ }
+ AbstractLogicalOperator doubleUnder = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
+ if (doubleUnder.getOperatorTag() != LogicalOperatorTag.NESTEDTUPLESOURCE) {
+ return false;
+ }
+ ILogicalExpression eu = ((SelectOperator) op).getCondition().getValue();
+ if (eu.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+ return false;
+ }
+ AbstractFunctionCallExpression f1 = (AbstractFunctionCallExpression) eu;
+ if (!f1.getFunctionIdentifier().equals(AlgebricksBuiltinFunctions.NOT)) {
+ return false;
+ }
+ ILogicalExpression a1 = f1.getArguments().get(0).getValue();
+ if (!a1.getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL)) {
+ return false;
+ }
+ AbstractFunctionCallExpression f2 = (AbstractFunctionCallExpression) a1;
+ if (!f2.getFunctionIdentifier().equals(AlgebricksBuiltinFunctions.IS_MISSING)) {
+ return false;
+ }
+ return true;
+ }
}
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/UnnestRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/UnnestRuntimeFactory.java
index 9ee5a48..1b42bb6 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/UnnestRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/UnnestRuntimeFactory.java
@@ -18,8 +18,6 @@
*/
package org.apache.hyracks.algebricks.runtime.operators.std;
-import java.io.ByteArrayOutputStream;
-import java.io.DataOutput;
import java.io.DataOutputStream;
import java.io.IOException;
import java.nio.ByteBuffer;
@@ -38,6 +36,7 @@
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ByteArrayAccessibleOutputStream;
import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
public class UnnestRuntimeFactory extends AbstractOneInputOneOutputRuntimeFactory {
@@ -83,14 +82,7 @@
@Override
public AbstractOneInputOneOutputOneFramePushRuntime createOneOutputPushRuntime(final IHyracksTaskContext ctx)
throws HyracksDataException {
- ByteArrayOutputStream bos = new ByteArrayOutputStream();
- DataOutput output = new DataOutputStream(bos);
- if (missingWriterFactory != null) {
- IMissingWriter missingWriter = missingWriterFactory.createMissingWriter();
- missingWriter.writeMissing(output);
- }
- byte[] missingBytes = bos.toByteArray();
- int missingBytesLen = bos.size();
+ ByteArrayAccessibleOutputStream missingBytes = leftOuter ? writeMissingBytes() : null;
IEvaluatorContext evalCtx = new EvaluatorContext(ctx);
return new AbstractOneInputOneOutputOneFramePushRuntime() {
private IPointable p = VoidPointable.FACTORY.createPointable();
@@ -154,7 +146,7 @@
}
if (unnestColIsProjected) {
if (missing) {
- tupleBuilder.addField(missingBytes, 0, missingBytesLen);
+ tupleBuilder.addField(missingBytes.getByteArray(), 0, missingBytes.size());
} else {
tupleBuilder.addField(p.getByteArray(), p.getStartOffset(), p.getLength());
}
@@ -166,7 +158,7 @@
if (positionWriter != null) {
// Write the positional variable
if (missing) {
- tupleBuilder.addField(missingBytes, 0, missingBytesLen);
+ tupleBuilder.addField(missingBytes.getByteArray(), 0, missingBytes.size());
} else {
positionWriter.write(tupleBuilder.getDataOutput(), positionIndex);
tupleBuilder.addFieldEndOffset();
@@ -181,4 +173,11 @@
}
};
}
+
+ private ByteArrayAccessibleOutputStream writeMissingBytes() throws HyracksDataException {
+ ByteArrayAccessibleOutputStream baos = new ByteArrayAccessibleOutputStream();
+ IMissingWriter missingWriter = missingWriterFactory.createMissingWriter();
+ missingWriter.writeMissing(new DataOutputStream(baos));
+ return baos;
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/org/apache/hyracks/examples/btree/client/PrimaryIndexSearchExample.java b/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/org/apache/hyracks/examples/btree/client/PrimaryIndexSearchExample.java
index ab3db7d..20e8eae 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/org/apache/hyracks/examples/btree/client/PrimaryIndexSearchExample.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/org/apache/hyracks/examples/btree/client/PrimaryIndexSearchExample.java
@@ -143,7 +143,7 @@
new IndexDataflowHelperFactory(storageManager, btreeSplitProvider);
BTreeSearchOperatorDescriptor btreeSearchOp = new BTreeSearchOperatorDescriptor(spec, recDesc, lowKeyFields,
highKeyFields, true, true, dataflowHelperFactory, false, false, null,
- NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+ NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
JobHelper.createPartitionConstraint(spec, btreeSearchOp, splitNCs);
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/org/apache/hyracks/examples/btree/client/SecondaryIndexSearchExample.java b/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/org/apache/hyracks/examples/btree/client/SecondaryIndexSearchExample.java
index 44b982b..26ddbd6 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/org/apache/hyracks/examples/btree/client/SecondaryIndexSearchExample.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/org/apache/hyracks/examples/btree/client/SecondaryIndexSearchExample.java
@@ -170,7 +170,7 @@
new IndexDataflowHelperFactory(storageManager, secondarySplitProvider);
BTreeSearchOperatorDescriptor secondarySearchOp = new BTreeSearchOperatorDescriptor(spec, secondaryRecDesc,
secondaryLowKeyFields, secondaryHighKeyFields, true, true, secondaryHelperFactory, false, false, null,
- NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+ NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
JobHelper.createPartitionConstraint(spec, secondarySearchOp, splitNCs);
@@ -188,7 +188,7 @@
new IndexDataflowHelperFactory(storageManager, primarySplitProvider);
BTreeSearchOperatorDescriptor primarySearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
primaryLowKeyFields, primaryHighKeyFields, true, true, primaryHelperFactory, false, false, null,
- NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+ NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
JobHelper.createPartitionConstraint(spec, primarySearchOp, splitNCs);
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/AbstractBTreeOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/AbstractBTreeOperatorTest.java
index 283b352..eac332d 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/AbstractBTreeOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/AbstractBTreeOperatorTest.java
@@ -185,7 +185,7 @@
// scan primary index
BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
lowKeyFields, highKeyFields, true, true, primaryHelperFactory, false, false, null,
- NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+ NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
// sort based on secondary keys
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreePrimaryIndexScanOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreePrimaryIndexScanOperatorTest.java
index e4214cc..9474b12 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreePrimaryIndexScanOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreePrimaryIndexScanOperatorTest.java
@@ -76,7 +76,7 @@
BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
DataSetConstants.primaryRecDesc, lowKeyFields, highKeyFields, true, true, primaryHelperFactory, false,
- false, null, NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+ false, null, NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { createFile(nc1) });
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreePrimaryIndexSearchOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreePrimaryIndexSearchOperatorTest.java
index 65bb2f0..af043ab 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreePrimaryIndexSearchOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreePrimaryIndexSearchOperatorTest.java
@@ -81,7 +81,7 @@
BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
DataSetConstants.primaryRecDesc, lowKeyFields, highKeyFields, true, true, primaryHelperFactory, false,
- false, null, NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+ false, null, NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { createFile(nc1) });
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreeSecondaryIndexInsertOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreeSecondaryIndexInsertOperatorTest.java
index b489474..93ab78d 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreeSecondaryIndexInsertOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreeSecondaryIndexInsertOperatorTest.java
@@ -83,9 +83,10 @@
int[] secondaryHighKeyFields = { 1 };
// search secondary index
- BTreeSearchOperatorDescriptor secondaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
- DataSetConstants.secondaryRecDesc, secondaryLowKeyFields, secondaryHighKeyFields, true, true,
- secondaryHelperFactory, false, false, null, NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+ BTreeSearchOperatorDescriptor secondaryBtreeSearchOp =
+ new BTreeSearchOperatorDescriptor(spec, DataSetConstants.secondaryRecDesc, secondaryLowKeyFields,
+ secondaryHighKeyFields, true, true, secondaryHelperFactory, false, false, null,
+ NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryBtreeSearchOp, NC1_ID);
@@ -95,9 +96,10 @@
int[] primaryHighKeyFields = { 1 };
// search primary index
- BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
- DataSetConstants.primaryRecDesc, primaryLowKeyFields, primaryHighKeyFields, true, true,
- primaryHelperFactory, false, false, null, NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+ BTreeSearchOperatorDescriptor primaryBtreeSearchOp =
+ new BTreeSearchOperatorDescriptor(spec, DataSetConstants.primaryRecDesc, primaryLowKeyFields,
+ primaryHighKeyFields, true, true, primaryHelperFactory, false, false, null,
+ NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { createFile(nc1) });
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreeSecondaryIndexSearchOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreeSecondaryIndexSearchOperatorTest.java
index a1d1502..a5a0bda 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreeSecondaryIndexSearchOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreeSecondaryIndexSearchOperatorTest.java
@@ -82,9 +82,10 @@
int[] secondaryHighKeyFields = { 1 };
// search secondary index
- BTreeSearchOperatorDescriptor secondaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
- DataSetConstants.secondaryRecDesc, secondaryLowKeyFields, secondaryHighKeyFields, true, true,
- secondaryHelperFactory, false, false, null, NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+ BTreeSearchOperatorDescriptor secondaryBtreeSearchOp =
+ new BTreeSearchOperatorDescriptor(spec, DataSetConstants.secondaryRecDesc, secondaryLowKeyFields,
+ secondaryHighKeyFields, true, true, secondaryHelperFactory, false, false, null,
+ NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryBtreeSearchOp, NC1_ID);
@@ -94,9 +95,10 @@
// coming from secondary index
// search primary index
- BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
- DataSetConstants.primaryRecDesc, primaryLowKeyFields, primaryHighKeyFields, true, true,
- primaryHelperFactory, false, false, null, NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+ BTreeSearchOperatorDescriptor primaryBtreeSearchOp =
+ new BTreeSearchOperatorDescriptor(spec, DataSetConstants.primaryRecDesc, primaryLowKeyFields,
+ primaryHighKeyFields, true, true, primaryHelperFactory, false, false, null,
+ NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { createFile(nc1) });
IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreeSecondaryIndexUpsertOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreeSecondaryIndexUpsertOperatorTest.java
index f9cdfcf..3691cd4 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreeSecondaryIndexUpsertOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreeSecondaryIndexUpsertOperatorTest.java
@@ -82,9 +82,10 @@
int[] secondaryHighKeyFields = { 1 };
// search secondary index
- BTreeSearchOperatorDescriptor secondaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
- DataSetConstants.secondaryRecDesc, secondaryLowKeyFields, secondaryHighKeyFields, true, true,
- secondaryHelperFactory, false, false, null, NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+ BTreeSearchOperatorDescriptor secondaryBtreeSearchOp =
+ new BTreeSearchOperatorDescriptor(spec, DataSetConstants.secondaryRecDesc, secondaryLowKeyFields,
+ secondaryHighKeyFields, true, true, secondaryHelperFactory, false, false, null,
+ NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryBtreeSearchOp, NC1_ID);
@@ -94,9 +95,10 @@
int[] primaryHighKeyFields = { 1 };
// search primary index
- BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
- DataSetConstants.primaryRecDesc, primaryLowKeyFields, primaryHighKeyFields, true, true,
- primaryHelperFactory, false, false, null, NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+ BTreeSearchOperatorDescriptor primaryBtreeSearchOp =
+ new BTreeSearchOperatorDescriptor(spec, DataSetConstants.primaryRecDesc, primaryLowKeyFields,
+ primaryHighKeyFields, true, true, primaryHelperFactory, false, false, null,
+ NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { createFile(nc1) });
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreePrimaryIndexSearchOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreePrimaryIndexSearchOperatorTest.java
index 9d3ba5c..da1441c 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreePrimaryIndexSearchOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreePrimaryIndexSearchOperatorTest.java
@@ -89,10 +89,10 @@
int[] lowKeyFields = { 0 };
int[] highKeyFields = { 1 };
- BTreeSearchOperatorDescriptor primaryBtreeSearchOp =
- new BTreeSearchOperatorDescriptor(spec, DataSetConstants.primaryAndFilterRecDesc, lowKeyFields,
- highKeyFields, true, true, primaryHelperFactory, false, false,
- NoopMissingWriterFactory.INSTANCE, NoOpOperationCallbackFactory.INSTANCE, null, null, true);
+ BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
+ DataSetConstants.primaryAndFilterRecDesc, lowKeyFields, highKeyFields, true, true, primaryHelperFactory,
+ false, false, NoopMissingWriterFactory.INSTANCE, NoOpOperationCallbackFactory.INSTANCE, null, null,
+ true, NoopMissingWriterFactory.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { createFile(nc1) });
@@ -147,7 +147,7 @@
BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
DataSetConstants.primaryAndFilterRecDesc, lowKeyFields, highKeyFields, true, true, primaryHelperFactory,
false, false, NoopMissingWriterFactory.INSTANCE, NoOpOperationCallbackFactory.INSTANCE, minFilterFields,
- maxFilterFields, true);
+ maxFilterFields, true, NoopMissingWriterFactory.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { createFile(nc1) });
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeSecondaryIndexSearchOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeSecondaryIndexSearchOperatorTest.java
index 97a4746..bed4794 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeSecondaryIndexSearchOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeSecondaryIndexSearchOperatorTest.java
@@ -90,10 +90,10 @@
int[] secondaryHighKeyFields = { 1 };
// search secondary index
- BTreeSearchOperatorDescriptor secondaryBtreeSearchOp =
- new BTreeSearchOperatorDescriptor(spec, DataSetConstants.secondaryWithFilterRecDesc,
- secondaryLowKeyFields, secondaryHighKeyFields, true, true, secondaryHelperFactory, false, false,
- NoopMissingWriterFactory.INSTANCE, NoOpOperationCallbackFactory.INSTANCE, null, null, true);
+ BTreeSearchOperatorDescriptor secondaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
+ DataSetConstants.secondaryWithFilterRecDesc, secondaryLowKeyFields, secondaryHighKeyFields, true, true,
+ secondaryHelperFactory, false, false, NoopMissingWriterFactory.INSTANCE,
+ NoOpOperationCallbackFactory.INSTANCE, null, null, true, NoopMissingWriterFactory.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryBtreeSearchOp, NC1_ID);
@@ -103,10 +103,11 @@
int[] maxFilterFields = { 3 };
// search primary index
- BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
- DataSetConstants.primaryRecDesc, primaryLowKeyFields, primaryHighKeyFields, true, true,
- primaryHelperFactory, false, false, NoopMissingWriterFactory.INSTANCE,
- NoOpOperationCallbackFactory.INSTANCE, minFilterFields, maxFilterFields, false);
+ BTreeSearchOperatorDescriptor primaryBtreeSearchOp =
+ new BTreeSearchOperatorDescriptor(spec, DataSetConstants.primaryRecDesc, primaryLowKeyFields,
+ primaryHighKeyFields, true, true, primaryHelperFactory, false, false,
+ NoopMissingWriterFactory.INSTANCE, NoOpOperationCallbackFactory.INSTANCE, minFilterFields,
+ maxFilterFields, false, NoopMissingWriterFactory.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { createFile(nc1) });
IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesSecondaryIndexSearchOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesSecondaryIndexSearchOperatorTest.java
index 04179c2..6a31962 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesSecondaryIndexSearchOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesSecondaryIndexSearchOperatorTest.java
@@ -95,7 +95,7 @@
int[] keyFields = { 0, 1, 2, 3 };
RTreeSearchOperatorDescriptor secondarySearchOp = new RTreeSearchOperatorDescriptor(spec,
secondaryWithFilterRecDesc, keyFields, true, true, secondaryHelperFactory, false, false, null,
- NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+ NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondarySearchOp, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { createFile(nc1) });
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/AbstractRTreeOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/AbstractRTreeOperatorTest.java
index 872502b..f849bc8 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/AbstractRTreeOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/AbstractRTreeOperatorTest.java
@@ -320,7 +320,7 @@
// scan primary index
BTreeSearchOperatorDescriptor primarySearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
lowKeyFields, highKeyFields, true, true, primaryHelperFactory, false, false, null,
- NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+ NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primarySearchOp, NC1_ID);
// load secondary index
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeSecondaryIndexInsertOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeSecondaryIndexInsertOperatorTest.java
index 784cd10..0fcf892 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeSecondaryIndexInsertOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeSecondaryIndexInsertOperatorTest.java
@@ -88,7 +88,7 @@
int[] keyFields = { 0, 1, 2, 3 };
RTreeSearchOperatorDescriptor secondarySearchOp =
new RTreeSearchOperatorDescriptor(spec, secondaryRecDesc, keyFields, true, true, secondaryHelperFactory,
- false, false, null, NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+ false, false, null, NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondarySearchOp, NC1_ID);
// fifth field from the tuples coming from secondary index
int[] primaryLowKeyFields = { 4 };
@@ -97,7 +97,7 @@
// search primary index
BTreeSearchOperatorDescriptor primarySearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
primaryLowKeyFields, primaryHighKeyFields, true, true, primaryHelperFactory, false, false, null,
- NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+ NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primarySearchOp, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { createFile(nc1) });
IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeSecondaryIndexScanOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeSecondaryIndexScanOperatorTest.java
index ff44d10..3cfef19 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeSecondaryIndexScanOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeSecondaryIndexScanOperatorTest.java
@@ -86,7 +86,7 @@
int[] keyFields = null;
RTreeSearchOperatorDescriptor secondarySearchOp =
new RTreeSearchOperatorDescriptor(spec, secondaryRecDesc, keyFields, true, true, secondaryHelperFactory,
- false, false, null, NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+ false, false, null, NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondarySearchOp, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { createFile(nc1) });
IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeSecondaryIndexSearchOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeSecondaryIndexSearchOperatorTest.java
index e234322..8ea0701 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeSecondaryIndexSearchOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeSecondaryIndexSearchOperatorTest.java
@@ -86,7 +86,7 @@
int[] keyFields = { 0, 1, 2, 3 };
RTreeSearchOperatorDescriptor secondarySearchOp =
new RTreeSearchOperatorDescriptor(spec, secondaryRecDesc, keyFields, true, true, secondaryHelperFactory,
- false, false, null, NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+ false, false, null, NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondarySearchOp, NC1_ID);
// fifth field from the tuples coming from secondary index
int[] primaryLowKeyFields = { 4 };
@@ -95,7 +95,7 @@
// search primary index
BTreeSearchOperatorDescriptor primarySearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
primaryLowKeyFields, primaryHighKeyFields, true, true, primaryHelperFactory, false, false, null,
- NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+ NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primarySearchOp, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { createFile(nc1) });
IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
index 128929c..2455625 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
@@ -46,6 +46,7 @@
protected final IMissingWriterFactory missingWriterFactory;
protected final ISearchOperationCallbackFactory searchCallbackFactory;
protected final boolean appendIndexFilter;
+ protected final IMissingWriterFactory nonFilterWriterFactory;
protected boolean appendOpCallbackProceedResult;
protected byte[] searchCallbackProceedResultFalseValue;
protected byte[] searchCallbackProceedResultTrueValue;
@@ -56,10 +57,11 @@
int[] lowKeyFields, int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
IIndexDataflowHelperFactory indexHelperFactory, boolean retainInput, boolean retainMissing,
IMissingWriterFactory missingWriterFactory, ISearchOperationCallbackFactory searchCallbackFactory,
- int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, boolean appendIndexFilter) {
+ int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, boolean appendIndexFilter,
+ IMissingWriterFactory nonFilterWriterFactory) {
this(spec, outRecDesc, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory,
retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, minFilterFieldIndexes,
- maxFilterFieldIndexes, appendIndexFilter, null, -1, false, null, null);
+ maxFilterFieldIndexes, appendIndexFilter, nonFilterWriterFactory, null, -1, false, null, null);
}
public BTreeSearchOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor outRecDesc,
@@ -67,8 +69,9 @@
IIndexDataflowHelperFactory indexHelperFactory, boolean retainInput, boolean retainMissing,
IMissingWriterFactory missingWriterFactory, ISearchOperationCallbackFactory searchCallbackFactory,
int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, boolean appendIndexFilter,
- ITupleFilterFactory tupleFilterFactory, long outputLimit, boolean appendOpCallbackProceedResult,
- byte[] searchCallbackProceedResultFalseValue, byte[] searchCallbackProceedResultTrueValue) {
+ IMissingWriterFactory nonFilterWriterFactory, ITupleFilterFactory tupleFilterFactory, long outputLimit,
+ boolean appendOpCallbackProceedResult, byte[] searchCallbackProceedResultFalseValue,
+ byte[] searchCallbackProceedResultTrueValue) {
super(spec, 1, 1);
this.indexHelperFactory = indexHelperFactory;
this.retainInput = retainInput;
@@ -82,6 +85,7 @@
this.minFilterFieldIndexes = minFilterFieldIndexes;
this.maxFilterFieldIndexes = maxFilterFieldIndexes;
this.appendIndexFilter = appendIndexFilter;
+ this.nonFilterWriterFactory = nonFilterWriterFactory;
this.outRecDescs[0] = outRecDesc;
this.tupleFilterFactory = tupleFilterFactory;
this.outputLimit = outputLimit;
@@ -97,8 +101,8 @@
recordDescProvider.getInputRecordDescriptor(getActivityId(), 0), lowKeyFields, highKeyFields,
lowKeyInclusive, highKeyInclusive, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter,
- tupleFilterFactory, outputLimit, appendOpCallbackProceedResult, searchCallbackProceedResultFalseValue,
- searchCallbackProceedResultTrueValue);
+ nonFilterWriterFactory, tupleFilterFactory, outputLimit, appendOpCallbackProceedResult,
+ searchCallbackProceedResultFalseValue, searchCallbackProceedResultTrueValue);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
index 77ef1dc..fb331bf 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
@@ -46,26 +46,27 @@
public BTreeSearchOperatorNodePushable(IHyracksTaskContext ctx, int partition, RecordDescriptor inputRecDesc,
int[] lowKeyFields, int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, IIndexDataflowHelperFactory indexHelperFactory,
- boolean retainInput, boolean retainMissing, IMissingWriterFactory missingWriterFactory,
- ISearchOperationCallbackFactory searchCallbackFactory, boolean appendIndexFilter)
- throws HyracksDataException {
+ boolean retainInput, boolean retainMissing, IMissingWriterFactory nonMatchWriterFactory,
+ ISearchOperationCallbackFactory searchCallbackFactory, boolean appendIndexFilter,
+ IMissingWriterFactory nonFilterWriterFactory) throws HyracksDataException {
this(ctx, partition, inputRecDesc, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive,
minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory, retainInput, retainMissing,
- missingWriterFactory, searchCallbackFactory, appendIndexFilter, null, -1, false, null, null);
+ nonMatchWriterFactory, searchCallbackFactory, appendIndexFilter, nonFilterWriterFactory, null, -1,
+ false, null, null);
}
public BTreeSearchOperatorNodePushable(IHyracksTaskContext ctx, int partition, RecordDescriptor inputRecDesc,
int[] lowKeyFields, int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, IIndexDataflowHelperFactory indexHelperFactory,
- boolean retainInput, boolean retainMissing, IMissingWriterFactory missingWriterFactory,
+ boolean retainInput, boolean retainMissing, IMissingWriterFactory nonMatchWriterFactory,
ISearchOperationCallbackFactory searchCallbackFactory, boolean appendIndexFilter,
- ITupleFilterFactory tupleFilterFactory, long outputLimit, boolean appendOpCallbackProceedResult,
- byte[] searchCallbackProceedResultFalseValue, byte[] searchCallbackProceedResultTrueValue)
- throws HyracksDataException {
+ IMissingWriterFactory nonFilterWriterFactory, ITupleFilterFactory tupleFilterFactory, long outputLimit,
+ boolean appendOpCallbackProceedResult, byte[] searchCallbackProceedResultFalseValue,
+ byte[] searchCallbackProceedResultTrueValue) throws HyracksDataException {
super(ctx, inputRecDesc, partition, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
- retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter,
- tupleFilterFactory, outputLimit, appendOpCallbackProceedResult, searchCallbackProceedResultFalseValue,
- searchCallbackProceedResultTrueValue);
+ retainInput, retainMissing, nonMatchWriterFactory, searchCallbackFactory, appendIndexFilter,
+ nonFilterWriterFactory, tupleFilterFactory, outputLimit, appendOpCallbackProceedResult,
+ searchCallbackProceedResultFalseValue, searchCallbackProceedResultTrueValue);
this.lowKeyInclusive = lowKeyInclusive;
this.highKeyInclusive = highKeyInclusive;
if (lowKeyFields != null && lowKeyFields.length > 0) {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorDescriptor.java
index e6637db..f8402e0 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorDescriptor.java
@@ -20,6 +20,7 @@
package org.apache.hyracks.storage.am.btree.dataflow;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider;
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -38,9 +39,10 @@
int[] lowKeyFields, int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
IIndexDataflowHelperFactory dataflowHelperFactory, boolean retainInput,
ISearchOperationCallbackFactory searchOpCallbackProvider, ITupleUpdaterFactory tupleUpdaterFactory,
- boolean appendIndexFilter) {
+ boolean appendIndexFilter, IMissingWriterFactory nonFilterWriterFactory) {
super(spec, outRecDesc, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive, dataflowHelperFactory,
- retainInput, false, null, searchOpCallbackProvider, null, null, appendIndexFilter);
+ retainInput, false, null, searchOpCallbackProvider, null, null, appendIndexFilter,
+ nonFilterWriterFactory);
this.tupleUpdaterFactory = tupleUpdaterFactory;
}
@@ -50,6 +52,7 @@
return new BTreeUpdateSearchOperatorNodePushable(ctx, partition,
recordDescProvider.getInputRecordDescriptor(getActivityId(), 0), lowKeyFields, highKeyFields,
lowKeyInclusive, highKeyInclusive, indexHelperFactory, retainInput, retainMissing, missingWriterFactory,
- searchCallbackFactory, tupleUpdaterFactory.createTupleUpdater(), appendIndexFilter);
+ searchCallbackFactory, tupleUpdaterFactory.createTupleUpdater(), appendIndexFilter,
+ nonFilterWriterFactory);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java
index a296672..a102e39 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java
@@ -41,10 +41,11 @@
int[] lowKeyFields, int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
IIndexDataflowHelperFactory indexHelperFactory, boolean retainInput, boolean retainMissing,
IMissingWriterFactory missingWriterFactory, ISearchOperationCallbackFactory searchCallbackFactory,
- ITupleUpdater tupleUpdater, boolean appendIndexFilter) throws HyracksDataException {
+ ITupleUpdater tupleUpdater, boolean appendIndexFilter, IMissingWriterFactory nonFilterWriterFactory)
+ throws HyracksDataException {
super(ctx, partition, inputRecDesc, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive, null, null,
indexHelperFactory, retainInput, retainMissing, missingWriterFactory, searchCallbackFactory,
- appendIndexFilter);
+ appendIndexFilter, nonFilterWriterFactory);
this.tupleUpdater = tupleUpdater;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/test/java/org/apache/hyracks/storage/am/btree/test/FramewriterTest.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/test/java/org/apache/hyracks/storage/am/btree/test/FramewriterTest.java
index 0600add..b42337c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/test/java/org/apache/hyracks/storage/am/btree/test/FramewriterTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/test/java/org/apache/hyracks/storage/am/btree/test/FramewriterTest.java
@@ -295,7 +295,7 @@
recordDescProvider.getInputRecordDescriptor(new ActivityId(new OperatorDescriptorId(0), 0),
0),
keys, keys, lowKeyInclusive, highKeyInclusive, keys, keys, pair.getLeft(), false, false,
- null, pair.getRight(), false);
+ null, pair.getRight(), false, null);
writers.add(writer);
}
}
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 a5bc206..eab8c96 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
@@ -89,6 +89,7 @@
protected PermutingFrameTupleReference maxFilterKey;
protected final boolean appendIndexFilter;
protected ArrayTupleBuilder nonFilterTupleBuild;
+ protected IMissingWriter nonFilterWriter;
protected final ISearchOperationCallbackFactory searchCallbackFactory;
protected boolean failed = false;
protected IOperatorStats stats;
@@ -110,28 +111,31 @@
// no filter and limit pushdown
public IndexSearchOperatorNodePushable(IHyracksTaskContext ctx, RecordDescriptor inputRecDesc, int partition,
int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, IIndexDataflowHelperFactory indexHelperFactory,
- boolean retainInput, boolean retainMissing, IMissingWriterFactory missingWriterFactory,
- ISearchOperationCallbackFactory searchCallbackFactory, boolean appendIndexFilter)
- throws HyracksDataException {
+ boolean retainInput, boolean retainMissing, IMissingWriterFactory nonMatchWriterFactory,
+ ISearchOperationCallbackFactory searchCallbackFactory, boolean appendIndexFilter,
+ IMissingWriterFactory nonFilterWriterFactory) throws HyracksDataException {
this(ctx, inputRecDesc, partition, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
- retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter, null, -1,
- false, null, null);
+ retainInput, retainMissing, nonMatchWriterFactory, searchCallbackFactory, appendIndexFilter,
+ nonFilterWriterFactory, null, -1, false, null, null);
}
public IndexSearchOperatorNodePushable(IHyracksTaskContext ctx, RecordDescriptor inputRecDesc, int partition,
int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, IIndexDataflowHelperFactory indexHelperFactory,
- boolean retainInput, boolean retainMissing, IMissingWriterFactory missingWriterFactory,
+ boolean retainInput, boolean retainMissing, IMissingWriterFactory nonMatchWriterFactory,
ISearchOperationCallbackFactory searchCallbackFactory, boolean appendIndexFilter,
- ITupleFilterFactory tupleFactoryFactory, long outputLimit, boolean appendSearchCallbackProceedResult,
- byte[] searchCallbackProceedResultFalseValue, byte[] searchCallbackProceedResultTrueValue)
- throws HyracksDataException {
+ IMissingWriterFactory nonFilterWriterFactory, ITupleFilterFactory tupleFactoryFactory, long outputLimit,
+ boolean appendSearchCallbackProceedResult, byte[] searchCallbackProceedResultFalseValue,
+ byte[] searchCallbackProceedResultTrueValue) throws HyracksDataException {
this.ctx = ctx;
this.indexHelper = indexHelperFactory.create(ctx.getJobletContext().getServiceContext(), partition);
this.retainInput = retainInput;
this.retainMissing = retainMissing;
this.appendIndexFilter = appendIndexFilter;
- if (this.retainMissing || this.appendIndexFilter) {
- this.nonMatchWriter = missingWriterFactory.createMissingWriter();
+ if (this.retainMissing) {
+ this.nonMatchWriter = nonMatchWriterFactory.createMissingWriter();
+ }
+ if (this.appendIndexFilter) {
+ this.nonFilterWriter = nonFilterWriterFactory.createMissingWriter();
}
this.inputRecDesc = inputRecDesc;
this.searchCallbackFactory = searchCallbackFactory;
@@ -197,7 +201,7 @@
if (appendIndexFilter) {
int numIndexFilterFields = index.getNumOfFilterFields();
nonFilterTupleBuild = new ArrayTupleBuilder(numIndexFilterFields);
- buildMissingTuple(numIndexFilterFields, nonFilterTupleBuild, nonMatchWriter);
+ buildMissingTuple(numIndexFilterFields, nonFilterTupleBuild, nonFilterWriter);
}
if (tupleFilterFactory != null) {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorDescriptor.java
index 23be280..3800d17 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorDescriptor.java
@@ -41,7 +41,7 @@
long outputLimit) {
super(spec, outRecDesc, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory,
retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, minFilterFieldIndexes,
- maxFilterFieldIndexes, false, tupleFilterFactory, outputLimit, false, null, null);
+ maxFilterFieldIndexes, false, null, tupleFilterFactory, outputLimit, false, null, null);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorNodePushable.java
index dddaab1..30813ef 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorNodePushable.java
@@ -51,7 +51,8 @@
ITupleFilterFactory tupleFilterFactory, long outputLimit) throws HyracksDataException {
super(ctx, partition, inputRecDesc, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive,
minFilterKeyFields, maxFilterKeyFields, indexHelperFactory, retainInput, retainMissing,
- missingWriterFactory, searchCallbackFactory, false, tupleFilterFactory, outputLimit, false, null, null);
+ missingWriterFactory, searchCallbackFactory, false, null, tupleFilterFactory, outputLimit, false, null,
+ null);
this.keyFields = lowKeyFields;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDiskComponentScanOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDiskComponentScanOperatorNodePushable.java
index 860d36e..4aa094b 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDiskComponentScanOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDiskComponentScanOperatorNodePushable.java
@@ -37,7 +37,7 @@
RecordDescriptor inputRecDesc, IIndexDataflowHelperFactory indexHelperFactory,
ISearchOperationCallbackFactory searchCallbackFactory) throws HyracksDataException {
super(ctx, inputRecDesc, partition, null, null, indexHelperFactory, false, false, null, searchCallbackFactory,
- false);
+ false, null);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorDescriptor.java
index 2cee54d..b5b951d 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorDescriptor.java
@@ -42,6 +42,7 @@
private final int[] minFilterFieldIndexes;
private final int[] maxFilterFieldIndexes;
private final boolean appendIndexFilter;
+ private final IMissingWriterFactory nonFilterWriterFactory;
private final boolean isFullTextSearchQuery;
private final IIndexDataflowHelperFactory indexHelperFactory;
private final IBinaryTokenizerFactory queryTokenizerFactory;
@@ -61,7 +62,7 @@
IInvertedIndexSearchModifierFactory searchModifierFactory, boolean retainInput, boolean retainMissing,
IMissingWriterFactory missingWriterFactory, ISearchOperationCallbackFactory searchCallbackFactory,
int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, boolean isFullTextSearchQuery, int numOfFields,
- boolean appendIndexFilter, int frameLimit) {
+ boolean appendIndexFilter, IMissingWriterFactory nonFilterWriterFactory, int frameLimit) {
super(spec, 1, 1);
this.indexHelperFactory = indexHelperFactory;
this.queryTokenizerFactory = queryTokenizerFactory;
@@ -76,6 +77,7 @@
this.maxFilterFieldIndexes = maxFilterFieldIndexes;
this.isFullTextSearchQuery = isFullTextSearchQuery;
this.appendIndexFilter = appendIndexFilter;
+ this.nonFilterWriterFactory = nonFilterWriterFactory;
this.numOfFields = numOfFields;
this.outRecDescs[0] = outRecDesc;
this.frameLimit = frameLimit;
@@ -89,6 +91,6 @@
recordDescProvider.getInputRecordDescriptor(getActivityId(), 0), partition, minFilterFieldIndexes,
maxFilterFieldIndexes, indexHelperFactory, retainInput, retainMissing, missingWriterFactory,
searchCallbackFactory, searchModifier, queryTokenizerFactory, fullTextConfigEvaluatorFactory,
- queryField, isFullTextSearchQuery, numOfFields, appendIndexFilter, frameLimit);
+ queryField, isFullTextSearchQuery, numOfFields, appendIndexFilter, nonFilterWriterFactory, frameLimit);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorNodePushable.java
index e4dbb43..571ae5c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorNodePushable.java
@@ -62,10 +62,11 @@
IMissingWriterFactory missingWriterFactory, ISearchOperationCallbackFactory searchCallbackFactory,
IInvertedIndexSearchModifier searchModifier, IBinaryTokenizerFactory binaryTokenizerFactory,
IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory, int queryFieldIndex,
- boolean isFullTextSearchQuery, int numOfFields, boolean appendIndexFilter, int frameLimit)
- throws HyracksDataException {
+ boolean isFullTextSearchQuery, int numOfFields, boolean appendIndexFilter,
+ IMissingWriterFactory nonFilterWriterFactory, int frameLimit) throws HyracksDataException {
super(ctx, inputRecDesc, partition, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
- retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter);
+ retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter,
+ nonFilterWriterFactory);
this.searchModifier = searchModifier;
this.binaryTokenizerFactory = binaryTokenizerFactory;
this.fullTextConfigEvaluatorFactory = fullTextConfigEvaluatorFactory;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java
index 46727cd..cc6d76d 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java
@@ -39,6 +39,7 @@
protected final int[] minFilterFieldIndexes;
protected final int[] maxFilterFieldIndexes;
protected final boolean appendIndexFilter;
+ protected final IMissingWriterFactory nonFilterWriterFactory;
protected final IIndexDataflowHelperFactory indexHelperFactory;
protected final boolean retainInput;
protected final boolean retainMissing;
@@ -52,18 +53,19 @@
boolean lowKeyInclusive, boolean highKeyInclusive, IIndexDataflowHelperFactory indexHelperFactory,
boolean retainInput, boolean retainMissing, IMissingWriterFactory missingWriterFactory,
ISearchOperationCallbackFactory searchCallbackFactory, int[] minFilterFieldIndexes,
- int[] maxFilterFieldIndexes, boolean appendIndexFilter) {
+ int[] maxFilterFieldIndexes, boolean appendIndexFilter, IMissingWriterFactory nonFilterWriterFactory) {
this(spec, outRecDesc, keyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory, retainInput,
retainMissing, missingWriterFactory, searchCallbackFactory, minFilterFieldIndexes,
- maxFilterFieldIndexes, appendIndexFilter, false, null, null);
+ maxFilterFieldIndexes, appendIndexFilter, nonFilterWriterFactory, false, null, null);
}
public RTreeSearchOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor outRecDesc, int[] keyFields,
boolean lowKeyInclusive, boolean highKeyInclusive, IIndexDataflowHelperFactory indexHelperFactory,
boolean retainInput, boolean retainMissing, IMissingWriterFactory missingWriterFactory,
ISearchOperationCallbackFactory searchCallbackFactory, int[] minFilterFieldIndexes,
- int[] maxFilterFieldIndexes, boolean appendIndexFilter, boolean appendOpCallbackProceedResult,
- byte[] searchCallbackProceedResultFalseValue, byte[] searchCallbackProceedResultTrueValue) {
+ int[] maxFilterFieldIndexes, boolean appendIndexFilter, IMissingWriterFactory nonFilterWriterFactory,
+ boolean appendOpCallbackProceedResult, byte[] searchCallbackProceedResultFalseValue,
+ byte[] searchCallbackProceedResultTrueValue) {
super(spec, 1, 1);
this.indexHelperFactory = indexHelperFactory;
this.retainInput = retainInput;
@@ -76,6 +78,7 @@
this.minFilterFieldIndexes = minFilterFieldIndexes;
this.maxFilterFieldIndexes = maxFilterFieldIndexes;
this.appendIndexFilter = appendIndexFilter;
+ this.nonFilterWriterFactory = nonFilterWriterFactory;
this.outRecDescs[0] = outRecDesc;
this.appendOpCallbackProceedResult = appendOpCallbackProceedResult;
this.searchCallbackProceedResultFalseValue = searchCallbackProceedResultFalseValue;
@@ -88,7 +91,7 @@
return new RTreeSearchOperatorNodePushable(ctx, partition,
recordDescProvider.getInputRecordDescriptor(getActivityId(), 0), keyFields, minFilterFieldIndexes,
maxFilterFieldIndexes, indexHelperFactory, retainInput, retainMissing, missingWriterFactory,
- searchCallbackFactory, appendIndexFilter, appendOpCallbackProceedResult,
+ searchCallbackFactory, appendIndexFilter, nonFilterWriterFactory, appendOpCallbackProceedResult,
searchCallbackProceedResultFalseValue, searchCallbackProceedResultTrueValue);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java
index 4d460b1..a8a4252 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java
@@ -42,23 +42,23 @@
int[] keyFields, int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes,
IIndexDataflowHelperFactory indexHelperFactory, boolean retainInput, boolean retainMissing,
IMissingWriterFactory missingWriterFactory, ISearchOperationCallbackFactory searchCallbackFactory,
- boolean appendIndexFilter) throws HyracksDataException {
+ boolean appendIndexFilter, IMissingWriterFactory nonFilterWriterFactory) throws HyracksDataException {
this(ctx, partition, inputRecDesc, keyFields, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
- retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter, false, null,
- null);
+ retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter,
+ nonFilterWriterFactory, false, null, null);
}
public RTreeSearchOperatorNodePushable(IHyracksTaskContext ctx, int partition, RecordDescriptor inputRecDesc,
int[] keyFields, int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes,
IIndexDataflowHelperFactory indexHelperFactory, boolean retainInput, boolean retainMissing,
IMissingWriterFactory missingWriterFactory, ISearchOperationCallbackFactory searchCallbackFactory,
- boolean appendIndexFilter, boolean appendOpCallbackProceedResult,
- byte[] searchCallbackProceedResultFalseValue, byte[] searchCallbackProceedResultTrueValue)
- throws HyracksDataException {
+ boolean appendIndexFilter, IMissingWriterFactory nonFilterWriterFactory,
+ boolean appendOpCallbackProceedResult, byte[] searchCallbackProceedResultFalseValue,
+ byte[] searchCallbackProceedResultTrueValue) throws HyracksDataException {
// TODO: predicate & limit pushdown not enabled for RTree yet
super(ctx, inputRecDesc, partition, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
- retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter, null, -1,
- appendOpCallbackProceedResult, searchCallbackProceedResultFalseValue,
+ retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter,
+ nonFilterWriterFactory, null, -1, appendOpCallbackProceedResult, searchCallbackProceedResultFalseValue,
searchCallbackProceedResultTrueValue);
if (keyFields != null && keyFields.length > 0) {
searchKey = new PermutingFrameTupleReference();