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;