fixed issue 731, 740, and more
commit 8911cc529e72e2bb544d9b472d6e10f173d173af
Author: Young-Seok <kisskys@gmail.com>
Date: Sun May 18 11:28:28 2014 -0700
another fix for picking available index for leftouterjoin plan
commit 9bce43087615fee53613467a027833dd53e190f9
Merge: c8e85ac efab69f
Author: Young-Seok <kisskys@gmail.com>
Date: Sun May 11 22:22:10 2014 -0700
merged master to kisskys/left-outer-join-issue branch
commit c8e85aca31545c13b2a02ff6dc259943e2cf66ad
Author: Young-Seok <kisskys@gmail.com>
Date: Sun May 11 20:17:17 2014 -0700
changes for left-outer-join to pick available indexes
Change-Id: Ib0fc186bc9388802f95445edee92c428b3bb69cc
Reviewed-on: http://fulliautomatix.ics.uci.edu:8443/34
Reviewed-by: Inci Cetindil <icetindil@gmail.com>
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/SelectOperator.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/SelectOperator.java
index 0bae2c4..5aa229c 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/SelectOperator.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/SelectOperator.java
@@ -38,9 +38,13 @@
public class SelectOperator extends AbstractLogicalOperator {
private final Mutable<ILogicalExpression> condition;
+ private final boolean retainNull;
+ private final LogicalVariable nullPlaceholderVar;
- public SelectOperator(Mutable<ILogicalExpression> condition) {
+ public SelectOperator(Mutable<ILogicalExpression> condition, boolean retainNull, LogicalVariable nullPlaceholderVar) {
this.condition = condition;
+ this.retainNull = retainNull;
+ this.nullPlaceholderVar = nullPlaceholderVar;
}
@Override
@@ -52,6 +56,14 @@
return condition;
}
+ public boolean getRetainNull() {
+ return retainNull;
+ }
+
+ public LogicalVariable getNullPlaceholderVariable() throws AlgebricksException {
+ return nullPlaceholderVar;
+ }
+
@Override
public void recomputeSchema() {
schema = new ArrayList<LogicalVariable>(inputs.get(0).getValue().getSchema());
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
index 97d257a..3b761da 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
@@ -666,7 +666,7 @@
@Override
public ILogicalOperator visitSelectOperator(SelectOperator op, Void arg) throws AlgebricksException {
- return new SelectOperator(deepCopyExpressionRef(op.getCondition()));
+ return new SelectOperator(deepCopyExpressionRef(op.getCondition()), op.getRetainNull(), op.getNullPlaceholderVariable());
}
@Override
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StreamSelectPOperator.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StreamSelectPOperator.java
index 686b21c..bbf8e9a 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StreamSelectPOperator.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StreamSelectPOperator.java
@@ -63,7 +63,8 @@
IScalarEvaluatorFactory cond = expressionRuntimeProvider.createEvaluatorFactory(select.getCondition()
.getValue(), context.getTypeEnvironment(op), inputSchemas, context);
StreamSelectRuntimeFactory runtime = new StreamSelectRuntimeFactory(cond, null,
- context.getBinaryBooleanInspectorFactory());
+ context.getBinaryBooleanInspectorFactory(), select.getRetainNull(), inputSchemas[0].findVariable(select
+ .getNullPlaceholderVariable()), context.getNullWriterFactory());
// contribute one Asterix framewriter
RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema, context);
builder.contributeMicroOperator(select, runtime, recDesc);
diff --git a/algebricks/algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/compiler/PigletCompiler.java b/algebricks/algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/compiler/PigletCompiler.java
index 89b2810..820d016 100644
--- a/algebricks/algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/compiler/PigletCompiler.java
+++ b/algebricks/algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/compiler/PigletCompiler.java
@@ -247,7 +247,7 @@
Pair<Relation, LogicalVariable> tempInput = translateScalarExpression(inputRel, conditionNode);
Relation rel = new Relation();
rel.op = new SelectOperator(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(
- tempInput.second)));
+ tempInput.second)), false, null);
rel.op.getInputs().add(new MutableObject<ILogicalOperator>(tempInput.first.op));
rel.schema.putAll(tempInput.first.schema);
return rel;
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/BreakSelectIntoConjunctsRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/BreakSelectIntoConjunctsRule.java
index 0d17dbf..53ca903 100644
--- a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/BreakSelectIntoConjunctsRule.java
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/BreakSelectIntoConjunctsRule.java
@@ -63,7 +63,8 @@
fst = false;
firstExpr = e;
} else {
- SelectOperator newSelect = new SelectOperator(new MutableObject<ILogicalExpression>(e));
+ SelectOperator newSelect = new SelectOperator(new MutableObject<ILogicalExpression>(e),
+ select.getRetainNull(), select.getNullPlaceholderVariable());
List<Mutable<ILogicalOperator>> botInpList = botOp.getInputs();
botInpList.clear();
botInpList.add(new MutableObject<ILogicalOperator>(newSelect));
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ExtractCommonExpressionsRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ExtractCommonExpressionsRule.java
index eafb3b0..41cf470 100644
--- a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ExtractCommonExpressionsRule.java
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ExtractCommonExpressionsRule.java
@@ -301,7 +301,8 @@
return false;
}
// Place a Select operator beneath op that contains the enclosing expression.
- SelectOperator selectOp = new SelectOperator(new MutableObject<ILogicalExpression>(enclosingExpr));
+ SelectOperator selectOp = new SelectOperator(new MutableObject<ILogicalExpression>(enclosingExpr),
+ false, null);
selectOp.getInputs().add(new MutableObject<ILogicalOperator>(op.getInputs().get(0).getValue()));
op.getInputs().get(0).setValue(selectOp);
// Set firstOp to be the select below op, since we want to assign the common subexpr there.
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/InlineVariablesRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/InlineVariablesRule.java
index aa6916d..6390efc 100644
--- a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/InlineVariablesRule.java
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/InlineVariablesRule.java
@@ -36,7 +36,9 @@
import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.algebra.plan.ALogicalPlanImpl;
import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
@@ -144,6 +146,15 @@
}
}
+ // Descend into subplan
+ if (op.getOperatorTag() == LogicalOperatorTag.SUBPLAN) {
+ ALogicalPlanImpl subPlan = (ALogicalPlanImpl) ((SubplanOperator) op).getNestedPlans().get(0);
+ Mutable<ILogicalOperator> subPlanRootOpRef = subPlan.getRoots().get(0);
+ if (inlineVariables(subPlanRootOpRef, context)) {
+ modified = true;
+ }
+ }
+
if (performBottomUpAction(op)) {
modified = true;
}
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IntroduceGroupByForSubplanRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IntroduceGroupByForSubplanRule.java
index b918fc5..1c66317 100644
--- a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IntroduceGroupByForSubplanRule.java
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IntroduceGroupByForSubplanRule.java
@@ -184,7 +184,7 @@
IFunctionInfo finfoNot = context.getMetadataProvider().lookupFunction(AlgebricksBuiltinFunctions.NOT);
ScalarFunctionCallExpression nonNullTest = new ScalarFunctionCallExpression(finfoNot,
new MutableObject<ILogicalExpression>(isNullTest));
- SelectOperator selectNonNull = new SelectOperator(new MutableObject<ILogicalExpression>(nonNullTest));
+ SelectOperator selectNonNull = new SelectOperator(new MutableObject<ILogicalExpression>(nonNullTest), false, null);
GroupByOperator g = new GroupByOperator();
Mutable<ILogicalOperator> newSubplanRef = new MutableObject<ILogicalOperator>(subplan);
NestedTupleSourceOperator nts = new NestedTupleSourceOperator(new MutableObject<ILogicalOperator>(g));
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PullSelectOutOfEqJoin.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PullSelectOutOfEqJoin.java
index 756e0f3..b268e77 100644
--- a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PullSelectOutOfEqJoin.java
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PullSelectOutOfEqJoin.java
@@ -76,7 +76,7 @@
}
// pull up
ILogicalExpression pulledCond = makeCondition(otherPredicates, context);
- SelectOperator select = new SelectOperator(new MutableObject<ILogicalExpression>(pulledCond));
+ SelectOperator select = new SelectOperator(new MutableObject<ILogicalExpression>(pulledCond), false, null);
ILogicalExpression newJoinCond = makeCondition(eqVarVarComps, context);
join.getCondition().setValue(newJoinCond);
select.getInputs().add(new MutableObject<ILogicalOperator>(join));
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushSelectIntoJoinRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushSelectIntoJoinRule.java
index bf053f3..76706a9 100644
--- a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushSelectIntoJoinRule.java
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushSelectIntoJoinRule.java
@@ -224,7 +224,8 @@
private static void copySelectToBranch(SelectOperator select, Mutable<ILogicalOperator> branch,
IOptimizationContext context) throws AlgebricksException {
- ILogicalOperator newSelect = new SelectOperator(select.getCondition());
+ ILogicalOperator newSelect = new SelectOperator(select.getCondition(), select.getRetainNull(),
+ select.getNullPlaceholderVariable());
Mutable<ILogicalOperator> newRef = new MutableObject<ILogicalOperator>(branch.getValue());
newSelect.getInputs().add(newRef);
branch.setValue(newSelect);
diff --git a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputOneFramePushRuntime.java b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputOneFramePushRuntime.java
index b079c3e..aaa6862 100644
--- a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputOneFramePushRuntime.java
+++ b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputOneFramePushRuntime.java
@@ -107,4 +107,12 @@
tRef = new FrameTupleReference();
}
+ protected final void initAccessAppendFieldRef(IHyracksTaskContext ctx) throws HyracksDataException {
+ frame = ctx.allocateFrame();
+ appender = new FrameTupleAppender(ctx.getFrameSize(), inputRecordDesc.getFieldCount());
+ appender.reset(frame, true);
+ tAccess = new FrameTupleAccessor(ctx.getFrameSize(), inputRecordDesc);
+ tRef = new FrameTupleReference();
+ }
+
}
diff --git a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/StreamSelectRuntimeFactory.java b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/StreamSelectRuntimeFactory.java
index 8632d62..17603e7 100644
--- a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/StreamSelectRuntimeFactory.java
+++ b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/StreamSelectRuntimeFactory.java
@@ -14,6 +14,7 @@
*/
package edu.uci.ics.hyracks.algebricks.runtime.operators.std;
+import java.io.DataOutput;
import java.nio.ByteBuffer;
import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -24,9 +25,13 @@
import edu.uci.ics.hyracks.algebricks.runtime.operators.base.AbstractOneInputOneOutputOneFramePushRuntime;
import edu.uci.ics.hyracks.algebricks.runtime.operators.base.AbstractOneInputOneOutputRuntimeFactory;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriter;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.data.std.api.IPointable;
import edu.uci.ics.hyracks.data.std.primitive.VoidPointable;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
public class StreamSelectRuntimeFactory extends AbstractOneInputOneOutputRuntimeFactory {
@@ -36,16 +41,30 @@
private IBinaryBooleanInspectorFactory binaryBooleanInspectorFactory;
+ private boolean retainNull;
+
+ private int nullPlaceholderVariableIndex;
+
+ private INullWriterFactory nullWriterFactory;
+
/**
* @param cond
* @param projectionList
* if projectionList is null, then no projection is performed
+ * @param retainNull
+ * @param nullPlaceholderVariableIndex
+ * @param nullWriterFactory
+ * @throws HyracksDataException
*/
public StreamSelectRuntimeFactory(IScalarEvaluatorFactory cond, int[] projectionList,
- IBinaryBooleanInspectorFactory binaryBooleanInspectorFactory) {
+ IBinaryBooleanInspectorFactory binaryBooleanInspectorFactory, boolean retainNull,
+ int nullPlaceholderVariableIndex, INullWriterFactory nullWriterFactory) {
super(projectionList);
this.cond = cond;
this.binaryBooleanInspectorFactory = binaryBooleanInspectorFactory;
+ this.retainNull = retainNull;
+ this.nullPlaceholderVariableIndex = nullPlaceholderVariableIndex;
+ this.nullWriterFactory = nullWriterFactory;
}
@Override
@@ -59,11 +78,13 @@
return new AbstractOneInputOneOutputOneFramePushRuntime() {
private IPointable p = VoidPointable.FACTORY.createPointable();
private IScalarEvaluator eval;
+ private INullWriter nullWriter = null;
+ private ArrayTupleBuilder nullTupleBuilder = null;
@Override
public void open() throws HyracksDataException {
if (eval == null) {
- initAccessAppendRef(ctx);
+ initAccessAppendFieldRef(ctx);
try {
eval = cond.createScalarEvaluator(ctx);
} catch (AlgebricksException ae) {
@@ -71,6 +92,15 @@
}
}
writer.open();
+
+ //prepare nullTupleBuilder
+ if (retainNull && nullWriter == null) {
+ nullWriter = nullWriterFactory.createNullWriter();
+ nullTupleBuilder = new ArrayTupleBuilder(1);
+ DataOutput out = nullTupleBuilder.getDataOutput();
+ nullWriter.writeNull(out);
+ nullTupleBuilder.addFieldEndOffset();
+ }
}
@Override
@@ -90,10 +120,43 @@
} else {
appendTupleToFrame(t);
}
+ } else {
+ if (retainNull) {
+ //keep all field values as is except setting nullPlaceholderVariable field to null
+ int i = 0;
+ int tryCount = 0;
+ while (true) {
+ for (i = 0; i < tRef.getFieldCount(); i++) {
+ if (i == nullPlaceholderVariableIndex) {
+ if (!appender.appendField(nullTupleBuilder.getByteArray(), 0,
+ nullTupleBuilder.getSize())) {
+ FrameUtils.flushFrame(frame, writer);
+ appender.reset(frame, true);
+ break;
+ }
+ } else {
+ if (!appender.appendField(tAccess, t, i)) {
+ FrameUtils.flushFrame(frame, writer);
+ appender.reset(frame, true);
+ break;
+ }
+ }
+ }
+
+ if (i == tRef.getFieldCount()) {
+ break;
+ } else {
+ tryCount++;
+ if (tryCount == 2) {
+ throw new IllegalStateException(
+ "Could not write frame (AbstractOneInputOneOutputOneFramePushRuntime).");
+ }
+ }
+ }
+ }
}
}
}
-
};
}
diff --git a/algebricks/algebricks-tests/src/test/java/edu/uci/ics/hyracks/algebricks/tests/pushruntime/PushRuntimeTest.java b/algebricks/algebricks-tests/src/test/java/edu/uci/ics/hyracks/algebricks/tests/pushruntime/PushRuntimeTest.java
index 2b29448..0876194 100644
--- a/algebricks/algebricks-tests/src/test/java/edu/uci/ics/hyracks/algebricks/tests/pushruntime/PushRuntimeTest.java
+++ b/algebricks/algebricks-tests/src/test/java/edu/uci/ics/hyracks/algebricks/tests/pushruntime/PushRuntimeTest.java
@@ -197,7 +197,7 @@
IScalarEvaluatorFactory cond = new IntegerGreaterThanEvalFactory(new IntegerConstantEvalFactory(2),
new TupleFieldEvaluatorFactory(0));
StreamSelectRuntimeFactory select = new StreamSelectRuntimeFactory(cond, new int[] { 0 },
- BinaryBooleanInspectorImpl.FACTORY);
+ BinaryBooleanInspectorImpl.FACTORY, false, -1, null);
RecordDescriptor selectDesc = intScannerDesc;
String filePath = PATH_ACTUAL + SEPARATOR + "scanSelectWrite.out";
@@ -450,7 +450,7 @@
IScalarEvaluatorFactory cond = new IntegerEqualsEvalFactory(new IntegerConstantEvalFactory(3),
new TupleFieldEvaluatorFactory(0)); // Canadian customers
StreamSelectRuntimeFactory select = new StreamSelectRuntimeFactory(cond, new int[] { 1 },
- BinaryBooleanInspectorImpl.FACTORY);
+ BinaryBooleanInspectorImpl.FACTORY, false, -1, null);
RecordDescriptor selectDesc = new RecordDescriptor(
new ISerializerDeserializer[] { IntegerSerializerDeserializer.INSTANCE });
@@ -520,7 +520,7 @@
IScalarEvaluatorFactory cond = new IntegerEqualsEvalFactory(new IntegerConstantEvalFactory(3),
new TupleFieldEvaluatorFactory(0)); // Canadian customers
StreamSelectRuntimeFactory select = new StreamSelectRuntimeFactory(cond, new int[] { 1 },
- BinaryBooleanInspectorImpl.FACTORY);
+ BinaryBooleanInspectorImpl.FACTORY, false, -1, null);
RecordDescriptor selectDesc = new RecordDescriptor(
new ISerializerDeserializer[] { IntegerSerializerDeserializer.INSTANCE });
@@ -853,7 +853,7 @@
IScalarEvaluatorFactory cond = new IntegerEqualsEvalFactory(new IntegerConstantEvalFactory(3),
new TupleFieldEvaluatorFactory(0)); // Canadian customers
StreamSelectRuntimeFactory select = new StreamSelectRuntimeFactory(cond, new int[] { 1 },
- BinaryBooleanInspectorImpl.FACTORY);
+ BinaryBooleanInspectorImpl.FACTORY, false, -1, null);
RecordDescriptor selectDesc = new RecordDescriptor(
new ISerializerDeserializer[] { IntegerSerializerDeserializer.INSTANCE });
diff --git a/hivesterix/hivesterix-translator/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/FilterVisitor.java b/hivesterix/hivesterix-translator/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/FilterVisitor.java
index 03c8173..642285d 100644
--- a/hivesterix/hivesterix-translator/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/FilterVisitor.java
+++ b/hivesterix/hivesterix-translator/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/FilterVisitor.java
@@ -42,7 +42,7 @@
t.rewriteExpression(predicate);
Mutable<ILogicalExpression> exprs = t.translateScalarFucntion(desc.getPredicate());
- ILogicalOperator currentOperator = new SelectOperator(exprs);
+ ILogicalOperator currentOperator = new SelectOperator(exprs, false, null);
currentOperator.getInputs().add(AlgebricksParentOperatorRef);
// populate the schema from upstream operator
diff --git a/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexSearchExample.java b/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexSearchExample.java
index 01004e6f..a3d0772 100644
--- a/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexSearchExample.java
+++ b/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexSearchExample.java
@@ -142,7 +142,7 @@
IIndexDataflowHelperFactory dataflowHelperFactory = new BTreeDataflowHelperFactory();
BTreeSearchOperatorDescriptor btreeSearchOp = new BTreeSearchOperatorDescriptor(spec, recDesc, storageManager,
lcManagerProvider, btreeSplitProvider, typeTraits, comparatorFactories, null, lowKeyFields,
- highKeyFields, true, true, dataflowHelperFactory, false, NoOpOperationCallbackFactory.INSTANCE);
+ highKeyFields, true, true, dataflowHelperFactory, false, false, null, NoOpOperationCallbackFactory.INSTANCE);
JobHelper.createPartitionConstraint(spec, btreeSearchOp, splitNCs);
// have each node print the results of its respective B-Tree
diff --git a/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexSearchExample.java b/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexSearchExample.java
index 5ff5003..3af2e18 100644
--- a/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexSearchExample.java
+++ b/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexSearchExample.java
@@ -169,7 +169,7 @@
BTreeSearchOperatorDescriptor secondarySearchOp = new BTreeSearchOperatorDescriptor(spec, secondaryRecDesc,
storageManager, lcManagerProvider, secondarySplitProvider, secondaryTypeTraits,
searchComparatorFactories, null, secondaryLowKeyFields, secondaryHighKeyFields, true, true,
- dataflowHelperFactory, false, NoOpOperationCallbackFactory.INSTANCE);
+ dataflowHelperFactory, false, false, null, NoOpOperationCallbackFactory.INSTANCE);
JobHelper.createPartitionConstraint(spec, secondarySearchOp, splitNCs);
// secondary index will output tuples with [UTF8String, Integer]
@@ -185,7 +185,7 @@
BTreeSearchOperatorDescriptor primarySearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
storageManager, lcManagerProvider, primarySplitProvider, primaryTypeTraits, primaryComparatorFactories,
null, primaryLowKeyFields, primaryHighKeyFields, true, true, dataflowHelperFactory, false,
- NoOpOperationCallbackFactory.INSTANCE);
+ false, null, NoOpOperationCallbackFactory.INSTANCE);
JobHelper.createPartitionConstraint(spec, primarySearchOp, splitNCs);
// have each node print the results of its respective B-Tree
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/AbstractBTreeOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/AbstractBTreeOperatorTest.java
index eec7225..428d4fd 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/AbstractBTreeOperatorTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/AbstractBTreeOperatorTest.java
@@ -228,7 +228,7 @@
BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
storageManager, lcManagerProvider, primarySplitProvider, primaryTypeTraits, primaryComparatorFactories,
primaryBloomFilterKeyFields, lowKeyFields, highKeyFields, true, true, dataflowHelperFactory, false,
- NoOpOperationCallbackFactory.INSTANCE);
+ false, null, NoOpOperationCallbackFactory.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
// sort based on secondary keys
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreePrimaryIndexScanOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreePrimaryIndexScanOperatorTest.java
index 9f164a5..76dd6f9 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreePrimaryIndexScanOperatorTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreePrimaryIndexScanOperatorTest.java
@@ -72,7 +72,7 @@
BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
storageManager, lcManagerProvider, primarySplitProvider, primaryTypeTraits, primaryComparatorFactories,
primaryBloomFilterKeyFields, lowKeyFields, highKeyFields, true, true, dataflowHelperFactory, false,
- NoOpOperationCallbackFactory.INSTANCE);
+ false, null, NoOpOperationCallbackFactory.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreePrimaryIndexSearchOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreePrimaryIndexSearchOperatorTest.java
index f0d8df8..2c41b39 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreePrimaryIndexSearchOperatorTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreePrimaryIndexSearchOperatorTest.java
@@ -77,7 +77,7 @@
BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
storageManager, lcManagerProvider, primarySplitProvider, primaryTypeTraits, primaryComparatorFactories,
primaryBloomFilterKeyFields, lowKeyFields, highKeyFields, true, true, dataflowHelperFactory, false,
- NoOpOperationCallbackFactory.INSTANCE);
+ false, null, NoOpOperationCallbackFactory.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreeSecondaryIndexInsertOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreeSecondaryIndexInsertOperatorTest.java
index e4c1f97..6962a82 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreeSecondaryIndexInsertOperatorTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreeSecondaryIndexInsertOperatorTest.java
@@ -81,7 +81,7 @@
BTreeSearchOperatorDescriptor secondaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
secondaryRecDesc, storageManager, lcManagerProvider, secondarySplitProvider, secondaryTypeTraits,
secondaryComparatorFactories, secondaryBloomFilterKeyFields, secondaryLowKeyFields,
- secondaryHighKeyFields, true, true, dataflowHelperFactory, false, NoOpOperationCallbackFactory.INSTANCE);
+ secondaryHighKeyFields, true, true, dataflowHelperFactory, false, false, null, NoOpOperationCallbackFactory.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryBtreeSearchOp, NC1_ID);
// second field from the tuples coming from secondary index
@@ -93,7 +93,7 @@
BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
storageManager, lcManagerProvider, primarySplitProvider, primaryTypeTraits, primaryComparatorFactories,
primaryBloomFilterKeyFields, primaryLowKeyFields, primaryHighKeyFields, true, true,
- dataflowHelperFactory, false, NoOpOperationCallbackFactory.INSTANCE);
+ dataflowHelperFactory, false, false, null, NoOpOperationCallbackFactory.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreeSecondaryIndexSearchOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreeSecondaryIndexSearchOperatorTest.java
index c912e7d..0341e74 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreeSecondaryIndexSearchOperatorTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreeSecondaryIndexSearchOperatorTest.java
@@ -80,7 +80,7 @@
BTreeSearchOperatorDescriptor secondaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
secondaryRecDesc, storageManager, lcManagerProvider, secondarySplitProvider, secondaryTypeTraits,
secondaryComparatorFactories, secondaryBloomFilterKeyFields, secondaryLowKeyFields,
- secondaryHighKeyFields, true, true, dataflowHelperFactory, false, NoOpOperationCallbackFactory.INSTANCE);
+ secondaryHighKeyFields, true, true, dataflowHelperFactory, false, false, null, NoOpOperationCallbackFactory.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryBtreeSearchOp, NC1_ID);
int[] primaryLowKeyFields = { 1 }; // second field from the tuples
@@ -92,7 +92,7 @@
BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
storageManager, lcManagerProvider, primarySplitProvider, primaryTypeTraits, primaryComparatorFactories,
primaryBloomFilterKeyFields, primaryLowKeyFields, primaryHighKeyFields, true, true,
- dataflowHelperFactory, false, NoOpOperationCallbackFactory.INSTANCE);
+ dataflowHelperFactory, false, false, null, NoOpOperationCallbackFactory.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreeSecondaryIndexUpsertOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreeSecondaryIndexUpsertOperatorTest.java
index 7a6fc73..6b503c1 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreeSecondaryIndexUpsertOperatorTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreeSecondaryIndexUpsertOperatorTest.java
@@ -80,7 +80,7 @@
BTreeSearchOperatorDescriptor secondaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
secondaryRecDesc, storageManager, lcManagerProvider, secondarySplitProvider, secondaryTypeTraits,
secondaryComparatorFactories, secondaryBloomFilterKeyFields, secondaryLowKeyFields,
- secondaryHighKeyFields, true, true, dataflowHelperFactory, false, NoOpOperationCallbackFactory.INSTANCE);
+ secondaryHighKeyFields, true, true, dataflowHelperFactory, false, false, null, NoOpOperationCallbackFactory.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryBtreeSearchOp, NC1_ID);
// second field from the tuples coming from secondary index
@@ -92,7 +92,7 @@
BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
storageManager, lcManagerProvider, primarySplitProvider, primaryTypeTraits, primaryComparatorFactories,
primaryBloomFilterKeyFields, primaryLowKeyFields, primaryHighKeyFields, true, true,
- dataflowHelperFactory, false, NoOpOperationCallbackFactory.INSTANCE);
+ dataflowHelperFactory, false, false, null, NoOpOperationCallbackFactory.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/invertedindex/AbstractfWordInvertedIndexTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/invertedindex/AbstractfWordInvertedIndexTest.java
index d9b4984..30451e1 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/invertedindex/AbstractfWordInvertedIndexTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/invertedindex/AbstractfWordInvertedIndexTest.java
@@ -237,7 +237,7 @@
BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
storageManager, lcManagerProvider, primaryFileSplitProvider, primaryTypeTraits,
primaryComparatorFactories, null, lowKeyFields, highKeyFields, true, true, btreeDataflowHelperFactory,
- false, NoOpOperationCallbackFactory.INSTANCE);
+ false, false, null, NoOpOperationCallbackFactory.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
return primaryBtreeSearchOp;
}
@@ -335,7 +335,7 @@
0, storageManager, btreeFileSplitProvider, lcManagerProvider, tokenTypeTraits,
tokenComparatorFactories, invListsTypeTraits, invListsComparatorFactories,
invertedIndexDataflowHelperFactory, tokenizerFactory, searchModifierFactory, invListsRecDesc, false,
- NoOpOperationCallbackFactory.INSTANCE);
+ false, null, NoOpOperationCallbackFactory.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, invIndexSearchOp, NC1_ID);
return invIndexSearchOp;
}
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/AbstractRTreeOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/AbstractRTreeOperatorTest.java
index ee08868..6fc1d4b 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/AbstractRTreeOperatorTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/AbstractRTreeOperatorTest.java
@@ -292,7 +292,7 @@
BTreeSearchOperatorDescriptor primarySearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
storageManager, lcManagerProvider, primarySplitProvider, primaryTypeTraits, primaryComparatorFactories,
null, lowKeyFields, highKeyFields, true, true, btreeDataflowHelperFactory, false,
- NoOpOperationCallbackFactory.INSTANCE);
+ false, null, NoOpOperationCallbackFactory.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primarySearchOp, NC1_ID);
// load secondary index
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/RTreeSecondaryIndexInsertOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/RTreeSecondaryIndexInsertOperatorTest.java
index 32380de..42cf8f0 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/RTreeSecondaryIndexInsertOperatorTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/RTreeSecondaryIndexInsertOperatorTest.java
@@ -91,7 +91,7 @@
RTreeSearchOperatorDescriptor secondarySearchOp = new RTreeSearchOperatorDescriptor(spec, secondaryRecDesc,
storageManager, lcManagerProvider, secondarySplitProvider, secondaryTypeTraits,
secondaryComparatorFactories, keyFields, rtreeDataflowHelperFactory, false,
- NoOpOperationCallbackFactory.INSTANCE);
+ false, null, NoOpOperationCallbackFactory.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondarySearchOp, NC1_ID);
// fifth field from the tuples coming from secondary index
@@ -103,7 +103,7 @@
BTreeSearchOperatorDescriptor primarySearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
storageManager, lcManagerProvider, primarySplitProvider, primaryTypeTraits, primaryComparatorFactories,
null, primaryLowKeyFields, primaryHighKeyFields, true, true, btreeDataflowHelperFactory, false,
- NoOpOperationCallbackFactory.INSTANCE);
+ false, null, NoOpOperationCallbackFactory.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primarySearchOp, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/RTreeSecondaryIndexScanOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/RTreeSecondaryIndexScanOperatorTest.java
index 40d9e5b..cbd8b4c 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/RTreeSecondaryIndexScanOperatorTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/RTreeSecondaryIndexScanOperatorTest.java
@@ -88,7 +88,7 @@
RTreeSearchOperatorDescriptor secondarySearchOp = new RTreeSearchOperatorDescriptor(spec, secondaryRecDesc,
storageManager, lcManagerProvider, secondarySplitProvider, secondaryTypeTraits,
secondaryComparatorFactories, keyFields, rtreeDataflowHelperFactory, false,
- NoOpOperationCallbackFactory.INSTANCE);
+ false, null, NoOpOperationCallbackFactory.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondarySearchOp, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/RTreeSecondaryIndexSearchOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/RTreeSecondaryIndexSearchOperatorTest.java
index b6f1000..2cabe52 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/RTreeSecondaryIndexSearchOperatorTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/RTreeSecondaryIndexSearchOperatorTest.java
@@ -89,7 +89,7 @@
RTreeSearchOperatorDescriptor secondarySearchOp = new RTreeSearchOperatorDescriptor(spec, secondaryRecDesc,
storageManager, lcManagerProvider, secondarySplitProvider, secondaryTypeTraits,
secondaryComparatorFactories, keyFields, rtreeDataflowHelperFactory, false,
- NoOpOperationCallbackFactory.INSTANCE);
+ false, null, NoOpOperationCallbackFactory.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondarySearchOp, NC1_ID);
// fifth field from the tuples coming from secondary index
@@ -101,7 +101,7 @@
BTreeSearchOperatorDescriptor primarySearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
storageManager, lcManagerProvider, primarySplitProvider, primaryTypeTraits, primaryComparatorFactories,
null, primaryLowKeyFields, primaryHighKeyFields, true, true, btreeDataflowHelperFactory, false,
- NoOpOperationCallbackFactory.INSTANCE);
+ false, null, NoOpOperationCallbackFactory.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primarySearchOp, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
index bd50981..2f3f075 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
@@ -18,6 +18,7 @@
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
@@ -45,11 +46,11 @@
IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
IBinaryComparatorFactory[] comparatorFactories, int[] bloomFilterKeyFields, int[] lowKeyFields,
int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
- IIndexDataflowHelperFactory dataflowHelperFactory, boolean retainInput,
- ISearchOperationCallbackFactory searchOpCallbackProvider) {
+ IIndexDataflowHelperFactory dataflowHelperFactory, boolean retainInput, boolean retainNull,
+ INullWriterFactory nullWriterFactory, ISearchOperationCallbackFactory searchOpCallbackProvider) {
super(spec, 1, 1, recDesc, storageManager, lifecycleManagerProvider, fileSplitProvider, typeTraits,
- comparatorFactories, bloomFilterKeyFields, dataflowHelperFactory, null, retainInput,
- NoOpLocalResourceFactoryProvider.INSTANCE, searchOpCallbackProvider,
+ comparatorFactories, bloomFilterKeyFields, dataflowHelperFactory, null, retainInput, retainNull,
+ nullWriterFactory, NoOpLocalResourceFactoryProvider.INSTANCE, searchOpCallbackProvider,
NoOpOperationCallbackFactory.INSTANCE);
this.lowKeyFields = lowKeyFields;
this.highKeyFields = highKeyFields;
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
index d68901f..82e52c0 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
@@ -67,4 +67,9 @@
highKeySearchCmp = BTreeUtils.getSearchMultiComparator(treeIndex.getComparatorFactories(), highKey);
return new RangePredicate(lowKey, highKey, lowKeyInclusive, highKeyInclusive, lowKeySearchCmp, highKeySearchCmp);
}
+
+ @Override
+ protected int getFieldCount() {
+ return ((ITreeIndex) index).getFieldCount();
+ }
}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorDescriptor.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorDescriptor.java
index cd64849..195e177 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorDescriptor.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorDescriptor.java
@@ -44,7 +44,7 @@
ISearchOperationCallbackFactory searchOpCallbackProvider, ITupleUpdaterFactory tupleUpdaterFactory) {
super(spec, recDesc, storageManager, lifecycleManagerProvider, fileSplitProvider, typeTraits,
comparatorFactories, bloomFilterKeyFields, lowKeyFields, highKeyFields, lowKeyInclusive,
- highKeyInclusive, dataflowHelperFactory, retainInput, searchOpCallbackProvider);
+ highKeyInclusive, dataflowHelperFactory, retainInput, false, null, searchOpCallbackProvider);
this.tupleUpdaterFactory = tupleUpdaterFactory;
}
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/AbstractIndexOperatorDescriptor.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/AbstractIndexOperatorDescriptor.java
index 1e0bb25..ee35393 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/AbstractIndexOperatorDescriptor.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/AbstractIndexOperatorDescriptor.java
@@ -15,6 +15,7 @@
package edu.uci.ics.hyracks.storage.am.common.dataflow;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
@@ -37,6 +38,8 @@
protected final IIndexDataflowHelperFactory dataflowHelperFactory;
protected final ITupleFilterFactory tupleFilterFactory;
protected final boolean retainInput;
+ protected final boolean retainNull;
+ protected final INullWriterFactory nullWriterFactory;
protected final ISearchOperationCallbackFactory searchOpCallbackFactory;
protected final IModificationOperationCallbackFactory modificationOpCallbackFactory;
protected final ILocalResourceFactoryProvider localResourceFactoryProvider;
@@ -45,7 +48,8 @@
RecordDescriptor recDesc, IStorageManagerInterface storageManager,
IIndexLifecycleManagerProvider lifecycleManagerProvider, IFileSplitProvider fileSplitProvider,
IIndexDataflowHelperFactory dataflowHelperFactory, ITupleFilterFactory tupleFilterFactory,
- boolean retainInput, ILocalResourceFactoryProvider localResourceFactoryProvider,
+ boolean retainInput, boolean retainNull, INullWriterFactory nullWriterFactory,
+ ILocalResourceFactoryProvider localResourceFactoryProvider,
ISearchOperationCallbackFactory searchOpCallbackFactory,
IModificationOperationCallbackFactory modificationOpCallbackFactory) {
super(spec, inputArity, outputArity);
@@ -54,6 +58,8 @@
this.lifecycleManagerProvider = lifecycleManagerProvider;
this.dataflowHelperFactory = dataflowHelperFactory;
this.retainInput = retainInput;
+ this.retainNull = retainNull;
+ this.nullWriterFactory = nullWriterFactory;
this.tupleFilterFactory = tupleFilterFactory;
this.localResourceFactoryProvider = localResourceFactoryProvider;
this.searchOpCallbackFactory = searchOpCallbackFactory;
@@ -94,10 +100,20 @@
}
@Override
+ public boolean getRetainNull() {
+ return retainNull;
+ }
+
+ @Override
+ public INullWriterFactory getNullWriterFactory() {
+ return nullWriterFactory;
+ }
+
+ @Override
public ISearchOperationCallbackFactory getSearchOpCallbackFactory() {
return searchOpCallbackFactory;
}
-
+
@Override
public IModificationOperationCallbackFactory getModificationOpCallbackFactory() {
return modificationOpCallbackFactory;
@@ -107,7 +123,7 @@
public ITupleFilterFactory getTupleFilterFactory() {
return tupleFilterFactory;
}
-
+
@Override
public ILocalResourceFactoryProvider getLocalResourceFactoryProvider() {
return localResourceFactoryProvider;
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/AbstractTreeIndexOperatorDescriptor.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/AbstractTreeIndexOperatorDescriptor.java
index 76fea07..d907316 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/AbstractTreeIndexOperatorDescriptor.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/AbstractTreeIndexOperatorDescriptor.java
@@ -16,6 +16,7 @@
package edu.uci.ics.hyracks.storage.am.common.dataflow;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
@@ -40,12 +41,13 @@
IIndexLifecycleManagerProvider lifecycleManagerProvider, IFileSplitProvider fileSplitProvider,
ITypeTraits[] typeTraits, IBinaryComparatorFactory[] comparatorFactories, int[] bloomFilterKeyFields,
IIndexDataflowHelperFactory dataflowHelperFactory, ITupleFilterFactory tupleFilterFactory,
- boolean retainInput, ILocalResourceFactoryProvider localResourceFactoryProvider,
+ boolean retainInput, boolean retainNull, INullWriterFactory nullWriterFactory,
+ ILocalResourceFactoryProvider localResourceFactoryProvider,
ISearchOperationCallbackFactory searchOpCallbackFactory,
IModificationOperationCallbackFactory modificationOpCallbackFactory) {
super(spec, inputArity, outputArity, recDesc, storageManager, lifecycleManagerProvider, fileSplitProvider,
- dataflowHelperFactory, tupleFilterFactory, retainInput, localResourceFactoryProvider,
- searchOpCallbackFactory, modificationOpCallbackFactory);
+ dataflowHelperFactory, tupleFilterFactory, retainInput, retainNull, nullWriterFactory,
+ localResourceFactoryProvider, searchOpCallbackFactory, modificationOpCallbackFactory);
this.typeTraits = typeTraits;
this.comparatorFactories = comparatorFactories;
this.bloomFilterKeyFields = bloomFilterKeyFields;
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IIndexOperatorDescriptor.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IIndexOperatorDescriptor.java
index 922dbe4..0a30ae3 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IIndexOperatorDescriptor.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IIndexOperatorDescriptor.java
@@ -16,6 +16,7 @@
package edu.uci.ics.hyracks.storage.am.common.dataflow;
import edu.uci.ics.hyracks.api.dataflow.IActivity;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
@@ -38,11 +39,15 @@
public boolean getRetainInput();
+ public boolean getRetainNull();
+
+ public INullWriterFactory getNullWriterFactory();
+
public ISearchOperationCallbackFactory getSearchOpCallbackFactory();
-
+
public IModificationOperationCallbackFactory getModificationOpCallbackFactory();
-
+
public ITupleFilterFactory getTupleFilterFactory();
-
+
public ILocalResourceFactoryProvider getLocalResourceFactoryProvider();
}
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexDropOperatorDescriptor.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexDropOperatorDescriptor.java
index 37edbd9..4983be3 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexDropOperatorDescriptor.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexDropOperatorDescriptor.java
@@ -41,8 +41,8 @@
// This should eventually not be *hacked*, but I don't know the proper fix yet. -zheilbron
super(spec, 0, 0, null, storageManager, lifecycleManagerProvider, fileSplitProvider, new ITypeTraits[] {
IntegerPointable.TYPE_TRAITS, IntegerPointable.TYPE_TRAITS }, new IBinaryComparatorFactory[] { null }, null,
- dataflowHelperFactory, null, false, NoOpLocalResourceFactoryProvider.INSTANCE,
- NoOpOperationCallbackFactory.INSTANCE, NoOpOperationCallbackFactory.INSTANCE);
+ dataflowHelperFactory, null, false, false,
+ null, NoOpLocalResourceFactoryProvider.INSTANCE, NoOpOperationCallbackFactory.INSTANCE, NoOpOperationCallbackFactory.INSTANCE);
}
@Override
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
index 2696af9..acc5bc2 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
@@ -15,9 +15,11 @@
package edu.uci.ics.hyracks.storage.am.common.dataflow;
import java.io.DataOutput;
+import java.io.IOException;
import java.nio.ByteBuffer;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriter;
import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -55,6 +57,9 @@
protected final RecordDescriptor inputRecDesc;
protected final boolean retainInput;
protected FrameTupleReference frameTuple;
+ protected final boolean retainNull;
+ protected ArrayTupleBuilder nullTupleBuild;
+ protected INullWriter nullWriter;
public IndexSearchOperatorNodePushable(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
IRecordDescriptorProvider recordDescProvider) {
@@ -62,6 +67,10 @@
this.ctx = ctx;
this.indexHelper = opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(opDesc, ctx, partition);
this.retainInput = opDesc.getRetainInput();
+ this.retainNull = opDesc.getRetainNull();
+ if (this.retainNull) {
+ this.nullWriter = opDesc.getNullWriterFactory().createNullWriter();
+ }
this.inputRecDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
}
@@ -73,12 +82,31 @@
return indexAccessor.createSearchCursor(false);
}
+ protected abstract int getFieldCount();
+
@Override
public void open() throws HyracksDataException {
accessor = new FrameTupleAccessor(ctx.getFrameSize(), inputRecDesc);
writer.open();
indexHelper.open();
index = indexHelper.getIndexInstance();
+
+ if (retainNull) {
+ int fieldCount = getFieldCount();
+ nullTupleBuild = new ArrayTupleBuilder(fieldCount);
+ DataOutput out = nullTupleBuild.getDataOutput();
+ for (int i = 0; i < fieldCount; i++) {
+ try {
+ nullWriter.writeNull(out);
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ nullTupleBuild.addFieldEndOffset();
+ }
+ } else {
+ nullTupleBuild = null;
+ }
+
try {
searchPred = createSearchPredicate();
writeBuffer = ctx.allocateFrame();
@@ -100,7 +128,9 @@
}
protected void writeSearchResults(int tupleIndex) throws Exception {
+ boolean matched = false;
while (cursor.hasNext()) {
+ matched = true;
tb.reset();
cursor.next();
if (retainInput) {
@@ -124,6 +154,20 @@
}
}
}
+
+ if (!matched && retainInput && retainNull) {
+ if (!appender.appendConcat(accessor, tupleIndex, nullTupleBuild.getFieldEndOffsets(),
+ nullTupleBuild.getByteArray(), 0, nullTupleBuild.getSize())) {
+ FrameUtils.flushFrame(writeBuffer, writer);
+ appender.reset(writeBuffer, true);
+ if (!appender.appendConcat(accessor, tupleIndex, nullTupleBuild.getFieldEndOffsets(),
+ nullTupleBuild.getByteArray(), 0, nullTupleBuild.getSize())) {
+ throw new HyracksDataException("Record size larger than frame size ("
+ + appender.getBuffer().capacity() + ")");
+ }
+ }
+ }
+
}
@Override
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorDescriptor.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorDescriptor.java
index a58e9e8..e8db013 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorDescriptor.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorDescriptor.java
@@ -47,8 +47,8 @@
IModificationOperationCallbackFactory modificationOpCallbackFactory) {
super(spec, 1, 0, null, storageManager, lifecycleManagerProvider, fileSplitProvider, typeTraits,
comparatorFactories, bloomFilterKeyFields, dataflowHelperFactory, null, false,
- NoOpLocalResourceFactoryProvider.INSTANCE, NoOpOperationCallbackFactory.INSTANCE,
- modificationOpCallbackFactory);
+ false, null,
+ NoOpLocalResourceFactoryProvider.INSTANCE, NoOpOperationCallbackFactory.INSTANCE, modificationOpCallbackFactory);
this.fieldPermutation = fieldPermutation;
this.fillFactor = fillFactor;
this.verifyInput = verifyInput;
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexCreateOperatorDescriptor.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexCreateOperatorDescriptor.java
index bd7b4fa..a1f19ba 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexCreateOperatorDescriptor.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexCreateOperatorDescriptor.java
@@ -40,7 +40,7 @@
IModificationOperationCallbackFactory modificationOpCallbackFactory) {
super(spec, 0, 0, null, storageManager, lifecycleManagerProvider, fileSplitProvider, typeTraits,
comparatorFactories, bloomFilterKeyFields, dataflowHelperFactory, null, false,
- localResourceFactoryProvider, NoOpOperationCallbackFactory.INSTANCE, modificationOpCallbackFactory);
+ false, null, localResourceFactoryProvider, NoOpOperationCallbackFactory.INSTANCE, modificationOpCallbackFactory);
}
@Override
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorDescriptor.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorDescriptor.java
index 83558b1..6840bd4 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorDescriptor.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorDescriptor.java
@@ -37,8 +37,8 @@
IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
IIndexDataflowHelperFactory dataflowHelperFactory, ISearchOperationCallbackFactory searchOpCallbackProvider) {
super(spec, 0, 1, recDesc, storageManager, lifecycleManagerProvider, fileSplitProvider, typeTraits, null, null,
- dataflowHelperFactory, null, false, NoOpLocalResourceFactoryProvider.INSTANCE,
- searchOpCallbackProvider, NoOpOperationCallbackFactory.INSTANCE);
+ dataflowHelperFactory, null, false, false,
+ null, NoOpLocalResourceFactoryProvider.INSTANCE, searchOpCallbackProvider, NoOpOperationCallbackFactory.INSTANCE);
}
@Override
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorDescriptor.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorDescriptor.java
index 9a97190..1e9903d 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorDescriptor.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorDescriptor.java
@@ -46,8 +46,8 @@
ITupleFilterFactory tupleFilterFactory, IModificationOperationCallbackFactory modificationOpCallbackProvider) {
super(spec, 1, 1, recDesc, storageManager, lifecycleManagerProvider, fileSplitProvider, typeTraits,
comparatorFactories, bloomFilterKeyFields, dataflowHelperFactory, tupleFilterFactory, false,
- NoOpLocalResourceFactoryProvider.INSTANCE, NoOpOperationCallbackFactory.INSTANCE,
- modificationOpCallbackProvider);
+ false, null,
+ NoOpLocalResourceFactoryProvider.INSTANCE, NoOpOperationCallbackFactory.INSTANCE, modificationOpCallbackProvider);
this.fieldPermutation = fieldPermutation;
this.op = op;
}
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorDescriptor.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorDescriptor.java
index 4bbdbfc..4a483dc 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorDescriptor.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorDescriptor.java
@@ -42,8 +42,8 @@
IIndexDataflowHelperFactory dataflowHelperFactory, ISearchOperationCallbackFactory searchOpCallbackProvider) {
super(spec, 0, 1, recDesc, storageManager, lifecycleManagerProvider, fileSplitProvider, typeTraits,
comparatorFactories, bloomFilterKeyFields, dataflowHelperFactory, null, false,
- NoOpLocalResourceFactoryProvider.INSTANCE, searchOpCallbackProvider,
- NoOpOperationCallbackFactory.INSTANCE);
+ false, null,
+ NoOpLocalResourceFactoryProvider.INSTANCE, searchOpCallbackProvider, NoOpOperationCallbackFactory.INSTANCE);
}
@Override
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/LSMTreeIndexCompactOperatorDescriptor.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/LSMTreeIndexCompactOperatorDescriptor.java
index 3c40f94..0931195 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/LSMTreeIndexCompactOperatorDescriptor.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/LSMTreeIndexCompactOperatorDescriptor.java
@@ -42,8 +42,8 @@
IModificationOperationCallbackFactory modificationOpCallbackProvider) {
super(spec, 0, 0, null, storageManager, lifecycleManagerProvider, fileSplitProvider, typeTraits,
comparatorFactories, bloomFilterKeyFields, dataflowHelperFactory, null, false,
- NoOpLocalResourceFactoryProvider.INSTANCE, NoOpOperationCallbackFactory.INSTANCE,
- modificationOpCallbackProvider);
+ false, null,
+ NoOpLocalResourceFactoryProvider.INSTANCE, NoOpOperationCallbackFactory.INSTANCE, modificationOpCallbackProvider);
}
@Override
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/LSMTreeIndexInsertUpdateDeleteOperatorDescriptor.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/LSMTreeIndexInsertUpdateDeleteOperatorDescriptor.java
index 7139583..0363d01 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/LSMTreeIndexInsertUpdateDeleteOperatorDescriptor.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/LSMTreeIndexInsertUpdateDeleteOperatorDescriptor.java
@@ -48,8 +48,8 @@
ITupleFilterFactory tupleFilterFactory, IModificationOperationCallbackFactory modificationOpCallbackProvider) {
super(spec, 1, 1, recDesc, storageManager, lifecycleManagerProvider, fileSplitProvider, typeTraits,
comparatorFactories, bloomFilterKeyFields, dataflowHelperFactory, tupleFilterFactory, false,
- NoOpLocalResourceFactoryProvider.INSTANCE, NoOpOperationCallbackFactory.INSTANCE,
- modificationOpCallbackProvider);
+ false, null,
+ NoOpLocalResourceFactoryProvider.INSTANCE, NoOpOperationCallbackFactory.INSTANCE, modificationOpCallbackProvider);
this.fieldPermutation = fieldPermutation;
this.op = op;
}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/AbstractLSMInvertedIndexOperatorDescriptor.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/AbstractLSMInvertedIndexOperatorDescriptor.java
index 675fb03..2484511 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/AbstractLSMInvertedIndexOperatorDescriptor.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/AbstractLSMInvertedIndexOperatorDescriptor.java
@@ -16,6 +16,7 @@
package edu.uci.ics.hyracks.storage.am.lsm.invertedindex.dataflow;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
@@ -48,13 +49,13 @@
ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenComparatorFactories,
ITypeTraits[] invListsTypeTraits, IBinaryComparatorFactory[] invListComparatorFactories,
IBinaryTokenizerFactory tokenizerFactory, IIndexDataflowHelperFactory dataflowHelperFactory,
- ITupleFilterFactory tupleFilterFactory, boolean retainInput,
- ILocalResourceFactoryProvider localResourceFactoryProvider,
+ ITupleFilterFactory tupleFilterFactory, boolean retainInput, boolean retainNull,
+ INullWriterFactory nullWriterFactory, ILocalResourceFactoryProvider localResourceFactoryProvider,
ISearchOperationCallbackFactory searchOpCallbackFactory,
IModificationOperationCallbackFactory modificationOpCallbackFactory) {
super(spec, inputArity, outputArity, recDesc, storageManager, lifecycleManagerProvider, fileSplitProvider,
- dataflowHelperFactory, tupleFilterFactory, retainInput, localResourceFactoryProvider,
- searchOpCallbackFactory, modificationOpCallbackFactory);
+ dataflowHelperFactory, tupleFilterFactory, retainInput, retainNull, nullWriterFactory,
+ localResourceFactoryProvider, searchOpCallbackFactory, modificationOpCallbackFactory);
this.invListsTypeTraits = invListsTypeTraits;
this.invListComparatorFactories = invListComparatorFactories;
this.tokenTypeTraits = tokenTypeTraits;
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexBulkLoadOperatorDescriptor.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexBulkLoadOperatorDescriptor.java
index 1c4235c..1f3ed76 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexBulkLoadOperatorDescriptor.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexBulkLoadOperatorDescriptor.java
@@ -50,8 +50,8 @@
IModificationOperationCallbackFactory modificationOpCallbackFactory) {
super(spec, 1, 0, null, storageManager, fileSplitProvider, lifecycleManagerProvider, tokenTypeTraits,
tokenComparatorFactories, invListsTypeTraits, invListComparatorFactories, tokenizerFactory,
- invertedIndexDataflowHelperFactory, null, false, NoOpLocalResourceFactoryProvider.INSTANCE,
- NoOpOperationCallbackFactory.INSTANCE, modificationOpCallbackFactory);
+ invertedIndexDataflowHelperFactory, null, false, false,
+ null, NoOpLocalResourceFactoryProvider.INSTANCE, NoOpOperationCallbackFactory.INSTANCE, modificationOpCallbackFactory);
this.fieldPermutation = fieldPermutation;
this.verifyInput = verifyInput;
this.numElementsHint = numElementsHint;
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexCompactOperator.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexCompactOperator.java
index 22e7505..d24a1be 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexCompactOperator.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexCompactOperator.java
@@ -43,8 +43,8 @@
IModificationOperationCallbackFactory modificationOpCallbackFactory) {
super(spec, 1, 1, null, storageManager, fileSplitProvider, lifecycleManagerProvider, tokenTypeTraits,
tokenComparatorFactories, invListsTypeTraits, invListComparatorFactories, tokenizerFactory,
- dataflowHelperFactory, null, false, NoOpLocalResourceFactoryProvider.INSTANCE,
- NoOpOperationCallbackFactory.INSTANCE, modificationOpCallbackFactory);
+ dataflowHelperFactory, null, false, false,
+ null, NoOpLocalResourceFactoryProvider.INSTANCE, NoOpOperationCallbackFactory.INSTANCE, modificationOpCallbackFactory);
}
@Override
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexCreateOperatorDescriptor.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexCreateOperatorDescriptor.java
index a67ae51..b5fd8c4 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexCreateOperatorDescriptor.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexCreateOperatorDescriptor.java
@@ -45,8 +45,8 @@
IModificationOperationCallbackFactory modificationOpCallbackFactory) {
super(spec, 0, 0, null, storageManager, fileSplitProvider, lifecycleManagerProvider, tokenTypeTraits,
tokenComparatorFactories, invListsTypeTraits, invListComparatorFactories, tokenizerFactory,
- btreeDataflowHelperFactory, null, false, localResourceFactoryProvider,
- NoOpOperationCallbackFactory.INSTANCE, modificationOpCallbackFactory);
+ btreeDataflowHelperFactory, null, false, false,
+ null, localResourceFactoryProvider, NoOpOperationCallbackFactory.INSTANCE, modificationOpCallbackFactory);
}
@Override
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexInsertUpdateDeleteOperator.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexInsertUpdateDeleteOperator.java
index 31797a7..550166d 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexInsertUpdateDeleteOperator.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexInsertUpdateDeleteOperator.java
@@ -50,8 +50,8 @@
ITupleFilterFactory tupleFilterFactory, IModificationOperationCallbackFactory modificationOpCallbackFactory) {
super(spec, 1, 1, recDesc, storageManager, fileSplitProvider, lifecycleManagerProvider, tokenTypeTraits,
tokenComparatorFactories, invListsTypeTraits, invListComparatorFactories, tokenizerFactory,
- dataflowHelperFactory, tupleFilterFactory, false, NoOpLocalResourceFactoryProvider.INSTANCE,
- NoOpOperationCallbackFactory.INSTANCE, modificationOpCallbackFactory);
+ dataflowHelperFactory, tupleFilterFactory, false, false,
+ null, NoOpLocalResourceFactoryProvider.INSTANCE, NoOpOperationCallbackFactory.INSTANCE, modificationOpCallbackFactory);
this.fieldPermutation = fieldPermutation;
this.op = op;
}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorDescriptor.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorDescriptor.java
index bd3f879..0b33aac 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorDescriptor.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorDescriptor.java
@@ -18,6 +18,7 @@
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
@@ -47,11 +48,13 @@
IBinaryComparatorFactory[] invListComparatorFactories,
IIndexDataflowHelperFactory btreeDataflowHelperFactory, IBinaryTokenizerFactory queryTokenizerFactory,
IInvertedIndexSearchModifierFactory searchModifierFactory, RecordDescriptor recDesc, boolean retainInput,
+ boolean retainNull, INullWriterFactory nullWriterFactory,
ISearchOperationCallbackFactory searchOpCallbackProvider) {
super(spec, 1, 1, recDesc, storageManager, fileSplitProvider, lifecycleManagerProvider, tokenTypeTraits,
tokenComparatorFactories, invListsTypeTraits, invListComparatorFactories, queryTokenizerFactory,
- btreeDataflowHelperFactory, null, retainInput, NoOpLocalResourceFactoryProvider.INSTANCE,
- searchOpCallbackProvider, NoOpOperationCallbackFactory.INSTANCE);
+ btreeDataflowHelperFactory, null, retainInput, retainNull, nullWriterFactory,
+ NoOpLocalResourceFactoryProvider.INSTANCE, searchOpCallbackProvider,
+ NoOpOperationCallbackFactory.INSTANCE);
this.queryField = queryField;
this.searchModifierFactory = searchModifierFactory;
}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorNodePushable.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorNodePushable.java
index ba3a7f1..7a88be3 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorNodePushable.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorNodePushable.java
@@ -57,4 +57,9 @@
invIndexSearchPred.setQueryTuple(frameTuple);
invIndexSearchPred.setQueryFieldIndex(queryFieldIndex);
}
+
+ @Override
+ protected int getFieldCount() {
+ return invListFields;
+ }
}
diff --git a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java
index f1f5a71..4498e6c 100644
--- a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java
+++ b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java
@@ -18,6 +18,7 @@
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
@@ -41,13 +42,12 @@
IStorageManagerInterface storageManager, IIndexLifecycleManagerProvider lifecycleManagerProvider,
IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
IBinaryComparatorFactory[] comparatorFactories, int[] keyFields,
- IIndexDataflowHelperFactory dataflowHelperFactory, boolean retainInput,
- ISearchOperationCallbackFactory searchOpCallbackFactory) {
+ IIndexDataflowHelperFactory dataflowHelperFactory, boolean retainInput, boolean retainNull,
+ INullWriterFactory nullWriterFactory, ISearchOperationCallbackFactory searchOpCallbackFactory) {
super(spec, 1, 1, recDesc, storageManager, lifecycleManagerProvider, fileSplitProvider, typeTraits,
- comparatorFactories, null, dataflowHelperFactory, null, retainInput,
+ comparatorFactories, null, dataflowHelperFactory, null, retainInput, retainNull, nullWriterFactory,
NoOpLocalResourceFactoryProvider.INSTANCE, searchOpCallbackFactory,
NoOpOperationCallbackFactory.INSTANCE);
-
this.keyFields = keyFields;
}
diff --git a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java
index 9cfdbf3..acb3ece 100644
--- a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java
+++ b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java
@@ -52,4 +52,9 @@
searchKey.reset(accessor, tupleIndex);
}
}
+
+ @Override
+ protected int getFieldCount() {
+ return ((ITreeIndex)index).getFieldCount();
+ }
}
\ No newline at end of file
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGen.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGen.java
index bde94a9..109a91a 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGen.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGen.java
@@ -356,7 +356,7 @@
typeTraits[1] = new TypeTraits(false);
BTreeSearchOperatorDescriptor scanner = new BTreeSearchOperatorDescriptor(spec, recordDescriptor,
storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits, comparatorFactories, null,
- null, null, true, true, getIndexDataflowHelperFactory(), false, NoOpOperationCallbackFactory.INSTANCE);
+ null, null, true, true, getIndexDataflowHelperFactory(), false, false, null, NoOpOperationCallbackFactory.INSTANCE);
setLocationConstraint(spec, scanner);
/**
@@ -619,7 +619,7 @@
BTreeSearchOperatorDescriptor scanner = new BTreeSearchOperatorDescriptor(spec, recordDescriptor,
storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits, comparatorFactories, null,
- null, null, true, true, getIndexDataflowHelperFactory(), false, NoOpOperationCallbackFactory.INSTANCE);
+ null, null, true, true, getIndexDataflowHelperFactory(), false, false, null, NoOpOperationCallbackFactory.INSTANCE);
setLocationConstraint(spec, scanner);
ExternalSortOperatorDescriptor sort = null;
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenInnerJoin.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenInnerJoin.java
index a0da2a3..a728d48 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenInnerJoin.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenInnerJoin.java
@@ -681,7 +681,7 @@
BTreeSearchOperatorDescriptor scanner = new BTreeSearchOperatorDescriptor(spec, recordDescriptor,
storageManagerInterface, lcManagerProvider, secondaryFileSplitProviderRead, typeTraits,
comparatorFactories, null, null, null, true, true, getIndexDataflowHelperFactory(), false,
- NoOpOperationCallbackFactory.INSTANCE);
+ false, null, NoOpOperationCallbackFactory.INSTANCE);
setLocationConstraint(spec, scanner);
/**
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinFunctionUpdateOperatorDescriptor.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinFunctionUpdateOperatorDescriptor.java
index 3403afc..5890514 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinFunctionUpdateOperatorDescriptor.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinFunctionUpdateOperatorDescriptor.java
@@ -69,8 +69,8 @@
IRuntimeHookFactory preHookFactory, IRuntimeHookFactory postHookFactory, RecordDescriptor... rDescs) {
super(spec, 1, outputArity, rDescs[0], storageManager, lcManagerProvider, fileSplitProvider, typeTraits,
comparatorFactories, highKeyFields, opHelperFactory, null, false,
- new TransientLocalResourceFactoryProvider(), NoOpOperationCallbackFactory.INSTANCE,
- NoOpOperationCallbackFactory.INSTANCE);
+ false, null,
+ new TransientLocalResourceFactoryProvider(), NoOpOperationCallbackFactory.INSTANCE, NoOpOperationCallbackFactory.INSTANCE);
this.isForward = isForward;
this.lowKeyFields = lowKeyFields;
this.highKeyFields = highKeyFields;
@@ -99,8 +99,8 @@
IRuntimeHookFactory postHookFactory, RecordDescriptor... rDescs) {
super(spec, 1, outputArity, rDescs[0], storageManager, lcManagerProvider, fileSplitProvider, typeTraits,
comparatorFactories, highKeyFields, opHelperFactory, null, false,
- new TransientLocalResourceFactoryProvider(), NoOpOperationCallbackFactory.INSTANCE,
- NoOpOperationCallbackFactory.INSTANCE);
+ false, null,
+ new TransientLocalResourceFactoryProvider(), NoOpOperationCallbackFactory.INSTANCE, NoOpOperationCallbackFactory.INSTANCE);
this.isForward = isForward;
this.lowKeyFields = lowKeyFields;
this.highKeyFields = highKeyFields;
@@ -132,8 +132,8 @@
IRuntimeHookFactory postHookFactory, RecordDescriptor... rDescs) {
super(spec, 1, outputArity, rDescs[0], storageManager, lcManagerProvider, fileSplitProvider, typeTraits,
comparatorFactories, highKeyFields, opHelperFactory, null, false,
- new TransientLocalResourceFactoryProvider(), NoOpOperationCallbackFactory.INSTANCE,
- NoOpOperationCallbackFactory.INSTANCE);
+ false, null,
+ new TransientLocalResourceFactoryProvider(), NoOpOperationCallbackFactory.INSTANCE, NoOpOperationCallbackFactory.INSTANCE);
this.isForward = isForward;
this.lowKeyFields = lowKeyFields;
this.highKeyFields = highKeyFields;
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinOperatorDescriptor.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinOperatorDescriptor.java
index 440ae86..0f003ba 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinOperatorDescriptor.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinOperatorDescriptor.java
@@ -56,8 +56,8 @@
IBinaryComparatorFactory[] comparatorFactories, boolean isForward, int[] lowKeyFields, int[] highKeyFields,
boolean lowKeyInclusive, boolean highKeyInclusive, IIndexDataflowHelperFactory opHelperFactory) {
super(spec, 1, 1, recDesc, storageManager, lcManagerProvider, fileSplitProvider, typeTraits,
- comparatorFactories, null, opHelperFactory, null, false, new TransientLocalResourceFactoryProvider(),
- NoOpOperationCallbackFactory.INSTANCE, NoOpOperationCallbackFactory.INSTANCE);
+ comparatorFactories, null, opHelperFactory, null, false, false,
+ null, new TransientLocalResourceFactoryProvider(), NoOpOperationCallbackFactory.INSTANCE, NoOpOperationCallbackFactory.INSTANCE);
this.isForward = isForward;
this.lowKeyFields = lowKeyFields;
this.highKeyFields = highKeyFields;
@@ -73,8 +73,8 @@
boolean lowKeyInclusive, boolean highKeyInclusive, IIndexDataflowHelperFactory opHelperFactory,
boolean isRightOuter, INullWriterFactory[] nullWriterFactories) {
super(spec, 1, 1, recDesc, storageManager, lcManagerProvider, fileSplitProvider, typeTraits,
- comparatorFactories, null, opHelperFactory, null, false, new TransientLocalResourceFactoryProvider(),
- NoOpOperationCallbackFactory.INSTANCE, NoOpOperationCallbackFactory.INSTANCE);
+ comparatorFactories, null, opHelperFactory, null, false, false,
+ null, new TransientLocalResourceFactoryProvider(), NoOpOperationCallbackFactory.INSTANCE, NoOpOperationCallbackFactory.INSTANCE);
this.isForward = isForward;
this.lowKeyFields = lowKeyFields;
this.highKeyFields = highKeyFields;
@@ -92,8 +92,8 @@
boolean lowKeyInclusive, boolean highKeyInclusive, IIndexDataflowHelperFactory opHelperFactory,
boolean isSetUnion) {
super(spec, 1, 1, recDesc, storageManager, lcManagerProvider, fileSplitProvider, typeTraits,
- comparatorFactories, null, opHelperFactory, null, false, new TransientLocalResourceFactoryProvider(),
- NoOpOperationCallbackFactory.INSTANCE, NoOpOperationCallbackFactory.INSTANCE);
+ comparatorFactories, null, opHelperFactory, null, false, false,
+ null, new TransientLocalResourceFactoryProvider(), NoOpOperationCallbackFactory.INSTANCE, NoOpOperationCallbackFactory.INSTANCE);
this.isForward = isForward;
this.lowKeyFields = lowKeyFields;
this.highKeyFields = highKeyFields;
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeIndexBulkReLoadOperatorDescriptor.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeIndexBulkReLoadOperatorDescriptor.java
index 4dd78c3..160e89a 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeIndexBulkReLoadOperatorDescriptor.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeIndexBulkReLoadOperatorDescriptor.java
@@ -44,8 +44,8 @@
ITypeTraits[] typeTraits, IBinaryComparatorFactory[] comparatorFactories, int[] fieldPermutation,
int[] bloomFilterFields, float fillFactor, IIndexDataflowHelperFactory opHelperFactory) {
super(spec, 1, 0, null, storageManager, lcManagerProvider, fileSplitProvider, typeTraits, comparatorFactories,
- bloomFilterFields, opHelperFactory, null, false, new TransientLocalResourceFactoryProvider(),
- NoOpOperationCallbackFactory.INSTANCE, NoOpOperationCallbackFactory.INSTANCE);
+ bloomFilterFields, opHelperFactory, null, false, false,
+ null, new TransientLocalResourceFactoryProvider(), NoOpOperationCallbackFactory.INSTANCE, NoOpOperationCallbackFactory.INSTANCE);
this.fieldPermutation = fieldPermutation;
this.storageManager = storageManager;
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeSearchFunctionUpdateOperatorDescriptor.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeSearchFunctionUpdateOperatorDescriptor.java
index 31496a9..6cdac98 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeSearchFunctionUpdateOperatorDescriptor.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeSearchFunctionUpdateOperatorDescriptor.java
@@ -62,8 +62,8 @@
IRuntimeHookFactory preHookFactory, IRuntimeHookFactory postHookFactory, RecordDescriptor... rDescs) {
super(spec, 1, outputArity, recDesc, storageManager, lcManagerProvider, fileSplitProvider, typeTraits,
comparatorFactories, null, dataflowHelperFactory, null, false,
- new TransientLocalResourceFactoryProvider(), NoOpOperationCallbackFactory.INSTANCE,
- NoOpOperationCallbackFactory.INSTANCE);
+ false, null,
+ new TransientLocalResourceFactoryProvider(), NoOpOperationCallbackFactory.INSTANCE, NoOpOperationCallbackFactory.INSTANCE);
this.isForward = isForward;
this.lowKeyFields = lowKeyFields;
this.highKeyFields = highKeyFields;